Re: [DISCUSS] KIP-117: Add a public AdministrativeClient API for Kafka admin operations

2017-02-02 Thread Becket Qin
batching blocking API is fine? Thanks, Jiangjie (Becket) Qin On Thu, Feb 2, 2017 at 5:54 PM, Dong Lin wrote: > Hey Colin, > > Thanks for the KIP. I have a few comments below: > > - I share similar view with Ismael that a Future-based API is better. > PurgeDataFrom() is a

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-19 Thread Becket Qin
ks, Jiangjie (Becket) Qin On Sat, Feb 18, 2017 at 9:35 PM, Dong Lin wrote: > I realized the main concern with this proposal is how user can interpret > this CPU-percentage based quota. Since this quota is exposed to user, we > need to explain to user how this quota is going to im

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-19 Thread Becket Qin
+1. Thanks for the great work on the KIP! I have only one minor question, in the wiki (and the doc) the new message set format has a "FirstSequence" field, should it just be "Sequence" if the sequence is always associated with a message set? On Fri, Feb 17, 2017 at 3:28 AM, Michael Pearce wrote:

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-20 Thread Becket Qin
hit. So it is not better than the request handling time quota. In fact I feel it is clearer to tell user that "you are limited because you have taken 30% of the CPU time on the broker" than otherwise something like "your request rate quota on metadata request has reached". Thank

Re: [VOTE] KIP-98: Exactly Once Delivery and Transactional Messaging

2017-02-20 Thread Becket Qin
first message's sequence is 5, then the last message's sequence number > would be 104, and the next message set's first sequence is expected to be > 105. > > > Guozhang > > > On Sun, Feb 19, 2017 at 4:48 PM, Becket Qin wrote: > > > +1. Thanks for th

Re: [DISCUSS] KIP-124: Request rate quotas

2017-02-20 Thread Becket Qin
the users is difficult. So in practice it would probably be more like first set a relative high protective CPU time quota for everyone and increase that for some individual clients on demand. Thanks, Jiangjie (Becket) Qin On Mon, Feb 20, 2017 at 5:48 PM, Guozhang Wang wrote: > This is a gr

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

2017-02-21 Thread Becket Qin
+KafkaProducer+to+batch+based+on+uncompressed+size Thanks, Jiangjie (Becket) Qin

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

2017-02-21 Thread Becket Qin
improve the ISR propagation performance. So I think if possible we may want to avoid using the same mechanism for offline replica propagation from broker to the controller again. That said, we can do that in a separate KIP also. Thanks, Jiangjie (Becket) Qin On Mon, Feb 20, 2017 at 10:17 PM, Jun

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

2017-02-21 Thread Becket Qin
they are in the same topic and similar fields. Thanks, Jiangjie (Becket) Qin On Tue, Feb 21, 2017 at 6:17 PM, Apurva Mehta wrote: > Hi Becket, Thanks for the kip. > > I think one of the risks here is that when compression estimation is > disabled, you could have much smaller

Re: [ANNOUNCE] Apache Kafka 0.10.2.0 Released

2017-02-22 Thread Becket Qin
Thanks Ewen :) On Wed, Feb 22, 2017 at 5:15 AM, Kenny Gorman wrote: > We are excited about this release! Excellent work! > > Thanks > Kenny Gorman > www.eventador.io > > > On Feb 22, 2017, at 2:33 AM, Ewen Cheslack-Postava > wrote: > > > > The Apache Kafka community is pleased to announce the r

Re: Messages corrupted in kafka

2016-03-23 Thread Becket Qin
some other clients? Jiangjie (Becket) Qin On Wed, Mar 23, 2016 at 8:28 PM, sunil kalva wrote: > can some one help me out here. > > On Wed, Mar 23, 2016 at 7:36 PM, sunil kalva > wrote: > > > Hi > > I am seeing few messages getting corrupted in kafka, It is not h

Re: Messages corrupted in kafka

2016-03-24 Thread Becket Qin
er i cant read and replay them again. > > On Thu, Mar 24, 2016 at 9:42 AM, Becket Qin wrote: > > > Hi Sunil, > > > > The messages in Kafka has a CRC stored with each of them. When consumer > > receives a message, it will compute the CRC from the message bytes an

Re: [VOTE] 0.10.0.0 RC1

2016-03-31 Thread Becket Qin
users. And I agree with Grant that it would be nice to have KIP-35 in this release. Thanks, Jiangjie (Becket) Qin On Wed, Mar 30, 2016 at 2:35 PM, Gwen Shapira wrote: > I think we are in agreement here. > Note that KIP-47 is already voted and merged AFAIK, so it will be in 0.10.0 > e

Re: [VOTE] 0.10.0.0 RC1

