Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-10-09 Thread Joel Koshy
gt; >> > > >> > > > > > > >> >> > > >> > > > > > > >> >> > > >> > > > > > > >> >> > > >> > > > > > > >> >> > > >> > > >

Re: [VOTE] KIP-354 Time-based log compaction policy

2018-11-09 Thread Joel Koshy
+1 with one suggestion on the proposed metric. You should probably include the unit. So for e.g., max-compaction-delay-secs. Joel On Tue, Nov 6, 2018 at 5:30 PM xiongqi wu wrote: > bump > Xiongqi (Wesley) Wu > > > On Thu, Sep 27, 2018 at 4:20 PM xiongqi wu wrote: > > > > > Thanks Eno, Brett, D

Re: [VOTE] KIP-354 Time-based log compaction policy

2018-12-06 Thread Joel Koshy
; > > > wrote: > > > > > > > > > > > > > > > Thanks Dong. > > > > > > > > I have updated the KIP. > > > > > > > > > > > > > > > > Xiongqi (Wesley) Wu > > > >

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-17 Thread Joel Koshy
Hey Becket - good point. Lucas and I were talking about this offline last week. It is true that there is only one request in flight for processing. However, there may be more during a controller failover but it should not be very high - basically the maximum number of controller failures that can o

Re: [VOTE] KIP-291: Have separate queues for control requests and data requests

2018-07-17 Thread Joel Koshy
+1 on the KIP. (I'm not sure we actually necessary to introduce the condition variables for the concern that Jun raised, but it's an implementation detail that we can defer to a discussion in the PR.) On Sat, Jul 14, 2018 at 10:45 PM, Lucas Wang wrote: > Hi Jun, > > I agree by using the conditi

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Joel Koshy
@Mayuresh - I like your idea. It appears to be a simpler less invasive alternative and it should work. Jun/Becket/others, do you see any pitfalls with this approach? On Wed, Jul 18, 2018 at 12:03 PM, Lucas Wang wrote: > @Mayuresh, > That's a very interesting idea that I haven't thought before. >

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-07-18 Thread Joel Koshy
est > queue. > > 3. Controller to broker connection failed and the controller reconnected > to > > the broker. > > 4. Controller sends a request R2 to the broker > > 5. Broker receives R2 and add it to the head of the request queue. > > Now on the brok

Re: [VOTE] KIP-340: Allow kafka-reassign-partitions.sh and kafka-log-dirs.sh to take admin client property file

2018-07-20 Thread Joel Koshy
+1 On Tue, Jul 17, 2018 at 1:42 PM, Ted Yu wrote: > +1 > > On Tue, Jul 17, 2018 at 1:40 PM Jason Gustafson > wrote: > > > +1. This is useful (though the naming inconsistencies in the tools are > > vexing, as always). > > > > -Jason > > > > On Tue, Jul 17, 2018 at 12:24 PM, Dong Lin wrote: > >

Re: [DISCUSS] KIP-291: Have separate queues for control requests and data requests

2018-08-24 Thread Joel Koshy
I had some offline discussions with Lucas on this KIP. While it is much more work than the original proposals, separating the control plane entirely removes any interference with the data plane as summarized under the rejected alternatives section. Just a few minor comments: - Can you update t

Streams Meetup at LinkedIn Sunnyvale, 6pm, Wednesday, March 20, 2019

