Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-04 Thread Dong Lin
> rationale for the design choices that were made and rejected. > > 3. Should changeReplicaDir be alterReplicaDir? We have used `alter` for > other methods. > > Thanks, > Ismael > > > > > On Fri, Aug 4, 2017 at 5:37 AM, Dong Lin wrote: > > > Hi all, >

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-08-04 Thread Dong Lin
think. > I think you're right about TopicPartitionReplica. At the protocol level we > could group topics and partitions together so avoid having the same topic > name multiple times when querying for the status of all the partitions of a > topic. > > Thanks again for taki

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-08-03 Thread Dong Lin
<https://cwiki.apache.org/confluence/display/KAFKA/KIP-113%3A+Support+replicas+movement+between+log+directories#KIP-113:Supportreplicasmovementbetweenlogdirectories-AdminClient>. Can you please review and comment if you have any concern? Thanks! Dong On Wed, Jul 12, 2017 at 10:44 AM, Dong Lin

Re: [DISCUSS] KIP-179: Change ReassignPartitionsCommand to use AdminClient

2017-08-03 Thread Dong Lin
Hey Tom, Thanks for the KIP. It seems that the prior discussion in this thread has focused on reassigning partitions (or AlterTopics). I haven't looked into this yet. I have two comments on the replicaStatus() API and the ReplicaStatusRequest: - It seems that the use-case for ReplicaStatusReques

[jira] [Created] (KAFKA-5695) Test DeleteRecordsRequest in AuthorizerIntegrationTest

2017-08-02 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5695: --- Summary: Test DeleteRecordsRequest in AuthorizerIntegrationTest Key: KAFKA-5695 URL: https://issues.apache.org/jira/browse/KAFKA-5695 Project: Kafka Issue Type

Re: 答复: [DISCUSS] KIP-178: Size-based log directory selection strategy

2017-08-02 Thread Dong Lin
_______ > 发件人: Dong Lin > 发送时间: 2017年8月3日 9:10 > 收件人: dev@kafka.apache.org > 主题: Re: [DISCUSS] KIP-178: Size-based log directory selection strategy > > Hey Xu, > > Thanks for the KIP. This is a very good idea to select log directory based > on the free disk space.

Re: [DISCUSS] KIP-178: Size-based log directory selection strategy

2017-08-02 Thread Dong Lin
the existing log directory selection algorithm without adding a new config. Thanks, Dong On Wed, Aug 2, 2017 at 6:10 PM, Dong Lin wrote: > Hey Xu, > > Thanks for the KIP. This is a very good idea to select log directory based > on the free disk space. Do you think we can simply simpl

Re: [DISCUSS] KIP-178: Size-based log directory selection strategy

2017-08-02 Thread Dong Lin
Hey Xu, Thanks for the KIP. This is a very good idea to select log directory based on the free disk space. Do you think we can simply simply change the implementation to select log directory based on the free disk space instead of adding a new config? Or is there any good reason that user will wan

[jira] [Created] (KAFKA-5694) Add ChangeReplicaDirRequest and DescribeReplicaDirRequest (KIP-113)

2017-08-02 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5694: --- Summary: Add ChangeReplicaDirRequest and DescribeReplicaDirRequest (KIP-113) Key: KAFKA-5694 URL: https://issues.apache.org/jira/browse/KAFKA-5694 Project: Kafka

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

2017-07-26 Thread Dong Lin
t; > +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, &g

[jira] [Created] (KAFKA-5627) Reduce classes needed for LeaderAndIsrPartitionState and MetadataPartitionState

2017-07-22 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5627: --- Summary: Reduce classes needed for LeaderAndIsrPartitionState and MetadataPartitionState Key: KAFKA-5627 URL: https://issues.apache.org/jira/browse/KAFKA-5627 Project: Kafka

[jira] [Created] (KAFKA-5626) Producer should be able to negotiate ProduceRequest version with broker

2017-07-21 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5626: --- Summary: Producer should be able to negotiate ProduceRequest version with broker Key: KAFKA-5626 URL: https://issues.apache.org/jira/browse/KAFKA-5626 Project: Kafka

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-07-12 Thread Dong Lin
status of selected partitions can be useful to check the whether the reassignment of the replicas to the specific log directories has been completed. I will assume this minor change is OK if there is no concern with it in the community :) Thanks, Dong On Mon, Jun 12, 2017 at 10:46 AM, Dong Lin

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-07-12 Thread Dong Lin
The protocol change has been updated in KIP-113 <https://cwiki.apache.org/confluence/display/KAFKA/KIP-113%3A+Support+replicas+movement+between+log+directories> . On Wed, Jul 12, 2017 at 10:44 AM, Dong Lin wrote: > Hi all, > > I have made a minor change to the DescribeDirsReque

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