2016-03-31 Thread Becket Qin
Sure, do we have a new intended release close date? On Thu, Mar 31, 2016 at 10:23 AM, Gwen Shapira wrote: > I'd LOVE to have KIP-33 get it. > > Can you work with Jun to make sure the timing will work? > > On Thu, Mar 31, 2016 at 9:20 AM, Becket Qin wrote: > > > Hi

Re: [VOTE] 0.10.0.0 RC1

2016-03-31 Thread Becket Qin
h time do we > need? > > I think 2 or 3 weeks is reasonable? Am I being too optimistic again? > > Gwen > > > On Thu, Mar 31, 2016 at 10:42 AM, Becket Qin wrote: > > > Sure, do we have a new intended release close date? > > > > On Thu, Mar 31, 2016 at 10:23 AM,

[VOTE] KIP-33 - Add a time based log index

2016-04-06 Thread Becket Qin
ommit/712357a3fbf1423e05f9eed7d2fed5b6fe6c37b7 Thanks, Jiangjie (Becket) Qin

Re: [VOTE] KIP-33 - Add a time based log index

2016-04-11 Thread Becket Qin
ki to clarify the question raised in the comments and submit a PR by tomorrow. I am currently cleaning up the documentation. Thanks, Jiangjie (Becket) Qin On Sun, Apr 10, 2016 at 9:25 PM, Jun Rao wrote: > Hi, Jiangjie, > > Thanks for the update. Looks good to me overall. Just a few

Re: [VOTE] KIP-33 - Add a time based log index

2016-04-13 Thread Becket Qin
Hi Jun and Guozhang, I have updated the KIP wiki to incorporate your comments. Please let me know if you prefer starting another discussion thread for further discussion. Thanks, Jiangjie (Becket) Qin On Mon, Apr 11, 2016 at 12:21 AM, Becket Qin wrote: > Hi Guozhang and Jun, > > T

Re: [VOTE] KIP-33 - Add a time based log index

2016-04-14 Thread Becket Qin
ect this. BTW, it seems that the current code will never hit the case where an index is missing. I commented on PR. Thanks, Jiangjie (Becket) Qin On Thu, Apr 14, 2016 at 10:00 AM, Jun Rao wrote: > Hi, Jiangjie, > > 11. Rebuilding all missing time indexes will make the upgrade proce

Re: [RELEASE] Anyone minds if we push the next RC another week away?

2016-04-16 Thread Becket Qin
+1 > On Apr 15, 2016, at 11:20 AM, Harsha wrote: > > +1 > >> On Fri, Apr 15, 2016, at 08:06 AM, Grant Henke wrote: >> +1 >> >> On Fri, Apr 15, 2016 at 10:05 AM, Ashish Singh >> wrote: >> >>> Good idea. Thanks! >>> On Friday, April 15, 2016, Ismael Juma wrote: +1

Re: [VOTE] KIP-33 - Add a time based log index

2016-04-18 Thread Becket Qin
, we have to take a look at all the segments. If we search forward, we will stop at the first segment whose max timestamp is greater than 80 (i.e all the previous segments has smaller timestamps) and start the finer search on that segment. > 7. Do you mind if I fix typos and minor grammar issu

Re: [VOTE] KIP-33 - Add a time based log index

2016-04-18 Thread Becket Qin
hen CreateTime is used?). > > 6. We say "When searching by timestamp, broker will start from the > earliest > > log segment and check the last time index entry.". The existing logic > > searches from newest segment backwards. Is there a reason why we are > &g

Re: [VOTE] KIP-33 - Add a time based log index

2016-04-19 Thread Becket Qin
Thanks Joel and Ismael. I just updated the KIP based on your feedback. KIP-33 has passed with +4 (binding) and +2 (non-binding) Thanks everyone for the reading, feedback and voting! Jiangjie (Becket) Qin On Tue, Apr 19, 2016 at 5:25 PM, Ismael Juma wrote: > Thanks Becket. I think it would

Re: [VOTE] 0.10.0.0 RC4

2016-05-13 Thread Becket Qin
Tom, Maybe it is mentioned and I missed. I am wondering if you see performance degradation on the consumer side when TLS is used? This could help us understand whether the issue is only producer related or TLS in general. Thanks, Jiangjie (Becket) Qin On Fri, May 13, 2016 at 6:19 AM, Tom

Re: [VOTE] 0.10.0.0 RC4

2016-05-13 Thread Becket Qin
will setup some throughput test and see if I can reproduce this issue. Thanks, Jiangjie (Becket) Qin On Fri, May 13, 2016 at 11:41 AM, Gwen Shapira wrote: > Becket, > > Did you try deploying one of the 0.10.0 candidates at LinkedIn? Did > you see this issue? > > Gwen > &g

