Purpose of mute/unmute logic in SocketServer

2017-11-14 Thread Javed, Haseeb
Hello, I was going through the SocketServer code to understand the networking internals of Kafka code base. I see that at some points in the Processor, we mute/unmute certain Kafka channels. These mute/unmute methods seem to be a wrapper to enable/disbale reading new data from respective chann

[GitHub] kafka pull request #4216: KAFKA-5859: Avoid retaining AbstractRequest in Req...

2017-11-14 Thread seglo
GitHub user seglo opened a pull request: https://github.com/apache/kafka/pull/4216 KAFKA-5859: Avoid retaining AbstractRequest in RequestChannel.Response This PR removes the need to keep a reference to the parsed `AbstractRequest` after it's been handled in `KafkaApis`. A reference

[GitHub] kafka pull request #4215: Kafka 6121 restore global consumer handle reset

2017-11-14 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/4215 Kafka 6121 restore global consumer handle reset - set auto.offset.reste to "none" for restore and global consumer - handle InvalidOffsetException for restore and global consumer - add correspon

[DISCUSS] Hide internal topics' directories

2017-11-14 Thread Hu Xi
Hi all, Many new Kafka users are confused about the fact there are a lot of sub-directories named '__consumer_offsets-***' or '__transaction_state-***' under the broker data directory and ask if they can be removed. Do you guys think it can be beneficial to hide those internal topics' directo

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

2017-11-14 Thread Jun Rao
Hi, Charly, Thanks for KIP-225. Your proposal looks reasonable. Hi, Jiangjie, Do you think the approach that KIP-225 proposes is better for exposing the per partition metric? Also, do we really need the per partition record-lag-avg and record-lag-max? It seems that an external monitoring system

[GitHub] kafka pull request #4214: MINOR: Make PushHttpMetricsReporter API compatible...

2017-11-14 Thread ewencp
GitHub user ewencp opened a pull request: https://github.com/apache/kafka/pull/4214 MINOR: Make PushHttpMetricsReporter API compatible with releases back to 0.8.2.2 This is follow up to #4072 which added the PushHttpMetricsReporter and converted some services to use it. We somehow

[GitHub] kafka pull request #4213: KAFKA-4115: Increasing the heap settings for conne...

2017-11-14 Thread wicknicks
GitHub user wicknicks opened a pull request: https://github.com/apache/kafka/pull/4213 KAFKA-4115: Increasing the heap settings for connect-distributed script Signed-off-by: Arjun Satish You can merge this pull request into a Git repository by running: $ git pull https://githu

Re: [DISCUSS] KIP-219 - Improve Quota Communication

2017-11-14 Thread Becket Qin
Hi Rajini, We are using SSL so we could use user quota. But I am not sure if that would solve the problem. The key issue in our case is that each broker can only handle ~300 MB/s of incoming bytes, but the MapReduce job is trying to push 1-2 GB/s, unless we can throttle the clients to 300 MB/s, th

[GitHub] kafka pull request #4212: MINOR: Introduction fixup

2017-11-14 Thread joel-hamill
GitHub user joel-hamill opened a pull request: https://github.com/apache/kafka/pull/4212 MINOR: Introduction fixup *Clarify multi-tenant support, geo-replication, and some grammar fixes.* ### Committer Checklist (excluded from commit message) - [ ] Verify design and impl

Re: [DISCUSS] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-14 Thread Matthias J. Sax
One more thing. Can you update the KIP accordingly. It still says: - Compatibility check: we will use a network client for this purpose, as it is a one-time thing. Additionally, I think we should add a "admin." prefix that allows to set certain config parameters for the admin client only. Simila

Re: [DISCUSS] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-14 Thread Matthias J. Sax
Thanks for looking into this into details! As mentioned, I would like to keep the check, but if it's too much overhead, I agree that it's not worth it. Thanks. -Matthias On 11/14/17 10:00 AM, Guozhang Wang wrote: > I looked into how to use a NetworkClient to replace StreamsKafkaClient to > do t

[GitHub] kafka pull request #4211: [WIP] KAFKA-6170: Add AdminClient to Streams

2017-11-14 Thread guozhangwang
GitHub user guozhangwang opened a pull request: https://github.com/apache/kafka/pull/4211 [WIP] KAFKA-6170: Add AdminClient to Streams 1. Add The AdminClient into Kafka Streams, which is shared among all the threads. 2. Refactored mutual dependency between StreamPartitionAssigno

[GitHub] kafka pull request #4210: KAFKA-6167: Timestamp on streams directory contain...

2017-11-14 Thread mjsax
GitHub user mjsax opened a pull request: https://github.com/apache/kafka/pull/4210 KAFKA-6167: Timestamp on streams directory contains a colon, which is an illegal character - change segment delimiter to . - added upgrade path - added test for old and new upgrade path Y