2017-07-12 Thread Dong Lin
Fri, Jun 23, 2017 at 5:16 AM, Mickael Maison > wrote: > > > +1 (non-binding) > > Thanks > > > > On Thu, Jun 22, 2017 at 6:07 PM, Onur Karaman > > wrote: > > > +1 > > > > > > On Thu, Jun 22, 2017 at 10:05 AM, Dong Lin > wr

Re: [ANNOUNCE] New Kafka PMC member Ismael Juma

2017-07-05 Thread Dong Lin
Congratulations Ismael! On Wed, Jul 5, 2017 at 1:55 PM, Jun Rao wrote: > Hi, Everyone, > > Ismael Juma has been active in the Kafka community since he became > a Kafka committer about a year ago. I am glad to announce that Ismael is > now a member of Kafka PMC. > > Congratulations, Ismael! > > J

[jira] [Created] (KAFKA-5544) The LastStableOffsetLag metric should be removed when partition is deleted

2017-06-29 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5544: --- Summary: The LastStableOffsetLag metric should be removed when partition is deleted Key: KAFKA-5544 URL: https://issues.apache.org/jira/browse/KAFKA-5544 Project: Kafka

[jira] [Created] (KAFKA-5542) Improve Java doc for LeaderEpochFileCache.endOffsetFor()

2017-06-29 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5542: --- Summary: Improve Java doc for LeaderEpochFileCache.endOffsetFor() Key: KAFKA-5542 URL: https://issues.apache.org/jira/browse/KAFKA-5542 Project: Kafka Issue Type: Bug

[jira] [Resolved] (KAFKA-5521) Support replicas movement between log directories (KIP-113)

2017-06-28 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin resolved KAFKA-5521. - Resolution: Duplicate Duplicate of https://issues.apache.org/jira/browse/KAFKA-5163 > Support repli

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

2017-06-28 Thread Dong Lin
PM, Abhishek Mendhekar < > > > abhishek.mendhe...@gmail.com> wrote: > > > > > > > Hi Dong, > > > > > > > > Thanks for the suggestion! > > > > > > > > I think TopicCount sounds reasonable to me and it definitely seems > >

[jira] [Created] (KAFKA-5521) Support replicas movement between log directories (KIP-113)

2017-06-26 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5521: --- Summary: Support replicas movement between log directories (KIP-113) Key: KAFKA-5521 URL: https://issues.apache.org/jira/browse/KAFKA-5521 Project: Kafka Issue Type

[jira] [Resolved] (KAFKA-5367) Producer should not expiry topic from metadata cache if accumulator still has data for this topic

2017-06-26 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5367?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin resolved KAFKA-5367. - Resolution: Invalid > Producer should not expiry topic from metadata cache if accumulator still

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

2017-06-22 Thread Dong Lin
Thanks for the KIP. +1 (non-binding) On Wed, Jun 21, 2017 at 1:17 PM, Abhishek Mendhekar < abhishek.mendhe...@gmail.com> wrote: > Hi Kafka Dev, > > I did like to start the voting on - > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 168%3A+Add+TotalTopicCount+metric+per+cluster > > Disc

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

2017-06-21 Thread Dong Lin
Hey Abhishek, I think the metric is useful. Sorry for being late on this. I am wondering if TopicCount is a better name than TotalTopicCount, given that we currently have metric with names OfflinePartitionsCount, LeaderCount, PartitionCount etc. Thanks, Dong On Fri, Jun 16, 2017 at 9:09 AM, Abhi

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-06-12 Thread Dong Lin
signed to > > disks just based on the partition count. So, it does seem possible for > > disks to get imbalanced from time to time. If someone can share some > > stats > > for that in practice, that will be very helpful. > > > > Thanks, > > > > Jun &

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