Re: [ANNOUNCE] New Kafka Committer Ewen Cheslack-Postava

2015-12-08 Thread Becket Qin
Congrats! Ewen! On Tue, Dec 8, 2015 at 11:39 AM, Edward Ribeiro wrote: > Congratulations, Ewen! :) > > Cheers, > Eddie > Em 08/12/2015 17:37, "Neha Narkhede" escreveu: > > > I am pleased to announce that the Apache Kafka PMC has voted to > > invite Ewen Cheslack-Postava as a committer and Ewen

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-12-10 Thread Becket Qin
think? Thanks, Jiangjie (Becket) Qin On Wed, Dec 9, 2015 at 2:42 PM, Jay Kreps wrote: > Hey Becket, > > That summary of pros and cons sounds about right to me. > > There are potentially two actions you could take when > max.message.time.difference is exceeded--override it or re

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-12-14 Thread Becket Qin
Hi Jun, Thanks a lot for the comments. Please see inline replies. Thanks, Jiangjie (Becket) Qin On Mon, Dec 14, 2015 at 10:19 AM, Jun Rao wrote: > Hi, Becket, > > Thanks for the proposal. Looks good overall. A few comments below. > > 1. KIP-32 didn't say what timestam

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-12-14 Thread Becket Qin
d build an empty log index. Thanks, Jiangjie (Becket) Qin On Mon, Dec 14, 2015 at 5:51 PM, Jun Rao wrote: > 1. I was thinking more about saving the decompression overhead in the > follower. Currently, the follower doesn't decompress the messages. To keep > it that way, the outer

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2015-12-15 Thread Becket Qin
mestamp". So in your example the "latest message" is 5. Thanks, Jiangjie (Becket) Qin On Tue, Dec 15, 2015 at 10:02 AM, Guozhang Wang wrote: > Jun, Jiangjie, > > I am confused about 3) here, if we use "the timestamp of the latest > message" > then do

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2015-12-22 Thread Becket Qin
Hi Anna, Thanks for initiating the voting process. I did not start the voting process because there were still some ongoing discussion with Jun about the timestamp regarding compressed messages. That is why the wiki page hasn't reflected the latest conversation as Guozhang pointed out. Like

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2015-12-28 Thread Becket Qin
message.format.version to use the release version. I did not change the internal version, we can discuss this in a separate thread. Thanks, Jiangjie (Becket) Qin > On Dec 24, 2015, at 5:38 AM, Guozhang Wang wrote: > > Also I agree with Gwen that such changes may worth a 0.10 release or even > 1.

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2016-01-05 Thread Becket Qin
implement everything else needed for message format change. > > Aside from that, I think the "Rejected Alternatives" section of the KIP is > excellent. Very good insight into what options were discussed and rejected. > > Aditya > >> On Mon, Dec 28, 2015 at 3

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2016-01-06 Thread Becket Qin
>> index for query but that is a separate proposal (KIP-33). Can we more >> crisply identify what exactly will change when this KIP (and 31) is >> implemented? It isn't super clear to me at this point. >> >> Aside from that, I think the "Rejected Alternatives"

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2016-01-06 Thread Becket Qin
clarified the scope of KIP-32 and KIP-33. Please let me know if you think we should make further improvements. Thanks, Jiangjie (Becket) Qin > On Jan 6, 2016, at 7:21 AM, Jay Kreps wrote: > > I'm +1 on the design, but this proposal doesn't look like it is quite > finished.

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2016-01-11 Thread Becket Qin
gt;>>>>> are doing the KIP” it probably shouldn’t have to exhaustively >> capture >>>>> “why >>>>>> we are doing the KIP *this way*”. i.e., some of the discussions are >>>>>> extremely nuanced and in this case spans multiple KIPs so links t

Re: Pluggable Log Compaction Policy

2016-01-21 Thread Becket Qin
I agree with Guozhang that this seems better to be a separate tool. Also, I am wondering if KIP-32 can be used here. We can have a timestamp based compaction policy if needed, for example, keep any message whose timestamp is greater than (MaxTimestamp - 24 hours). Jiangjie (Becket) Qin On Thu

Re: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-24 Thread Becket Qin
record setting while onAcknowledgement() is a producer level setting. Other than that, is there any difference between them? Thanks, Jiangjie (Becket) Qin On Fri, Jan 22, 2016 at 6:21 PM, Neha Narkhede wrote: > James, > > That is one of the many monitoring use cases for the in

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2016-01-26 Thread Becket Qin
o see if people have concerns over the modified approach. Thanks, Jiangjie (Becket) Qin On Tue, Dec 15, 2015 at 11:45 AM, Becket Qin wrote: > Jun, > > 1. I agree it would be nice to have the timestamps used in a unified way. > My concern is that if we let server change timestamp

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2016-01-26 Thread Becket Qin
type change on a topic rarely happens and if it is ever needed, the existing data should be wiped out. Thanks, Jiangjie (Becket) Qin On Tue, Jan 26, 2016 at 2:07 PM, Becket Qin wrote: > Bump up this thread per discussion on the KIP hangout. > > During the implementation of the KIP, G