[jira] [Created] (KAFKA-6209) Lag is inconsistent when manually committing offset for transactionnal messages

2017-11-14 Thread Fabien Chaillou (JIRA)
Fabien Chaillou created KAFKA-6209: -- Summary: Lag is inconsistent when manually committing offset for transactionnal messages Key: KAFKA-6209 URL: https://issues.apache.org/jira/browse/KAFKA-6209 Pro

Re: [VOTE] KIP-224: Add configuration parameters `retries` to Streams API

2017-11-14 Thread Guozhang Wang
+1 On Tue, Nov 14, 2017 at 3:02 AM, Damian Guy wrote: > +1 > > On Tue, 14 Nov 2017 at 02:40 Bill Bejeck wrote: > > > Thanks for the KIP, +1 > > > > -Bill > > > > On Mon, Nov 13, 2017 at 7:25 PM, Ted Yu wrote: > > > > > +1 > > > > > > On Mon, Nov 13, 2017 at 4:20 PM, Matthias J. Sax < > matth..

Re: [DISCUSS] KIP-211: Revise Expiration Semantics of Consumer Group Offsets

2017-11-14 Thread Vahid S Hashemian
Thanks Jeff. I'll wait until EOD tomorrow (Wednesday), and then I'll start a vote. --Vahid From: Jeff Widman To: dev@kafka.apache.org Date: 11/14/2017 11:35 AM Subject:Re: [DISCUSS] KIP-211: Revise Expiration Semantics of Consumer Group Offsets Any other input on this? Ot

Re: [DISCUSS] KIP-211: Revise Expiration Semantics of Consumer Group Offsets

2017-11-14 Thread Jeff Widman
Any other input on this? Otherwise Vahid what do you think about moving this to a vote? On Tue, Nov 7, 2017 at 2:34 PM, Jeff Widman wrote: > Any other feedback from folks on KIP-211? > > A prime benefit of this KIP is that it removes the need for the consumer > to commit offsets for partitions

Re: [VOTE] 0.11.0.2 RC0

2017-11-14 Thread Gwen Shapira
+1 (binding) Validated signatures, compiled sources, ran through quickstart on my machine. On Mon, Nov 13, 2017 at 7:53 PM Ted Yu wrote: > Understood. > > The test passed when run standalone. > > +1 from me. > > On Mon, Nov 13, 2017 at 1:19 AM, Rajini Sivaram > wrote: > > > Hi Ted, > > > > Tha

[jira] [Created] (KAFKA-6208) Reduce startup time for Kafka Connect workers

2017-11-14 Thread Randall Hauch (JIRA)
Randall Hauch created KAFKA-6208: Summary: Reduce startup time for Kafka Connect workers Key: KAFKA-6208 URL: https://issues.apache.org/jira/browse/KAFKA-6208 Project: Kafka Issue Type: Impro

Re: [DISCUSS] KIP-212: Enforce set of legal characters for connector names

2017-11-14 Thread Randall Hauch
Thanks for updating the KIP to reflect the current process. However, I still question whether it is necessary to have a KIP - it depends on whether it was possible with prior versions to have connectors with zero-length or blank names. Have you tried both of these cases? On Fri, Nov 10, 2017 at 3:

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

2017-11-14 Thread Jorge Esteban Quilcate Otoya
Added. El mar., 14 nov. 2017 a las 19:00, Ted Yu () escribió: > Please fill in JIRA number in Status section. > > On Tue, Nov 14, 2017 at 9:57 AM, Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > > > JIRA issue title updated. > > > > El mar., 14 nov. 2017 a las 18:45, Ted Yu ()

[VOTE] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-14 Thread Guozhang Wang
Hello folks, I have filed a new KIP on adding AdminClient into Streams for internal topic management. Please review and cast your vote on this thread. *https://cwiki.apache.org/confluence/display/KAFKA/KIP-220%3A+Add+AdminClient+into+Kafka+Streams%27+ClientSupplier

Re: [DISCUSS] KIP-220: Add AdminClient into Kafka Streams' ClientSupplier

2017-11-14 Thread Guozhang Wang
I looked into how to use a NetworkClient to replace StreamsKafkaClient to do this one-time checking, and the complexity is actually pretty high: since it is a barebone NetworkClient, we have to handle the connection / readiness / find a broker to send to / etc logic plus introducing all these depen

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