2017-06-08 Thread Dong Lin
Hey Ismael, Thanks for the feedback! Could you please vote for the KIP if it looks good? Then I will find two more committers to vote as well. Thanks, Dong On Tue, May 30, 2017 at 9:08 AM, Dong Lin wrote: > Thanks Edoardo and everyone for the comment! That is a very good point. I >

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

2017-06-08 Thread Dong Lin
> +1 (non binding) > > > > Thanks, > > Michał > > > > On 02/06/17 10:18, Mickael Maison wrote: > > +1 (non binding) > > Thanks for the KIP > > > > On Thu, Jun 1, 2017 at 5:44 PM, Dong Lin ** > > wrote: > > > > Hi all, > &

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-06-07 Thread Dong Lin
plica from the two over-loaded disks to the two under-loaded disks on the same broKER. Does this make sense? Thanks, Dong On Wed, Jun 7, 2017 at 2:12 PM, Dong Lin wrote: > Hey Sriram, > > Thanks for raising these concerns. Let me answer these questions below: > > -

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-06-07 Thread Dong Lin
nd intra broker movements of > partition the same. Deploy the code, use it and see if it is an actual > problem and you get great savings by avoiding the network route to move > partitions within the same broker. If so, add this optimization. > > On Wed, Jun 7, 2017 at 1:03 PM

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-06-07 Thread Dong Lin
yhow, not sure the arithmetic is right there, but i think that is the > type of argument that would be helpful to think about the tradeoff in > complexity. > > -Jay > > > > On Tue, Jun 6, 2017 at 7:09 PM, Dong Lin wrote: > > > Hey Sriram, > > > > Thanks for t

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-06-06 Thread Dong Lin
Hey Sriram, Thanks for taking time to review the KIP. Please see below my answers to your questions: >1. Could you pick a hardware/Kafka configuration and go over what is the >average disk/partition repair/restore time that we are targeting for a >typical JBOD setup? We currently don't have this

Re: [VOTE] KIP-113 - Support replicas movement between log directories

2017-06-06 Thread Dong Lin
, > > > > Thanks for the KIP. +1 from me. > > > > Jun > > > > On Mon, Apr 3, 2017 at 9:49 AM, Dong Lin wrote: > > > > > Hi all, > > > > > > It seems that there is no further concern with the KIP-113. We would > like > > >

Re: [VOTE] KIP-113 - Support replicas movement between log directories

2017-06-03 Thread Dong Lin
t;per-broker or per-thread? >- > >Reassignment JSON: can log_dirs be made optional? i.e., its absence >would mean “any” >- > >Can you also explicitly state somewhere that “any” translates to >round-robin assignment today? > > > On Mon, Apr

[jira] [Created] (KAFKA-5367) Producer should not expiry topic from metadata cache if accumulator still has data for this topic

2017-06-02 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5367: --- Summary: Producer should not expiry topic from metadata cache if accumulator still has data for this topic Key: KAFKA-5367 URL: https://issues.apache.org/jira/browse/KAFKA-5367

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

2017-06-01 Thread Dong Lin
Hi all, Can you please vote for KIP-164? The KIP can be found at https://cwiki.apache.org/confluence/display/KAFKA/KIP-164-+Add+UnderMinIsrPartitionCount+and+per-partition+UnderMinIsr+metrics . Thanks, Dong

Re: [DISCUSS] KIP-108: Create Topic Policy

2017-05-30 Thread Dong Lin
roposal should be done via a separate KIP > as KIP-108 was specifically about create topic requests that are done via > the Kafka protocol. > > Ismael > > On Wed, May 31, 2017 at 12:37 AM, Dong Lin wrote: > > > On Tue, May 30, 2017 at 4:26 PM, Colin McCabe > wrote: >

Re: [DISCUSS] KIP-108: Create Topic Policy

2017-05-30 Thread Dong Lin
On Tue, May 30, 2017 at 4:26 PM, Colin McCabe wrote: > On Tue, May 30, 2017, at 15:55, Dong Lin wrote: > > Hey Colin, > > > > I think one big advantage of the broker side config is that it can not be > > ignored by the malicious client, right? > > Hi Dong, >

Re: [DISCUSS] KIP-108: Create Topic Policy

2017-05-30 Thread Dong Lin
Hey Colin, I think one big advantage of the broker side config is that it can not be ignored by the malicious client, right? Thanks, Dong On Tue, May 30, 2017 at 3:53 PM, Dong Lin wrote: > Do we have an old version of bin/kafka-topics.sh which creates topic via > ZK and still allows u