Re: [DISCUSS] KIP-32 - Add CreateTime and LogAppendTime to Kafka message

2016-01-27 Thread Becket Qin
Hi Guozhang, That makes sense. I will update the KIP wiki and bump up the voting thread to let people know about this change. Thanks, Jiangjie (Becket) Qin On Tue, Jan 26, 2016 at 10:55 PM, Guozhang Wang wrote: > One motivation of my proposal is actually to avoid any clients trying to >

Re: [VOTE] KIP-32 Add CreateTime and LogAppendTime to Kafka message.

2016-01-27 Thread Becket Qin
. Based on the current consensus, I have updated the KIP wiki with changed proposal and will implement it Please let us know if you have any concern over the changed proposal. Thanks, Jiangjie (Becket) Qin On Thu, Jan 21, 2016 at 10:08 AM, Jiangjie Qin wrote: > I updated KIP-32 wiki with Anna

Re: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-27 Thread Becket Qin
Mayuresh, Regarding your use case about mirror maker. Is it good enough as long as we know there is no message for the topic in the producer anymore? If that is the case, call producer.flush() is sufficient. Thanks, Jiangjie (Becket) Qin On Tue, Jan 26, 2016 at 6:18 PM, Mayuresh Gharat wrote

Re: [DISCUSS] KIP-45 Standardize all client sequence interaction on j.u.Collection.

2016-01-27 Thread Becket Qin
sonally I think the current interface handles both single partition and a list of partitions better. It is not ideal that we have to adapt to the interface. I just feel it is weirder to create a new list. Thanks, Jiangjie (Becket) Qin On Wed, Jan 27, 2016 at 2:24 PM, Gwen Shapira wrote: > I

[DISCUSS] KIP-33 - Add a time based log index to Kafka

2016-01-27 Thread Becket Qin
touched during the discussion on KIP-32. So I also put the link to KIP-32 here for reference. https://cwiki.apache.org/confluence/display/KAFKA/KIP-32+-+Add+timestamps+to+Kafka+message Thanks, Jiangjie (Becket) Qin

Re: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-28 Thread Becket Qin
ConsumerRecord. Can you clarify the message size you are referring to? Does it include the message header overhead or not? From user's point of view, they probably don't care about header size. Thanks, Jiangjie (Becket) Qin On Wed, Jan 27, 2016 at 8:26 PM, Neha Narkhede wrote: > Anna, > &g

Re: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-28 Thread Becket Qin
, but (2) is difficult to get at Record level when we use compression. Thanks, Jiangjie (Becket) Qin On Thu, Jan 28, 2016 at 10:55 AM, Anna Povzner wrote: > Hi Becket, > > The use-case for CRC is end-to-end audit, rather than checking whether a > single message is corrupt or not. &

Re: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-29 Thread Becket Qin
e CRC could even be invalid if the broker ever overwritten any field or did format conversion. Thanks, Jiangjie (Becket) Qin On Thu, Jan 28, 2016 at 5:58 PM, Anna Povzner wrote: > On a second thought, yes, I think we should expose record size that > represents application bytes. This

Re: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-29 Thread Becket Qin
es to user. Other than this I think everything looks good. Nice work, Anna. Thanks, Jiangjie (Becket) Qin On Fri, Jan 29, 2016 at 8:11 AM, Joel Koshy wrote: > Responding to some of the earlier comments in the thread: > > @Jay/@Neha, > > I think any one of onCommit/onAppend/onAr

Re: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-29 Thread Becket Qin
to be faster than java CRC32 in Java 1.6. It seems no longer the case. What I see is that Java CRC32 class is 2x faster than the Crc32 class we are using now. Thanks, Jiangjie (Becket) Qin On Fri, Jan 29, 2016 at 1:46 PM, Anna Povzner wrote: > Joel, thanks for your feedback. I updated the wiki

Re: Pluggable Log Compaction Policy

2016-02-01 Thread Becket Qin
Hi Bill, The PR is still under review. It might take some more time because it touches a bunch of files. You can watch KAFKA-3025 so once it gets closed you will get email notification. Looking forward to your tool. Thanks, Jiangjie (Becket) Qin On Mon, Feb 1, 2016 at 6:54 AM, Bill Warshaw