2017-11-14 Thread Ted Yu
Please fill in JIRA number in Status section. On Tue, Nov 14, 2017 at 9:57 AM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > JIRA issue title updated. > > El mar., 14 nov. 2017 a las 18:45, Ted Yu () > escribió: > > > Can you fill in JIRA number (KAFKA-6058 > >

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

2017-11-14 Thread Jorge Esteban Quilcate Otoya
JIRA issue title updated. El mar., 14 nov. 2017 a las 18:45, Ted Yu () escribió: > Can you fill in JIRA number (KAFKA-6058 > ) ? > > If one JIRA is used for the two additions, consider updating the JIRA > title. > > On Tue, Nov 14, 2017 at 9:04 AM

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

2017-11-14 Thread Ted Yu
Can you fill in JIRA number (KAFKA-6058 ) ? If one JIRA is used for the two additions, consider updating the JIRA title. On Tue, Nov 14, 2017 at 9:04 AM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Hi all, > > As I didn't se

[VOTE] KIP-222 - Add "describe consumer group" to KafkaAdminClient

2017-11-14 Thread Jorge Esteban Quilcate Otoya
Hi all, As I didn't see any further discussion around this KIP, I'd like to start voting. KIP documentation: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=74686265 Cheers, Jorge.

[jira] [Created] (KAFKA-6207) Include start of record when RecordIsTooLarge

2017-11-14 Thread Tadhg Pearson (JIRA)
Tadhg Pearson created KAFKA-6207: Summary: Include start of record when RecordIsTooLarge Key: KAFKA-6207 URL: https://issues.apache.org/jira/browse/KAFKA-6207 Project: Kafka Issue Type: Bug

Re: [DISCUSS] KIP-219 - Improve Quota Communication

2017-11-14 Thread Rajini Sivaram
Hi Becket, For the specific scenario that you described, would it be possible to use user quotas rather than client-id quotas? With user quotas, perhaps we can throttle more easily before reading requests as well (as you mentioned, the difficulty with client-id quota is that we have to read partia

Re: [DISCUSS] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-11-14 Thread Matt Farmer
I responded before reading your code review and didn't see the bit about how ProducerFencedException is self-healing. This error handling logic is *quite* confusing to reason about... I think I'm going to sit down with the code a bit more today, but I'm inclined to think that if the fenced exceptio

Re: [DISCUSS] KIP-210: Provide for custom error handling when Kafka Streams fails to produce

2017-11-14 Thread Matt Farmer
Hi there, Following up here... > One tiny comment: I would prefer to remove the "Always" from the handler > implementation names -- it sounds "cleaner" to me without it. Let me think on this. I generally prefer expressiveness to clean-ness, and I think that comes out in the names I chose for th

[GitHub] kafka pull request #4209: Add cell-sec entry point

2017-11-14 Thread iso88592
Github user iso88592 closed the pull request at: https://github.com/apache/kafka/pull/4209 ---

[GitHub] kafka pull request #4209: Add cell-sec entry point

2017-11-14 Thread iso88592
GitHub user iso88592 opened a pull request: https://github.com/apache/kafka/pull/4209 Add cell-sec entry point Change-Id: I3daa7bba925b9f052cec6de0dd385c98292ca45a Entry points for cell based security You can merge this pull request into a Git repository by running: $

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

2017-11-14 Thread charly molter
Hi Jun, Hu, I have KIP-225 open for adding tags to records-lag: https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=74686649 I have a patch more or less ready so I could probably get the fix checked in (after the vote) and you could build on top of it. Otherwise we could merge both K

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

2017-11-14 Thread Hu Xi
Jun, Let me double confirm with your comments: 1 remove partition-level records-lead-avg and records-lead-min since they both can be deduced by external monitoring system. 2 Tag partition-level records-lead with topic&partition info If they are the case you expect, do we need to do the same

Re: [VOTE] KIP-171 - Extend Consumer Group Reset Offset for Stream Application

2017-11-14 Thread Jorge Esteban Quilcate Otoya
Thanks to everyone for your feedback. KIP has been accepted and discussion is moved to PR. Cheers, Jorge. El lun., 6 nov. 2017 a las 17:31, Rajini Sivaram () escribió: > +1 (binding) > Thanks for the KIP, Jorge. > > Regards, > > Rajini > > On Tue, Oct 31, 2017 at 9:58 AM, Damian Guy wrote: >

Re: [VOTE] KIP-224: Add configuration parameters `retries` to Streams API

2017-11-14 Thread Damian Guy
+1 On Tue, 14 Nov 2017 at 02:40 Bill Bejeck wrote: > Thanks for the KIP, +1 > > -Bill > > On Mon, Nov 13, 2017 at 7:25 PM, Ted Yu wrote: > > > +1 > > > > On Mon, Nov 13, 2017 at 4:20 PM, Matthias J. Sax > > wrote: > > > > > Hi @all, > > > > > > I would like to start the vote for KIP-224: > > >