Re: [DISCUSS] KIP-108: Create Topic Policy

2017-05-30 Thread Dong Lin
K to prevent unprivileged users from modifying the > topic configurations? > > best, > Colin > > > On Tue, May 30, 2017, at 15:02, Dong Lin wrote: > > Hey Ismael, > > > > I agree that it makes sense not to cover ZK-based topic creation with the > > topic creation

Re: [DISCUSS] KIP-108: Create Topic Policy

2017-05-30 Thread Dong Lin
is to > limit ZK access to brokers only. > > Ismael > > On Tue, May 30, 2017 at 10:19 PM, Dong Lin wrote: > > > Hey Ismael, > > > > Thanks for the KIP. This is definitely useful. > > > > Does the KIP apply the topic creation policy to ZK-based topic

Re: [DISCUSS] KIP-108: Create Topic Policy

2017-05-30 Thread Dong Lin
Hey Ismael, Thanks for the KIP. This is definitely useful. Does the KIP apply the topic creation policy to ZK-based topic creation? If not, which seems to be the case from my understanding, should we have a new broker config to disable ZK-based topic creation? This seems necessary to prevent user

[jira] [Updated] (KAFKA-5341) Add UnderMinIsrPartitionCount and per-partition UnderMinIsr metrics

2017-05-30 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-5341: Description: We currently have under replicated partitions, but we do not have a metric to track the

[jira] [Updated] (KAFKA-5341) Add UnderMinIsrPartitionCount and per-partition UnderMinIsr metrics