Re: [VOTE] KIP-42 -- Add Producer and Consumer Interceptors.

2016-02-01 Thread Becket Qin
+1 (non-binding). Nice wiki. Thanks Anna. On Mon, Feb 1, 2016 at 12:00 PM, Anna Povzner wrote: > Hi All, > > I am opening the voting thread for KIP-42: Add Producer and Consumer > Interceptors. > > For reference, here's the KIP wiki: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-42%

Re: [DISCUSS] KIP-33 - Add a time based log index to Kafka

2016-02-02 Thread Becket Qin
Bump up. Because most of the changes in KIP-33 have been discussed in KIP-31 and KIP-32 earlier, if there is no further concerns or comments, I will initiate the voting shortly. Thanks, Jiangjie (Becket) Qin On Wed, Jan 27, 2016 at 7:17 PM, Becket Qin wrote: > Hi, > > Since KIP-31 a

[VOTE] KIP-33 - Add a time based log index to Kafka

2016-02-03 Thread Becket Qin
/confluence/display/KAFKA/KIP -32+-+Add+timestamps+to+Kafka+message Thanks, Jiangjie (Becket) Qin

Re: Kafka 0.9.0.1 plan

2016-02-05 Thread Becket Qin
, Jiangjie (Becket) Qin On Fri, Feb 5, 2016 at 10:28 AM, Jun Rao wrote: > Hi, Everyone, > > We have fixed a few critical bugs since 0.9.0.0 was released and are still > investigating a few more issues. The current list of issues tracked for > 0.9.0.1 can be found below. Among them,

[POLL] Make next Kafka Release 0.10.0.0 instead of 0.9.1.0

2016-02-09 Thread Becket Qin
what do people think of making the next release 0.10.0.0. Thanks, Jiangjie (Becket) Qin

Re: [POLL] Make next Kafka Release 0.10.0.0 instead of 0.9.1.0

2016-02-09 Thread Becket Qin
, Jiangjie (Becket) Qin On Tue, Feb 9, 2016 at 10:43 AM, Ismael Juma wrote: > Hi Becket, > > Thanks for starting the discussion. > > Given the significance of the changes, I think 0.10.0.0 is appropriate. > > However, I think we have to be careful about the fact that we wi

Re: [POLL] Make next Kafka Release 0.10.0.0 instead of 0.9.1.0

2016-02-19 Thread Becket Qin
Hi Ismael, I was trying to see what people think here. I will create an official [VOTE] thread. Thanks, Jiangjie (Becket) Qin On Thu, Feb 18, 2016 at 6:14 AM, Tom Graves wrote: > +1 > Tom > > On Tuesday, February 9, 2016 12:08 PM, Becket Qin < > becket@gmail.com>

[VOTE] Make next Kafka release 0.10.0.0 instead of 0.9.1.0

2016-02-19 Thread Becket Qin
, etc. We feel it is better to make next Kafka release 0.10.0.0 instead of 0.9.1.0. Some previous discussions are in the following thread. http://mail-archives.apache.org/mod_mbox/kafka-dev/201602.mbox/%3ccabtagwfzigx1frzd020vk9fanj0s9nkszfuwk677bqxfuuc...@mail.gmail.com%3E Thanks, Jiangjie (Becket

Re: [VOTE] Make next Kafka release 0.10.0.0 instead of 0.9.1.0

2016-02-19 Thread Becket Qin
the wishlist. After all, given all the new features we have already had, I think 0.10.0.0 is a justifiable major release. Thanks, Jiangjie (Becket) Qin On Fri, Feb 19, 2016 at 1:09 PM, Ashish Singh wrote: > +1 (non-binding) > > On Fri, Feb 19, 2016 at 1:03 PM, Jason Gustafson > wrote

Re: [DISCUSS] KIP-47 - Add timestamp-based log deletion policy

2016-02-19 Thread Becket Qin
against other existing configurations? As Jun pointed out, it seems work for CreateTime as well based on the current implementation plan of time based log index. Thanks, Jiangjie (Becket) Qin On Fri, Feb 19, 2016 at 4:02 PM, Jun Rao wrote: > Hi, Bill, > > I replied with the following

Re: [DISCUSS] KIP-47 - Add timestamp-based log deletion policy

2016-02-22 Thread Becket Qin
Hi Jun, I think it makes sense to implement KIP-47 after KIP-33 so we can make it work for both LogAppendTime and CreateTime. And yes, I'm actively working on KIP-33. I had a voting thread on KIP-33 before and I'll bump it up. Thanks, Jiangjie (Becket) Qin On Mon, Feb 22, 2016

Re: [DISCUSS] Deprecating the old Scala producers for the next release