2019-03-07 Thread Joel Koshy
*[bcc: (users,dev)@kafka.apache.org ]* Hi everyone, The Streams Infrastructure team at LinkedIn invites you to attend a Streams Processing meetup on Wednesday, March 20 at LinkedIn’s Sunnyvale campus. (This meetup focuses on Apache Kafka, Apache Samza, and related streami

Re: [DISCUSS] KIP-168: Add TotalTopicCount metric per cluster

2017-06-26 Thread Joel Koshy
+1 on the original KIP I actually prefer TotalTopicCount because it makes it clearer that it is a cluster-wide count. OfflinePartitionsCount is global to the cluster (but it is fairly clear that the controller is SoT on that). TopicCount on the other hand could be misread as a local count since Par

Re: [VOTE] KIP-168: Add TotalTopicCount metric per cluster

2017-07-13 Thread Joel Koshy
+1 On Thu, Jul 13, 2017 at 12:24 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > +1 (non-binding) > > > > > From: Dong Lin > To: dev@kafka.apache.org > Date: 07/12/2017 10:43 AM > Subject:Re: [VOTE] KIP-168: Add TotalTopicCount metric per cluster > > > > +1 (non-bindi

Re: [VOTE] KIP-164 Add unavailablePartitionCount and per-partition Unavailable metrics

2017-07-25 Thread Joel Koshy
+1 On Thu, Jul 20, 2017 at 10:30 AM, Becket Qin wrote: > +1, Thanks for the KIP. > > On Thu, Jul 20, 2017 at 7:08 AM, Ismael Juma wrote: > > > Thanks for the KIP, +1 (binding). > > > > On Thu, Jun 1, 2017 at 9:44 AM, Dong Lin wrote: > > > > > Hi all, > > > > > > Can you please vote for KIP-164

[ANNOUNCE] New Kafka PMC member: Jiangjie (Becket) Qin

2017-08-23 Thread Joel Koshy
Hi everyone, Jiangjie (Becket) Qin has been a Kafka committer in October 2016 and has contributed significantly to several major patches, reviews and discussions since. I am glad to announce that Becket is now a member of the Apache Kafka PMC. Congratulations Becket! Joel

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 fe

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 investigati

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 pos

Kafka/Samza meetup at LinkedIn, June 15, 6pm

2016-05-27 Thread Joel Koshy
[bcc: us...@kafka.apache.org, dev@kafka.apache.org] Hi everyone, We would like to invite you to our first Stream Processing Meetup at LinkedIn on June 15 at 6pm. Please RSVP here: http://www.meetup.com/Stream-Processing-Meetup-LinkedIn/events/231454378 Going forward (at LinkedIn) we will host me

Re: [kafka-clients] [VOTE] 0.10.0.1 RC2

2016-08-05 Thread Joel Koshy
+1 (binding) Thanks Ismael! On Thu, Aug 4, 2016 at 6:54 AM, Ismael Juma wrote: > Hello Kafka users, developers and client-developers, > > This is the third candidate for the release of Apache Kafka 0.10.0.1. This > is a bug fix release and it includes fixes and improvements from 53 JIRAs > (inc

Re: [DISCUSS] KIP-72 Allow Sizing Incoming Request Queue in Bytes

2016-08-08 Thread Joel Koshy
> > > > 2. Good point about the consequence when the processor threads are > blocking. I agree it would be important to keep the processor thread > running, but I am not sure if it would be a problem of the current > proposal. In most of the time, the request queue should be close to empty, > so th

Re: [DISCUSS] KIP-72 Allow Sizing Incoming Request Queue in Bytes

2016-08-08 Thread Joel Koshy
> > . >> >> > Hi Becket, > > I don't think progress can be made in the processor's run loop if the > queue fills up. i.e., I think Jun's point is that if the queue is full > (either due to the proposed max.bytes or today due to max.requests hitting > the limit) then processCompletedReceives will bl

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-08 Thread Joel Koshy
Hi Ben, Thanks for the detailed write-up. So the proposal involves self-throttling on the fetcher side and throttling at the leader. Can you elaborate on the reasoning that is given on the wiki: *“The throttle is applied to both leaders and followers. This allows the admin to exert strong guarante

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-09 Thread Joel Koshy
e issue with approach (1) is that it departs somewhat from the existing > > quotas implementation, and must include a notion of fairness within, the > > now size-bounded, request and response. The issue with (2) is > guaranteeing > > ordering of updates when replicas shift threads, but this

Re: [DISCUSS] KIP-73: Replication Quotas

2016-08-09 Thread Joel Koshy
with an example how this would be handled : > > *guaranteeing > > ordering of updates when replicas shift threads* > > > > Thanks, > > > > Mayuresh > > > > > > On Tue, Aug 9, 2016 at 10:49 AM, Joel Koshy wrote: > > > > &g

Re: Consumer Offset Migration Tool

2016-08-09 Thread Joel Koshy
There is an old jira on getting the old import/export ZK offsets tool to work for Kafka-based offsets . Ideally, that should be done in the kafka-consumer-groups tool. We could have kafka-consumer-groups tool support import/export for ZK-based offse

Re: [VOTE] KIP-15 add a close with timeout to new producer

2015-03-26 Thread Joel Koshy
Talked to Jiangjie offline - actually looking at the code, we could just extend java.lang.Error. Alternately we could throw an IllegalArgumentException and though we catch Exception, we could catch that explicitly and rethrow to cause the sender to just exit. On Thu, Mar 26, 2015 at 01:29:41PM -07

Re: [VOTE] KIP-15 add a close with timeout to new producer

2015-03-26 Thread Joel Koshy
the user won't see it unless they check the logs, which > is the same as logging the error. > > Is there a problem with just logging an error and then blocking for the > amount of time requested? > > -Jay > > On Thu, Mar 26, 2015 at 2:03 PM, Joel Koshy wrote: > >

Re: Review Request 27391: Rebase patch and incorporate Joel's comments

2015-03-26 Thread Joel Koshy
explicit commit timestamp is provided we calculate retention from that explicit commit timestamp // - If v2 we use the default expiration timestamp ``` - Joel Koshy On March 26, 2015, 7:28 p.m., Guozhang Wang wrote: > > --

Re: [DISCUSSION] Keep docs updated per jira

2015-03-26 Thread Joel Koshy
+1 It is indeed too easy to forget and realize only much later that a jira needed a doc update. So getting into the habit of asking "did you update the docs" as part of review will definitely help. On Thu, Mar 26, 2015 at 06:36:43PM -0700, Gwen Shapira wrote: > I strongly support the goal of keep

Re: [VOTE] KIP-15 Add a close method with timeout to new producer

2015-03-27 Thread Joel Koshy
+1 On Fri, Mar 27, 2015 at 01:34:54PM -0700, Neha Narkhede wrote: > +1 > > Thanks Becket! > > On Fri, Mar 27, 2015 at 1:19 PM, Jay Kreps wrote: > > > +1 > > > > Thanks for your patience Jiangjie! > > > > -Jay > > > > On Fri, Mar 27, 2015 at 11:48 AM, Jiangjie Qin > > wrote: > > > > > > > > >

Re: Metrics package discussion

2015-03-27 Thread Joel Koshy
For the samples: it will be at least double that estimate I think since the long array contains (eight byte) references to the actual longs, each of which also have some object overhead. Re: testing: actually, it looks like YM metrics does allow you to drop in your own clock: https://github.com/dr

Re: KIP discussion Mar 24 at 11am PST

2015-03-27 Thread Joel Koshy
I think we decided that we should reconvene next Tuesday to collect everyone's thoughts on metrics and discuss other KIPs. Can everyone still do Tuesday or do people feel we need to push that further out by a few days? Thanks, Joel On Tue, Mar 24, 2015 at 12:28:04PM -0700, Jun Rao wrote: > Just

Re: [DISCUSSION] Keep docs updated per jira

2015-03-30 Thread Joel Koshy
Also for the wikis - those should probably correspond to the latest released version right? So for e.g., if we add or modify the protocol on trunk we can add it to the wiki but mark it with some highlight or similar just to make it clear that it is a change on trunk only. Thanks, Joel On Thu, Ma

Re: KIP Hangout - March 31, 2015 - TL;DR;

2015-03-31 Thread Joel Koshy
We will be using KM for quota'ing on the new client-id-specific metrics. On Tue, Mar 31, 2015 at 08:44:44PM +, Jiangjie Qin wrote: > Thanks a lot for the summary, Gwen! > About the Quota, does that mean the first quota implementation will be > based on YM? I¹m thinking can we pursue a quota so

Re: Review Request 30809: Patch for KAFKA-1888

2015-03-31 Thread Joel Koshy
ache.org/r/30809/#comment127011> should these (and below) be in basedir? That said I don't see this created anywhere. - Joel Koshy On March 23, 2015, 6:54 p.m., Abhishek Nigam wrote: > >

Re: KIP Hangout - March 31, 2015 - TL;DR;

2015-03-31 Thread Joel Koshy
nks for clarification Joel. Just wondering if we are going to depend on > any KM specific features? Asking this because KM metric config has quota > in side it. > > On 3/31/15, 1:53 PM, "Joel Koshy" wrote: > > >We will be using KM for quota'ing on the new clien

Re: Review Request 28769: Patch for KAFKA-1809

2015-04-02 Thread Joel Koshy
? core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala <https://reviews.apache.org/r/28769/#comment127771> ", " system_test/replication_testsuite/testcase_0001/testcase_0001_properties.json <https://revi

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-03 Thread Joel Koshy
; From: Steven Wu [stevenz...@gmail.com] >> > Sent: Friday, March 20, 2015 9:14 AM >> > To: dev@kafka.apache.org >> > Subject: Re: [KIP-DISCUSSION] KIP-13 Quotas >> > >> > +1 on Jun's suggestion of maintaining one set/style of metrics at broker. >>

Re: Review Request 28769: Patch for KAFKA-1809

2015-04-03 Thread Joel Koshy
> On April 3, 2015, 12:41 a.m., Joel Koshy wrote: > > clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java, > > line 26 > > <https://reviews.apache.org/r/28769/diff/22/?file=914062#file914062line26> > > > > Can we go with TR

Re: Review Request 28769: Patch for KAFKA-1809

2015-04-03 Thread Joel Koshy
protocol enum in core, I think this looks good. Jun, do you want to review again? - Joel Koshy On April 3, 2015, 2:04 a.m., Gwen Shapira wrote: > > --- > This is an automatically generated e-mail. To reply, visit

Re: Review Request 31850: Patch for KAFKA-1660

2015-04-06 Thread Joel Koshy
Can you revert this? i.e., I think the previous version with locally declared accums is cleaner. core/src/test/scala/integration/kafka/api/ProducerSendTest.scala <https://reviews.apache.org/r/31850/#comment128171> Can you also add a test for calling close with a non-zero timeo

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-07 Thread Joel Koshy
implications of this choice, so as long as we do that I'm > > > happy. > > > > > > > > -Jay > > > > > > > > On Fri, Apr 3, 2015 at 1:10 PM, Aditya Auradkar < > > > > aaurad...@linkedin.com.invalid> wrote: > > > &g

Re: Review Request 32931: Revert the changes in NoOffset error code

2015-04-07 Thread Joel Koshy
8501> indicate -> indicates - Joel Koshy On April 7, 2015, 6:31 p.m., Guozhang Wang wrote: > > --- > This is an automatically generated e-mail. To reply, visit: &g

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-08 Thread Joel Koshy
> server-side > > > > > > versus in the clients to justify putting it in the protocol. But I > > > > think > > > > > > you guys may have other use cases in mind of how a client would > > make > > > > some > > > > > &

Re: jira backlogs

2015-04-08 Thread Joel Koshy
The weekly email sounds good, but I would much rather if that were integrated into the wiki/jira system than run a custom script on non-apache infra. We did something similar during 0.8 replication development where we embedded JQL links in the wiki and had a nice dashboard view of jiras on a wiki

Re: Review Request 31850: Patch for KAFKA-1660

2015-04-08 Thread Joel Koshy
the read lock. clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java <https://reviews.apache.org/r/31850/#comment128673> Similar comment as above. Once all accesses of closed are protected by the loc

Re: [VOTE] KIP-13 Quotas

2015-04-08 Thread Joel Koshy
Agreed on that - it is understood from the text of the KIP "Clients that send current version (0) of those requests will not receive the quota status flag in the response" => you have to send version (1) to get the new repsonse, but it should probably be clarified on the page. I'm +1 as well on th

Re: Review Request 33049: Patch for KAFKA-2084

2015-04-17 Thread Joel Koshy
/33049/ > --- > > (Updated April 15, 2015, 5:36 p.m.) > > > Review request for kafka and Joel Koshy. > > > Bugs: KAFKA-2084 > https://issues.apache.org/jira/browse/KAFKA-2084 > > > Repository: k

Re: [DISCUSS] KIP-11- Authorization design for kafka security

2015-04-20 Thread Joel Koshy
Hi Parth, Nice work on this KIP. I did another read through and had a few more comments (with edits after I went through the thread). Many of these comments were brought up by others as well, so it appears that the KIP would benefit from an update at this point to incorporate comments from the th

Re: Review Request 32650: Patch for KAFKA-2000

2015-04-20 Thread Joel Koshy
art-up there is no _guarantee_ that you have the most current information in the cache (say, if the controller failed to send an UpdateMetadataRequest to the broker by the time the compactor task runs) - Joel Koshy On March 30, 2015, 9:47 p.m., Sriharsha Chintalapani

Re: Review Request 32650: Patch for KAFKA-2000

2015-04-20 Thread Joel Koshy
> On April 20, 2015, 11:18 p.m., Joel Koshy wrote: > > core/src/main/scala/kafka/server/OffsetManager.scala, line 124 > > <https://reviews.apache.org/r/32650/diff/1/?file=909897#file909897line124> > > > > I think there is an issue in relying on the metadata

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-21 Thread Joel Koshy
; > From: Jun Rao [j...@confluent.io] > > > Sent: Thursday, April 09, 2015 8:59 AM > > > To: dev@kafka.apache.org > > > Subject: Re: [KIP-DISCUSSION] KIP-13 Quotas > > > > > > Since we are also thinking about evolving the fetch request protocol in

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-21 Thread Joel Koshy
rol be disabled/enabled without affect anything > else? From the design wiki page, it seems to me that each request will at > least pay a penalty of checking quota enablement. > > Thanks. > > Tong Li > OpenStack & Kafka Community Development > Building 501/B205 > liton.

Re: Review Request 33417: Patch for KAFKA-2138

2015-04-22 Thread Joel Koshy
che/kafka/clients/producer/internals/RecordAccumulatorTest.java <https://reviews.apache.org/r/33417/#comment131514> Node1 should be the only ready node - Joel Koshy On April 21, 2015, 10:51 p.m., Jiangjie Qin wrote: > > ---

Re: Review Request 33417: Patch for KAFKA-2138

2015-04-22 Thread Joel Koshy
> On April 21, 2015, 11:56 p.m., Guozhang Wang wrote: > > This piece of logic has been quite complex and awkward to me now, for > > example in ready() a node will only not be considered if ALL of its > > partitions are either not sendable or are in the backoff period, and the > > reason we wan

Re: Review Request 32650: Patch for KAFKA-2000

2015-04-23 Thread Joel Koshy
nds update metadata request first. - Go with the more conservative fix which is to purge on metadataCache.removePartitionInfo Also, we should add a unit test to verify offsets are in fact removed after deletion. - Joel Koshy On March 30, 2015, 9:47 p.m., Sriharsha Chintal

Re: Review Request 32650: Patch for KAFKA-2000

2015-04-23 Thread Joel Koshy
> On April 23, 2015, 9:51 p.m., Joel Koshy wrote: > > core/src/main/scala/kafka/server/OffsetManager.scala, line 124 > > <https://reviews.apache.org/r/32650/diff/1/?file=909897#file909897line124> > > > > A safer fix is to proactively purge as part of

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-24 Thread Joel Koshy
ts? > > -Jay > > > On Fri, Apr 24, 2015 at 2:18 PM, Jun Rao wrote: > > > Joel, > > > > What you suggested makes sense. Not sure if there is a strong need to > > throttle TMR though since it should be infrequent. > > > > Thanks, &g

Re: [KIP-DISCUSSION] KIP-13 Quotas

2015-04-24 Thread Joel Koshy
ing should take place? > > On Tue, Apr 21, 2015 at 4:38 PM, Joel Koshy wrote: > > > In either approach I'm not sure we considered being able to turn it > > off completely. IOW, no it is not a "plugin" if that's what you are > > asking. We can set very

Re: Perf testing flush() - issues found

2015-04-28 Thread Joel Koshy
- What is the record size? - Is this a local setup? i.e., producer/broker running local? - Any overrides apart from batch size? E.g., linger time. - Can you establish a baseline - with the old producer's sync-send? Thanks, Joel On Wed, Apr 29, 2015 at 12:58:43AM +, Roshan Naik wrote: > Based

Re: [DISCUSS] KIP-21 Configuration Management

2015-04-30 Thread Joel Koshy
>1. I have deep concerns about managing configuration in ZooKeeper. >First, Producers and Consumers shouldn't depend on ZK at all, this seems >to add back a dependency we are trying to get away from. The KIP probably needs to be clarified here - I don't think Aditya was referring to cl

Re: [DISCUSS] KIP-12 - Kafka Sasl/Kerberos implementation

2015-04-30 Thread Joel Koshy
Just went through this thread. I'm on-board with this as well. @Gwen - yes at LinkedIn we do need to support both authenticated/unauthenticated users on the same Kafka cluster because we cannot switch all clients simultaneously. I would be surprised if this is unique to LinkedIn. Also, I think th

Re: Review Request 33557: Patch for KAFKA-1936

2015-05-04 Thread Joel Koshy
have been called failed append. - Joel Koshy On April 27, 2015, 4:26 a.m., Dong Lin wrote: > > --- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-05 Thread Joel Koshy
Good discussion. Since we will be talking about this at 11am, I wanted to organize these comments into requirements to see if we are all on the same page. REQUIREMENT 1: Needs to accept dynamic config changes. This needs to be general enough to work for all configs that we envision may need to acc

Review Request 33916: Patch for KAFKA-2163

2015-05-06 Thread Joel Koshy
18680ce100f10035175cc0263ba7787ab0f6a17a Diff: https://reviews.apache.org/r/33916/diff/ Testing --- Thanks, Joel Koshy

Re: [KIP-DISCUSSION] KIP-22 Expose a Partitioner interface in the new producer

2015-05-06 Thread Joel Koshy
+1 with a minor comment: do we need an init method given it extends Configurable? Also, can you move this wiki out of drafts and add it to the table in https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals? Thanks, Joel On Wed, May 06, 2015 at 07:46:46AM -0700, Sriharsha

Re: Review Request 32650: Patch for KAFKA-2000

2015-05-06 Thread Joel Koshy
art of the issue is we have little to no test coverage on the controller. - Joel Koshy On May 3, 2015, 5:39 p.m., Sriharsha Chintalapani wrote: > > --- > This is an automatically generated e-mail.

Re: Review Request 33557: Patch for KAFKA-1936

2015-05-11 Thread Joel Koshy
and so on. The current code assumes everything is a meter (which it is) but the above may be clearer and makes fewer assumptions about the underlying metric types. It may also eliminate the need for the enumeration. What do you think? - Joel Koshy On May 4, 2015, 10:

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-11 Thread Joel Koshy
> Chef > > > > > for > > > > > > the config push. Not sure if Puppet works in a similar way. > > > > > > > > > > > > Also for storing the configs, we probably can't store the > > > broker/global > > > >

Re: Review Request 33916: Patch for KAFKA-2163

2015-05-11 Thread Joel Koshy
you have a strong preference. core/src/main/scala/kafka/server/OffsetManager.scala <https://reviews.apache.org/r/33916/#comment134283> Similar to above - this used to be debug level in the earlier version, but can be very useful for troubleshooting any issues. - Joel Koshy On M

Re: Review Request 24214: Patch for KAFKA-1374

2015-05-12 Thread Joel Koshy
compress messages in a preallocated buffer. It would be preferable to avoid having this compression logic in different places. - Joel Koshy On Jan. 17, 2015, 6:53 p.m., Manikumar Reddy O wrote: > > --- > This is an automatically

Re: Review Request 33557: Patch for KAFKA-1936

2015-05-12 Thread Joel Koshy
/KafkaRequestHandler.scala <https://reviews.apache.org/r/33557/#comment134386> Just a minor naming comment: these `mark*Rate` methods can just be called `record*` For e.g., `recordMessagesIn` - Joel Koshy On May 12, 2015, 2:32 p.m., Dong Lin

Re: [DISCUSS] KIP-21 Configuration Management

2015-05-12 Thread Joel Koshy
gt; > I like this approach (obviously). > > > > I am also OK with supporting broker re-read of config file based on ZK > > > > watch instead of SIGHUP, if we see this as more consistent with the > > rest > > > of > > > > our code base. > >

Re: Review Request 31850: Patch for KAFKA-1660

2015-05-12 Thread Joel Koshy
t; This tests idempotence of the close call. core/src/test/scala/integration/kafka/api/ProducerSendTest.scala <https://reviews.apache.org/r/31850/#comment134417> `100 -> %d, (i + 1) * numRecords` - Joel Koshy On April 30, 2015, 12:37 a.m

Re: Review Request 34070: Patch for KAFKA-2186

2015-05-12 Thread Joel Koshy
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/34070/#review83472 --- Ship it! Ship It! - Joel Koshy On May 12, 2015, 1:39 a.m

Re: Review Request 34070: Patch for KAFKA-2186

2015-05-12 Thread Joel Koshy
> On May 12, 2015, 2:31 a.m., Aditya Auradkar wrote: > > core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java, line 73 > > > > > > How does this work if the consumer doesn't own these partitions? Is it > >

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-05-13 Thread Joel Koshy
Just had a few minor questions before I join the vote thread. Apologies if these have been discussed: - Do we need DecreasePartitionsNotAllowed? i.e., can we just return InvalidPartitions instead? - AdminClient.listTopics: should we allow listing all partitions? Or do you intend for the client

Re: Can we throw away "ResponsesBeingSent" Gauge?

2015-05-14 Thread Joel Koshy
I'm also not sure how useful it is, but there is some discussion on it here: https://issues.apache.org/jira/browse/KAFKA-1597 On Thu, May 14, 2015 at 09:26:02PM +0300, Gwen Shapira wrote: > Hi, > > As part of KAFKA-1928, we need to consolidate existing SocketServer metrics > into Selector metric

Re: [VOTE] KIP-4 Admin Commands / Phase-1

2015-05-14 Thread Joel Koshy
+1 On Thu, May 07, 2015 at 04:25:06PM -0700, Jun Rao wrote: > +1 on the vote. > > Thanks, > > Jun > > On Thu, May 7, 2015 at 8:29 AM, Andrii Biletskyi < > andrii.bilets...@stealth.ly> wrote: > > > Jun, > > > > 1. My bad, updated description, but forgot the schema itself. Fixed. > > > > 2. Okay

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-15 Thread Joel Koshy
dealing with revoked certificates after an client authenticates)? Per your jira comment we can use an authorizer to block the client in this case, but if you have a proposal on handling periodic renegotiation it would be useful to discuss that. I agree we don't need to implement it now. - Joel

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-15 Thread Joel Koshy
clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java <https://reviews.apache.org/r/33620/#comment135117> @override annotations on this and couple other methods. - Joel Koshy On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: > > -

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-15 Thread Joel Koshy
example, or the SSLEngineResult.getStatus() returns CLOSED." Do we need to do this? i.e., it appears we are not handling this during the handshake. - Joel Koshy On May 15, 2015, 2:18 p.m., Sriharsha Chintalapani wrote: > > --

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-15 Thread Joel Koshy
/SSLTransportLayer.java <https://reviews.apache.org/r/33620/#comment135138> Sriharsha, this is the line from the ref guide under blocking tasks I was referring to: "The engine will block future wrap/unwrap calls until all of the outstanding tasks are completed" - Joel Koshy On Ma

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-15 Thread Joel Koshy
> On May 15, 2015, 10:54 p.m., Joel Koshy wrote: > > clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java, > > line 153 > > <https://reviews.apache.org/r/33620/diff/5/?file=957065#file957065line153> > > > > I think Michael mea

Re: Review Request 33620: Patch for KAFKA-1690

2015-05-15 Thread Joel Koshy
> On May 16, 2015, 12:07 a.m., Joel Koshy wrote: > > clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java, > > line 35 > > <https://reviews.apache.org/r/33620/diff/6/?file=961241#file961241line35> > > > > Can we just use a fix

Re: jira backlogs

2015-05-18 Thread Joel Koshy
mail that gets sent out to > the entire dev@ mailing list so everyone is on the loop about the patches > that need reviews, who is on-point and which ones have no reviewers. > > On Wed, Apr 8, 2015 at 10:22 AM, Joel Koshy wrote: > > > The weekly email sounds good, but I would mu

Re: jira backlogs

2015-05-18 Thread Joel Koshy
I forgot to add - I believe you need to be logged into jira at the time you view the page. The search basically filters by patch-available, non-empty reviewer and ordered by updated date. Thanks, Joel On Mon, May 18, 2015 at 03:10:03PM -0700, Joel Koshy wrote: > Here is (roughly) what I had

Re: Review Request 34241: Patch for KAFKA-2190

2015-05-19 Thread Joel Koshy
tps://reviews.apache.org/r/34241/#comment135484> Do you think we should add that this really only works when in-flight requests is one? core/src/main/scala/kafka/tools/MirrorMaker.scala <https://reviews.apache.org/r/34241/#comment135480> `info("Flushing producer."

Re: Review Request 24214: Patch for KAFKA-1374

2015-05-19 Thread Joel Koshy
use Stream.cons for convenience. core/src/test/scala/unit/kafka/log/LogTest.scala <https://reviews.apache.org/r/24214/#comment135496> Few more minor edits - to test appending keyed compressed messages. - Joel Koshy On May 18, 2015, 5:29 p.m., Manikumar Reddy O wrote: > >

Review Request 34397: Patch for KAFKA-1374

2015-05-19 Thread Joel Koshy
/LogTest.scala 76d3bfd378f32fd2b216b3ebdec86e2070491924 Diff: https://reviews.apache.org/r/34397/diff/ Testing --- Thanks, Joel Koshy

Re: jira backlogs

2015-05-19 Thread Joel Koshy
nks, Joel. Do you know why it doesn't seem to reflect the recent > changes? For example, kafka-1928 still has jkreps listed as the reviewer. > > Jun > > On Mon, May 18, 2015 at 3:10 PM, Joel Koshy wrote: > >> Here is (roughly) what I had in mind for this: >>

Re: Review Request 34397: Patch for KAFKA-1374

2015-05-19 Thread Joel Koshy
scala 844589427cb9337acd89a5239a98b811ee58118e core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala 3b5aa9dc3b7ac5893c1d281ae1326be0e9ed8aad core/src/test/scala/unit/kafka/log/LogTest.scala 76d3bfd378f32fd2b216b3ebdec86e2070491924 Diff: https://reviews.apache.org/r/34397/diff/ Testing --- Thanks,

Re: Review Request 34397: Patch for KAFKA-1374

2015-05-19 Thread Joel Koshy
63487#file963487line432> > > > > I think the coding style convention is > > > > if { > > > > } else { > > > > } Will do. - Joel ------- Thi

Re: [DISCUSS] KIP-19 Add a request timeout to NetworkClient

2015-05-20 Thread Joel Koshy
> The fact that I understand the producer internals and am still struggling > to understand the implications of the different settings, how I would set > them, and how they potentially interact such that I could set invalid > combinations seems like a red flag to me... Being able to say "I want > p

Re: [Vote] KIP-11 Authorization design for kafka security

2015-05-20 Thread Joel Koshy
+1 On Fri, May 15, 2015 at 04:18:49PM +, Parth Brahmbhatt wrote: > Hi, > > Opening the voting thread for KIP-11. > > Link to the KIP: > https://cwiki.apache.org/confluence/display/KAFKA/KIP-11+-+Authorization+Interface > Link to Jira: https://issues.apache.org/jira/browse/KAFKA-1688 > > Th

Re: [DISCUSS] KIP 20 Enable log preallocate to improve consume performance under windows and some old Linux file system

2015-05-20 Thread Joel Koshy
I think in general it is fine (even good) if a VOTE thread has a lot of discussion. The only issue I can think of is the one that Gwen made reference to: early votes -> update KIP/whatever is being voted on due to more discussion -> later votes as it then becomes unclear on what exactly each vo

Re: Request for permission to edit pages in the wiki

2015-05-20 Thread Joel Koshy
Done On Wed, May 20, 2015 at 11:45:48PM +0100, Ismael Juma wrote: > Hi, > > In order to edit the "Patch submission and review" page with information on > how to merge GitHub pull requests, it would be helpful to have edit > permission for Kafka's Confluence pages. My Confluence account id is > `i

Re: [VOTE] KIP-21 Dynamic Configuration

2015-05-20 Thread Joel Koshy
+1 On Wed, May 20, 2015 at 05:33:31AM +, Aditya Auradkar wrote: > Thanks Andrii. I'll make the changes. > > I've also updated KIP-21 to include the new config requests. Take a look and > vote. > https://cwiki.apache.org/confluence/display/KAFKA/KIP-21+-+Dynamic+Configuration > > Aditya > __

Re: Request for permission to edit pages in the wiki

2015-05-21 Thread Joel Koshy
Sorry about that - should work now. On Thu, May 21, 2015 at 11:22:40PM +0100, Ismael Juma wrote: > On Wed, May 20, 2015 at 11:54 PM, Joel Koshy wrote: > > > Done > > > > Thank you Joel, but the "Edit" link is still not available to me. > > Best, > I

Re: [DISCUSS] KIP-4 - Command line and centralized administrative operations (Thread 2)

2015-05-21 Thread Joel Koshy
anks, > > Mayuresh > > > On Thu, May 14, 2015 at 10:22 AM, Jun Rao wrote: > > > For ListTopics, we decided not to add a ListTopics request for now and just > > rely on passing in an empty list to TMR. We can revisit this in the future > > if it becomes an

  1   2   3   4   5   6   7   8   9   10   >