2017-05-30 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5341?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-5341: Summary: Add UnderMinIsrPartitionCount and per-partition UnderMinIsr metrics (was: Add

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

2017-05-30 Thread Dong Lin
ark, SO21 2JN > > > > > > > > > > > > > > > From: Mickael Maison > > > To: dev@kafka.apache.org > > > Date: 30/05/2017 10:51 > > > Subject:Re: [DISCUSS] KIP-164 Add unavailablePartitionCount and > &

[jira] [Created] (KAFKA-5341) Add unavailablePartitionCount and per-partition Unavailable metrics (KIP-164)

2017-05-28 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5341: --- Summary: Add unavailablePartitionCount and per-partition Unavailable metrics (KIP-164) Key: KAFKA-5341 URL: https://issues.apache.org/jira/browse/KAFKA-5341 Project: Kafka

[DISCUSS] KIP-164 Add unavailablePartitionCount and per-partition Unavailable metrics

2017-05-28 Thread Dong Lin
Hi, We created KIP-164 to propose adding per-partition metric *Unavailable* and per-broker metric *UnavailablePartitionCount* The KIP wik can be found at https://cwiki.apache.org/confluence/display/KAFKA/KIP-164-+Add+unavailablePartitionCount+and+per-partition+Unavailable+metrics . Comments are

[jira] [Created] (KAFKA-5335) Controller should batch updatePartitionReassignmentData() operation

2017-05-26 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5335: --- Summary: Controller should batch updatePartitionReassignmentData() operation Key: KAFKA-5335 URL: https://issues.apache.org/jira/browse/KAFKA-5335 Project: Kafka

Re: [VOTE] KIP-153 (separating replication traffic from BytesOutPerSec metric)

2017-05-09 Thread Dong Lin
+1 On Sun, May 7, 2017 at 7:40 PM, Jun Rao wrote: > Hi, Everyone, > > Since this is a relatively simple change, I would like to start the voting > process for KIP-153 : Include only client traffic in BytesOutPerSec metric. > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-153+% > 3A+Incl

[jira] [Created] (KAFKA-5163) Support replicas movement between log directories (KIP-113)

2017-05-02 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5163: --- Summary: Support replicas movement between log directories (KIP-113) Key: KAFKA-5163 URL: https://issues.apache.org/jira/browse/KAFKA-5163 Project: Kafka Issue Type

[jira] [Commented] (KAFKA-4740) Using new consumer API with a Deserializer that throws SerializationException can lead to infinite loop

2017-04-28 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4740?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15989453#comment-15989453 ] Dong Lin commented on KAFKA-4740: - I realized that this is a known problem since

Re: [VOTE] KIP-112 - Handle disk failure for JBOD

2017-04-27 Thread Dong Lin
Thanks to everyone who voted and provided feedback! This KIP is now adopted with 3 binding +1s (Jun, Joel, Becket) and 1 non-binding +1s (Radai) Dong On Thu, Apr 27, 2017 at 4:12 PM, Dong Lin wrote: > Thanks for the vote Jun! > > I think that statement is probably OK because it ass

Re: [VOTE] KIP-112 - Handle disk failure for JBOD

2017-04-27 Thread Dong Lin
lica is manually deleted from the log directory. > > Jun > > On Wed, Apr 26, 2017 at 11:27 AM, Dong Lin wrote: > > > Thanks for the vote! > > > > Discussed with Joel offline. I have updated the KIP to specify that > > controller will consider a

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-04-26 Thread Dong Lin
Hey Jun, Ismael, Thanks for all the review! Can you vote for KIP-112 if you are OK with the latest design doc? Thanks, Dong On Thu, Mar 30, 2017 at 3:29 PM, Dong Lin wrote: > Hi all, > > Thanks for all the comments. I am going to open the voting thread if > there is no further

Re: [VOTE] KIP-112 - Handle disk failure for JBOD

2017-04-26 Thread Dong Lin
JBOD. > > On Wed, Apr 5, 2017 at 5:46 PM, Becket Qin wrote: > > > +1 > > > > Thanks for the KIP. Made a pass and had some minor change. > > > > On Mon, Apr 3, 2017 at 3:16 PM, radai > wrote: > > > > > +1, LGTM > > > > > > On Mon, Apr

Re: [VOTE] KIP-126 - Allow KafkaProducer to split and resend oversized batches.

2017-04-25 Thread Dong Lin
+1 (non-binding) On Tue, Apr 25, 2017 at 12:33 PM, Becket Qin wrote: > Hi, > > I would like to start the voting on KIP-126. The KIP is intended to solve > the problem that RecordTooLargeExceptions are thrown from the producer due > to inaccurate estimation of the compression ratio. The solution

Re: [VOTE] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-04-25 Thread Dong Lin
+1 (non-binding) On Tue, Apr 25, 2017 at 10:42 AM, Becket Qin wrote: > Hi, > > I would like to start the voting on KIP-126. The KIP is intended to solve > the problem that RecordTooLargeExceptions are thrown from the producer due > to inaccurate estimation of the compression ratio. The solution

Re: [ANNOUNCE] New committer: Rajini Sivaram

2017-04-24 Thread Dong Lin
Congratulations Rajini! On Mon, Apr 24, 2017 at 4:52 PM, Becket Qin wrote: > Congratulations! Rajini! Great work! > > On Mon, Apr 24, 2017 at 3:33 PM, Jason Gustafson > wrote: > > > Woohoo! Great work, Rajini! > > > > On Mon, Apr 24, 2017 at 3:27 PM, Jun Rao wrote: > > > > > Congratulations, R

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-04-20 Thread Dong Lin
from > the existing metrics. And I think batch-split-rate is already a good > indication on whether the batch split has caused performance problem or > not. > > So I am not sure if it is worth having an explicit batch-split-ratio metric > in this case. > > Thanks, > >

[jira] [Updated] (KAFKA-5086) Update topic expiry time in Metadata every time the topic metadata is requested

2017-04-18 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5086?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-5086: Summary: Update topic expiry time in Metadata every time the topic metadata is requested (was: Topic

[jira] [Created] (KAFKA-5086) Topic expiry time in Metadata should be updated every time the topic metadata is requested

2017-04-18 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5086: --- Summary: Topic expiry time in Metadata should be updated every time the topic metadata is requested Key: KAFKA-5086 URL: https://issues.apache.org/jira/browse/KAFKA-5086

[jira] [Updated] (KAFKA-5078) PartitionRecords.fetchRecords(...) should defer exception to the next call if iterator has already moved across any valid record

2017-04-17 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-5078: Description: Suppose there are two valid records followed by one invalid records in the

[jira] [Updated] (KAFKA-5078) PartitionRecords.fetchRecords(...) should defer exception to the next call if iterator has already moved across any valid record

2017-04-17 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5078?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-5078: Description: Suppose there are two valid records followed by one invalid records in the

[jira] [Created] (KAFKA-5078) PartitionRecords.fetchRecords(...) should defer exception to the next call if iterator has already moved across any valid record

2017-04-17 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-5078: --- Summary: PartitionRecords.fetchRecords(...) should defer exception to the next call if iterator has already moved across any valid record Key: KAFKA-5078 URL: https://issues.apache.org

[jira] [Updated] (KAFKA-5075) Defer exception to the next pollOnce() if consumer's fetch position has already increased

2017-04-15 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-5075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-5075: Summary: Defer exception to the next pollOnce() if consumer's fetch position has already increased

[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2017-04-06 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15959741#comment-15959741 ] Dong Lin commented on KAFKA-3042: - [~pengwei] Sorry, I didn't notice your messa

[VOTE] KIP-113 - Support replicas movement between log directories

2017-04-03 Thread Dong Lin
Hi all, It seems that there is no further concern with the KIP-113. We would like to start the voting process. The KIP can be found at *https://cwiki.apache.org/confluence/display/KAFKA/KIP-113%3A+Support+replicas+movement+between+log+directories

[VOTE] KIP-112 - Handle disk failure for JBOD

2017-04-03 Thread Dong Lin
Hi all, It seems that there is no further concern with the KIP-112. We would like to start the voting process. The KIP can be found at *https://cwiki.apache.org/confluence/display/KAFKA/KIP-112%3A+Handle+disk+failure+for+JBOD

Re: [VOTE] KIP-134: Delay initial consumer group rebalance

2017-03-30 Thread Dong Lin
+1 (non-binding) Thanks! On Thu, Mar 30, 2017 at 6:03 PM, Becket Qin wrote: > +1 Thanks for the KIP! > > On Thu, Mar 30, 2017 at 12:55 PM, Jason Gustafson > wrote: > > > +1 Thanks for the KIP! > > > > On Thu, Mar 30, 2017 at 12:51 PM, Guozhang Wang > > wrote: > > > > > +1 > > > > > > Sorry a

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-30 Thread Dong Lin
t; people, we can start the vote. > > Thanks, > > Jun > > On Thu, Mar 30, 2017 at 10:59 AM, Dong Lin wrote: > > > Hey Jun, > > > > Thanks much for the comment! Do you think we start vote for KIP-112 and > > KIP-113 if there is no further concern? > &g

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-03-30 Thread Dong Lin
Hi all, Thanks for all the comments. I am going to open the voting thread if there is no further concern with the KIP. Dong On Wed, Mar 15, 2017 at 5:25 PM, Ismael Juma wrote: > Thanks for the updates Dong, they look good to me. > > Ismael > > On Wed, Mar 15, 2017 at 5:50 PM,

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-30 Thread Dong Lin
ure that all replicas will be in the right log dirs. Solution > (1) doesn't guarantee that. That seems better and we can go with your > current solution then. > > Thanks, > > Jun > > On Fri, Mar 24, 2017 at 4:28 PM, Dong Lin wrote: > > > Hey Jun, > > >

[jira] [Commented] (KAFKA-4973) Transient failure of AdminClientTest.testDeleteRecordsWithException

2017-03-29 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4973?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15947479#comment-15947479 ] Dong Lin commented on KAFKA-4973: - [~ijuma] Certainly. Thanks for filing the ticke

Re: [DISCUSS] KIP-134: Delay initial consumer group rebalance

2017-03-28 Thread Dong Lin
I think it makes sense to reset the timer when group changes. This allows broker to possibly wait longer if more consumers are restarted in parallel. For example, user may start a large number of consumers in sequence using script. The time for all consumers to send JoinGroupRequest will likely be

Re: [DISCUSS] KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user

2017-03-27 Thread Dong Lin
; Ismael, mentioned on the jira, we need a specific non retriable error code > in this case, with specific message, at least until the other KIP is ready. > > Thanks, > > Mayuresh > On Mon, Mar 27, 2017 at 10:55 PM Dong Lin wrote: > > > Hey Mayuresh, > > >

Re: [DISCUSS] KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user

2017-03-27 Thread Dong Lin
in the jira ticket, we wanted something more specific than > general. > The main issue is that we do not propagate server side error messages to > clients, right now. I am working on a KIP proposal to propose it. > > Thanks, > > Mayuresh > > On Mon, Mar 27, 201

[jira] [Updated] (KAFKA-4962) Unify the implentation of schemas in Protocol.java

2017-03-27 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4962?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-4962: Description: The schemas implementation in Protocol.java is currently inconsistent. For example, the

[jira] [Updated] (KAFKA-4962) Unify the implentation of schemas in Protocol.java

2017-03-27 Thread Dong Lin (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-4962?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dong Lin updated KAFKA-4962: Summary: Unify the implentation of schemas in Protocol.java (was: Unify the way of implementing schemas in

[jira] [Created] (KAFKA-4962) Unify the way of implementing schemas in Protocol.java

2017-03-27 Thread Dong Lin (JIRA)
Dong Lin created KAFKA-4962: --- Summary: Unify the way of implementing schemas in Protocol.java Key: KAFKA-4962 URL: https://issues.apache.org/jira/browse/KAFKA-4962 Project: Kafka Issue Type

Re: [DISCUSS] KIP-135 : Send of null key to a compacted topic should throw non-retriable error back to user

2017-03-27 Thread Dong Lin
Hey Mayuresh, Thanks for the patch. I am wondering if it would be better to add a more general error, e.g. InvalidMessageException. The benefit is that we can reuse this for other message level error instead of adding one exception class for each possible exception in the future. This is similar t

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-24 Thread Dong Lin
will complete the pending reassignment process > by putting some replicas in the wrong log dirs. > > Implementation wise, (1) and (2) seem to be the same. So, it seems to me > that (1) is better? > > Thanks, > > Jun > > > On Thu, Mar 23, 2017 at 11:54 PM, Dong Lin

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-23 Thread Dong Lin
s down when the partition > reassignment tool is started, does the tool just hang (keep retrying > ChangeReplicaDirRequest) until the broker comes back? Currently, the > partition reassignment tool doesn't block. > > Thanks, > > Jun > > > On Tue, Mar 21, 2017 at 11:24

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-03-22 Thread Dong Lin
Never mind about my second comment. I misunderstood the semantics of producer's batch.size. On Wed, Mar 22, 2017 at 10:20 AM, Dong Lin wrote: > Hey Becket, > > In addition to the batch-split-rate, should we also add batch-split-ratio > sensor to gauge the probability that we ha

Re: [DISCUSS] KIP-126 - Allow KafkaProducer to batch based on uncompressed size

2017-03-22 Thread Dong Lin
ze > > > is > > > > > > exceeded? > > > > > > > > > > > > -Jason > > > > > > > > > > > > On Mon, Mar 13, 2017 at 8:10 PM, Becket Qin < > becket@gmail.com> > > > > > wrote

Re: [VOTE] KIP-122: Add Reset Consumer Group Offsets tooling

2017-03-21 Thread Dong Lin
ts stored in Kafka, but it will not protect against an > > active consumer group in that case? > > > > Thanks, > > Jason > > > > On Tue, Mar 14, 2017 at 10:13 AM, Dong Lin wrote: > > > > > +1 (non-binding) > > > > > > On Tue, Ma

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-21 Thread Dong Lin
porary or pending. If all replicas are in the > final state, the tool checks if all replicas are in the expected log dirs. > If they are not, output a warning (and perhaps suggest the users to move > the data again). However, this should be rare. > > Thanks, > > Jun > > >

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-20 Thread Dong Lin
Hey Jun, Thanks for the response! It seems that we have only two remaining issues. Please see my reply below. On Mon, Mar 20, 2017 at 7:45 AM, Jun Rao wrote: > Hi, Dong, > > Thanks for the update. A few replies inlined below. > > On Thu, Mar 16, 2017 at 12:28 AM, Dong Lin wrote

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-16 Thread Dong Lin
Hey Jun, After thinking about 14 more, I think your solution is reasonable. I have updated the KIP to specify that the number of ReplicaMoveThread defaults to # log dirs. Thanks! Dong On Thu, Mar 16, 2017 at 12:28 AM, Dong Lin wrote: > Hey Jun, > > Thanks for your comment! Plea

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-16 Thread Dong Lin
can cancel movement across log directories by first sending DescribeDirsRequest, figure out the source directory of those replicas that are being moved, and then send ChangeReplicaDirRequest to move replica to the source log directory. But "any" seems like an easier and reasonable approach to cancel repl

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-03-15 Thread Dong Lin
When changing the code, I realized that it feels weird to have DeleteRequest and DeleteTopicsRequest. Thus I would follow the suggestion and change it to DeleteRecordsRequest in this KIP, unless we decide to use PurgeRequest. On Wed, Mar 15, 2017 at 12:04 PM, Dong Lin wrote: > Hey Jason, Ism

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-03-15 Thread Dong Lin
to the Java AdminClient (KIP-117). If it's the Scala one, then it > > doesn't even need to be in the KIP as the Scala AdminClient is internal > and > > no compatibility guarantees are offered (the methods that exist there > never > > went through a KIP for example

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-03-15 Thread Dong Lin
gt; would make sense to add a short sentence stating that this scenario is > covered in KIP-113. People won't necessarily read both KIPs at the same > time and it's helpful to cross-reference when it makes sense. > > Thanks for your work on this. > > Ismael > > O

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-03-14 Thread Dong Lin
roker. This is a safe approach that is no worse than current practice. Would this address your concern if I specify the problem and the two solutions in the KIP? Thanks, Dong On Tue, Mar 14, 2017 at 3:29 PM, Dong Lin wrote: > Hey Ismael, > > Thanks for the comment. Please see my reply

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-03-14 Thread Dong Lin
Hey Ismael, Thanks for the comment. Please see my reply below. On Tue, Mar 14, 2017 at 10:31 AM, Ismael Juma wrote: > Thanks Dong. Comments inline. > > On Fri, Mar 10, 2017 at 6:25 PM, Dong Lin wrote: > > > > I get your point. But I am not sure we should recommend user

Re: [VOTE] KIP-122: Add Reset Consumer Group Offsets tooling

2017-03-14 Thread Dong Lin
+1 (non-binding) On Tue, Mar 14, 2017 at 8:53 AM, Bill Bejeck wrote: > +1 > > On Tue, Mar 14, 2017 at 11:50 AM, Grant Henke wrote: > > > +1. Agreed. This is a great tool to have. > > > > On Tue, Mar 14, 2017 at 12:33 AM, Gwen Shapira > wrote: > > > > > +1 (binding) > > > > > > Nice job - this

Re: [VOTE] KIP-117: Add a public AdminClient API for Kafka admin operations

2017-03-14 Thread Dong Lin
+1 On Tue, Mar 14, 2017 at 8:50 AM, Grant Henke wrote: > +1 > > On Tue, Mar 14, 2017 at 2:44 AM, Sriram Subramanian > wrote: > > > +1 (binding) > > > > Nice work in driving this. > > > > On Mon, Mar 13, 2017 at 10:31 PM, Gwen Shapira > wrote: > > > > > +1 (binding) > > > > > > I expressed few

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-13 Thread Dong Lin
directories. On the other hand we can have multiple threads to move replicas to the same log directory. For example, if broker uses SSD, the CPU instead of disk IO may be the replica move bottleneck and it will be faster to move replicas using multiple threads per log directory. > &

Re: [DISCUSS] KIP-112: Handle disk failure for JBOD

2017-03-10 Thread Dong Lin
Hey Ismael, Thanks for your comments. Please see my reply below. On Fri, Mar 10, 2017 at 9:12 AM, Ismael Juma wrote: > Hi Dong, > > Thanks for the updates, they look good. A couple of comments below. > > On Tue, Mar 7, 2017 at 7:30 PM, Dong Lin wrote: > > > &g

Re: [VOTE] KIP-107: Add purgeDataBefore() API in AdminClient

2017-03-10 Thread Dong Lin
equest to FetchRecordsRequest > and ProduceRequest to ProduceRequestsRequest.) > > -Jason > > On Tue, Mar 7, 2017 at 10:11 AM, Dong Lin wrote: > > > Hi Jun, Ismael, > > > > I think making the API similar to a future KIP is desirable but not > > required.

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-09 Thread Dong Lin
t for controller to send LeaderAndIsrRequest to broker. The previous approach of creating an empty directory seems hacky. On Thu, Mar 9, 2017 at 6:33 PM, Dong Lin wrote: > Hey Jun, > > Thanks for your comments! I have updated the KIP to address your comments. > Please see my reply in

Re: [DISCUSS] KIP-113: Support replicas movement between log directories

2017-03-09 Thread Dong Lin
equest anyway, we can also use this request to expose stats like the individual log size instead of using JMX. One drawback of using JMX is that user has to manage the JMX port and related credentials if they haven't already done this, which is the case at LinkedIn. > Thanks, > > J

<    1   2   3   4   5   6   7   8   9   10   >