2016-02-22 Thread Becket Qin
+1 on deprecating old producer. On Mon, Feb 22, 2016 at 9:36 AM, Ismael Juma wrote: > Hi all, > > The new Java producer was introduced in 0.8.2.0 (released in February > 2015). It has become the default implementation for various tools since > 0.9.0.0 (released in October 2015) and it is the onl

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-02-23 Thread Becket Qin
Bump. Per Jun's comments during KIP hangout, I have updated wiki with the upgrade plan or KIP-33. Let's vote! Thanks, Jiangjie (Becket) Qin On Wed, Feb 3, 2016 at 10:32 AM, Becket Qin wrote: > Hi all, > > I would like to initiate the vote for KIP-33. > > https://cw

Re: [VOTE] Make next Kafka release 0.10.0.0 instead of 0.9.1.0

2016-02-23 Thread Becket Qin
Thanks everyone for voting. The vote has passed with +6 (binding) and +5(non-binding) Jiangjie (Becket) Qin On Tue, Feb 23, 2016 at 2:38 PM, Harsha wrote: > +1 > > On Tue, Feb 23, 2016, at 02:25 PM, Christian Posta wrote: > > +1 non binding > > > > On Tue, Fe

Re: [VOTE] Make next Kafka release 0.10.0.0 instead of 0.9.1.0

2016-02-24 Thread Becket Qin
Thanks a lot for help out, Ismael. I was wondering why you changed some of the fix version back to 0.9.1.0 and now I understand. Thanks, Jiangjie (Becket) Qin On Tue, Feb 23, 2016 at 10:37 PM, Ismael Juma wrote: > I also changed the "Fix version" for a number of recent JIRAs

Re: Kafka Version 0.9.0 rebalance question

2016-02-24 Thread Becket Qin
ngjie (Becket) Qin On Wed, Feb 24, 2016 at 4:58 AM, tuyang wrote: > Hi, > after reviewed Kafka 0.9.0 code, I have a rebalance question: > coordinator will choose a client as leader to execute assign, > after transfer it to coordinator, when assignments are Propagate to

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-02-24 Thread Becket Qin
ex entry to find the message with correct timestamp. The good thing is that this guarantees there will not be huge time indices. We also save the new configuration. What do you think? Thanks, Jiangjie (Becket) Qin On Wed, Feb 24, 2016 at 1:00 PM, Guozhang Wang wrote: > Thanks Jiangjie, a f

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-02-24 Thread Becket Qin
made was that as long as a message timestamp shows time.index.interval.ms has passed since the timestamp of last time index entry, we will insert another timestmap index entry. Previously we always insert time index at time.index.interval.ms bucket boundaries. Thanks, Jiangjie (Becket) Qin On Wed

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-02-28 Thread Becket Qin
l create a separate ticket for the index file initial size. Thanks, Jiangjie (Becket) Qin On Thu, Feb 25, 2016 at 3:30 PM, Guozhang Wang wrote: > Jiangjie, > > I was originally only thinking about the "time.index.size.max.bytes" config > in addition to the "offset.inde

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-02-29 Thread Becket Qin
- because all the older segment with even larger timestamp have been removed. So in the case you mentioned, during startup we will remove all the segments and roll out a new empty segment. Thanks, Jiangjie (Becket) Qin On Mon, Feb 29, 2016 at 6:09 PM, Jun Rao wrote: > Hi, Becket, > >

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-03-01 Thread Becket Qin
g the acceptable time difference threshold appropriately. Thanks, Jiangjie (Becket) Qin On Tue, Mar 1, 2016 at 11:55 AM, Jun Rao wrote: > Jiangjie, > > Currently, we roll a new log segment if the index is full. We can probably > just do the same on the time index. This will bound the

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-03-01 Thread Becket Qin
index. Thanks, Jiangjie (Becket) Qin On Tue, Mar 1, 2016 at 4:30 PM, Jun Rao wrote: > Hi, Jiangjie, > > I was thinking perhaps just reusing index.interval.bytes is enough. Not > sure if there is much value in adding an additional time.index.interval.ms > . > > For 1, the time

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-03-02 Thread Becket Qin
>> > > > >> >> > >> -Jay > > > >> >> > >> > > > >> >> > >> > > > >> >> > >> > > > >> >> > >> On Mon, Oct 12, 2015 at 8:11 PM, Jiangjie Qin > > >

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-03-02 Thread Becket Qin
supported after the protocol negotiation finishes. Thanks, Jiangjie (Becket) Qin On Wed, Mar 2, 2016 at 5:58 PM, Dana Powers wrote: > In kafka-python we've been doing something like: > > if version >= (0, 9): > Do cool new stuff > elif version >= (0, 8, 2): &

Re: [DISCUSS] KIP-35 - Retrieve protocol version

2016-03-03 Thread Becket Qin
supported by the Kafka broker, which makes it become scenario C. Thoughts? Thanks, Jiangjie (Becket) Qin On Thu, Mar 3, 2016 at 11:24 AM, Ashish Singh wrote: > On Wed, Mar 2, 2016 at 8:29 PM, Becket Qin wrote: > > > Hi Jason, > > > > I was thinking that every time

Re: [VOTE] Deprecating the old Scala producers for 0.10.0.0

2016-03-03 Thread Becket Qin
+1 (non-binding) On Thu, Mar 3, 2016 at 3:39 PM, Gwen Shapira wrote: > +1 > > On Thu, Mar 3, 2016 at 2:36 PM, Ismael Juma wrote: > > Hi all, > > > > The new Java producer was introduced in 0.8.2.0 (released in February > > 2015). It has become the default implementation for various tools since

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-03-07 Thread Becket Qin
Hi Jun, What do you think about the above solution? I am trying to include KIP-33 into 0.10.0 because the log retention has been a long pending issue. Thanks, Jiangjie (Becket) Qin On Tue, Mar 1, 2016 at 8:18 PM, Becket Qin wrote: > Hi Jun, > > I see. If we only use index.interval.b

Re: [VOTE] Release plan - Kafka 0.10.0

2016-03-07 Thread Becket Qin
+1 (non-binding) BTW, if possible, I would like to have KIP-33 in 0.10.0 to solve the long pending retention issue. On Mon, Mar 7, 2016 at 3:26 PM, Joel Koshy wrote: > +1 > > On Mon, Mar 7, 2016 at 9:27 AM, Gwen Shapira wrote: > > > Greetings Kafka Developer Community, > > > > As you all know,

Re: [DISCUSS] KIP-45 Standardize all client sequence interaction on j.u.Collection.

2016-03-07 Thread Becket Qin
Hi Jason, Yes, 0.9 clients should still work with 0.10 brokers. Thanks, Jiangjie (Becket) Qin On Mon, Mar 7, 2016 at 4:10 PM, Jason Gustafson wrote: > +users > > On Mon, Mar 7, 2016 at 4:09 PM, Jason Gustafson > wrote: > > > Hey Ismael, > > > > Thanks f

Re: [VOTE] KIP-33 - Add a time based log index to Kafka

2016-03-08 Thread Becket Qin
retention becomes easier. And searching by timestamp is not complicated. Thanks, Jiangjie (Becket) Qin On Mon, Mar 7, 2016 at 2:52 PM, Becket Qin wrote: > Hi Jun, > > What do you think about the above solution? I am trying to include KIP-33 > into 0.10.0 because the log retention has

Re: [VOTE] KIP-33 - Add a time based log index

2016-08-30 Thread Becket Qin
large timestamp will be accepted and cause frequent log rolling. Any concern regarding this change? Thanks, Jiangjie (Becket) Qin On Mon, Jun 13, 2016 at 2:30 PM, Guozhang Wang wrote: > Thanks Jiangjie, > > I see the need for sensitive data purging, the above proposed change LGTM. &g

[DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-08-30 Thread Becket Qin
/confluence/pages/viewpage.action?pageId=65868090 Comments are welcome. Thanks, Jiangjie (Becket) Qin

Re: [VOTE] KIP-33 - Add a time based log index

2016-08-31 Thread Becket Qin
ng > > > On Tue, Aug 30, 2016 at 4:33 PM, Becket Qin wrote: > > > Hi folks, > > > > Here is another update on the change of time based log rolling. > > > > After the latest implementation, we encountered KAFKA-4099. The issue is > > that if users

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-08-31 Thread Becket Qin
reasonable delay bound. This use case was discussed in KIP-31/32. 3) TimestampOffset is just a simple container class of Timestamp and Offset. Thanks, Jiangjie (Becket) Qin On Wed, Aug 31, 2016 at 1:38 PM, Mayuresh Gharat wrote: > Hi Becket, > > Thanks for the write up. I had few minor

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-02 Thread Becket Qin
seful feature (that may need a interface change, but I honestly do not think people would frequently query many different timestamps for the same partition) Have a good long weekend! Thanks, Jiangjie (Becket) Qin On Fri, Sep 2, 2016 at 6:10 PM, Ismael Juma wrote: > Thanks for the proposal B

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-06 Thread Becket Qin
optimize for it and complicates the protocol. Thanks, Jiangjie (Becket) Qin On Mon, Sep 5, 2016 at 11:21 PM, Magnus Edenhill wrote: > Good write-up Qin, the API looks promising. > > I have one comment: > > 2016-09-03 5:20 GMT+02:00 Becket Qin : > > > The currently offsetsForTi

Re: [ANNOUNCE] New committer: Jason Gustafson

2016-09-06 Thread Becket Qin
Congrats, Jason! On Tue, Sep 6, 2016 at 5:09 PM, Onur Karaman wrote: > congrats jason! > > On Tue, Sep 6, 2016 at 4:12 PM, Sriram Subramanian > wrote: > > > Congratulations Jason! > > > > On Tue, Sep 6, 2016 at 3:40 PM, Vahid S Hashemian < > > vahidhashem...@us.ibm.com > > > wrote: > > > > > Co

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-07 Thread Becket Qin
Thanks everyone for all the feedback. If there is no further concerns or comments I will start a voting thread on this KIP tomorrow. Thanks, Jiangjie (Becket) Qin On Tue, Sep 6, 2016 at 9:48 AM, Becket Qin wrote: > Hi Magnus, > > Thanks for the comments. I agree that querying messag

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-07 Thread Becket Qin
am open to option (1) as well. Thanks, Jiangjie (Becket) Qin On Wed, Sep 7, 2016 at 11:06 AM, Jason Gustafson wrote: > Hey Becket, > > Thanks for the KIP. As I understand, the intention is to preserve the > current behavior with a timestamp of -1 indicating latest timestamp and -2

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-07 Thread Becket Qin
to the passed offset, so having > Long.MAX_VALUE return the latest value doesn't seem very intuitive to me. I > would actually expect it to return null. > > Given that, I think I prefer having the custom methods. What do you think? > > Thanks, > Jason > > On Wed, Sep 7,

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-08 Thread Becket Qin
er but there seems only minor difference. What do you think? I haven't seen a planned release date yet, but I can probably get it done in 2-3 weeks with reasonable rounds of reviews. Thanks, Jiangjie (Becket) Qin On Thu, Sep 8, 2016 at 6:24 PM, Jun Rao wrote: > Hi, Jiangjie, > > Th

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-09 Thread Becket Qin
ssage.NoTimestamp is also -1. Using -1 instead of null has at least two benefits. 1) it works for primitive type as well as classes. 2) it is easy to send via wire protocols For the other use cases, Option/Null could be better. Thanks, Jiangjie (Becket) Qin On Fri, Sep 9, 2016 at 8:06 AM, Ismael

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-09 Thread Becket Qin
() because the returned timestamp could be -1. So we can probably just make the interface only return the offsets. I'll make this change and start a voting thread. Thanks for all the feedback. Jiangjie (Becket) Qin On Fri, Sep 9, 2016 at 10:55 AM, Becket Qin wrote: > Completely agree that w

[VOTE] KIP-79 - ListOffsetRequest v1 and search by timestamp methods in new consumer.

2016-09-09 Thread Becket Qin
pages/viewpage.action?pageId=65868090 Thanks, Jiangjie (Becket) Qin

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-10 Thread Becket Qin
get rid all the primitive type fields in the messages (which is a backwards incompatible change). If it is a problem, we may still have the inconsistent representation of the missing values. Jiangjie (Becket) Qin On Sat, Sep 10, 2016 at 1:01 AM, Ismael Juma wrote: > Becket, comments inl

Re: Agenda item for next KIP meeting

2016-09-12 Thread Becket Qin
Hey Ismael, I probably replied too late, but is it possible to add KIP-79 to the agenda as well? Jiangjie (Becket) Qin On Fri, Sep 9, 2016 at 9:57 AM, radai wrote: > Hi, > > I'd like to discuss KIP-72. could you please add me? > > On Fri, Sep 9, 2016 at 7:48 AM, Ismael J

Re: [DISCUSS] KIP-79 - ListOffsetRequest v1 and offsetForTime() method in new consumer.

2016-09-13 Thread Becket Qin
timestamp support, I am happy to add that later. At this point it seems more of an efficiency optimization. So I just want to avoid unnecessarily complicate the protocol and risk the potential abuse. Thanks, Jiangjie (Becket) Qin On Tue, Sep 13, 2016 at 3:08 PM, Jason Gustafson wrote: >

Re: [VOTE] 0.10.1 Release Plan

2016-09-13 Thread Becket Qin
+1 (non-binding) On Tue, Sep 13, 2016 at 5:33 PM, Dana Powers wrote: > +1 >

Re: [DISCUSS] KIP-48 Support for delegation tokens as an authentication mechanism

2016-09-15 Thread Becket Qin
According to the meeting minutes of KIP hangout on 8/30, it seems the KIP wiki needs some update? KIP48 (delegation tokens): Harsha will update the wiki with more details on how to use delegation tokens and how to configure it. Not sure if that has been done or not. On Thu, Sep 15, 2016 at 3:20

<    1   2   3   4   >