Question about ZookeeperConsumerConnector

2014-11-10 Thread Becket Qin
$$anonfun$cleanShutdown$1.apply(MirrorMaker.scala:185) at scala.collection.immutable.List.foreach(List.scala:318) at kafka.tools.MirrorMaker$.cleanShutdown(MirrorMaker.scala:185) at kafka.tools.MirrorMaker$$anon$1.run(MirrorMaker.scala:169) Thanks. Jiangjie (Becket) Qin

Re: 0.9.0 release branch

2015-11-02 Thread Becket Qin
Hi Jun, I added KAFKA-2722 as a blocker for 0.9. It fixes the ISR propagation scalability issue we saw. Thanks, Jiangjie (Becket) Qin On Mon, Nov 2, 2015 at 9:16 AM, Jun Rao <j...@confluent.io> wrote: > Hi, everyone, > > We are getting close to the 0.9.0 release. The current

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

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

2015-12-07 Thread Becket Qin
thoughts? Thanks, Jiangjie (Becket) Qin On Mon, Dec 7, 2015 at 10:33 AM, Jiangjie Qin <j...@linkedin.com.invalid> wrote: > Bump up this thread. > > Just to recap, the last proposal Jay made (with some implementation details > added) was: > >1. Allow user to stamp the mes

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 <j...@confluent.io> 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.differen

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

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

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 <j...@confluent.io> wrote: > Hi, Becket, > > Thanks for the proposal. Looks good overall. A few comments below. > > 1. KIP-32 didn

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

2016-01-05 Thread Becket Qin
P-31 and KIP-32 will 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, 20

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.

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.

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 <wangg...@gmail.com> wrote: > > Also I agree with Gwen that such changes may worth

Re: [DISCUSS] KIP-62: Allow consumer to send heartbeats from a background thread

2016-06-06 Thread Becket Qin
by user thread? This is implementation detail but may be worth thinking about a bit more. Thanks, Jiangjie (Becket) Qin On Mon, Jun 6, 2016 at 11:27 AM, Guozhang Wang <wangg...@gmail.com> wrote: > Jiangjie: > > About doing the rebalance in the background thread, I'm

Re: [DISCUSS] scalability limits in the coordinator

2016-06-09 Thread Becket Qin
> >> > > > > > > > >> > > > > > > >> > > > > > Sorry - I'm officially confused. I think it may not be > required > >> - > >> > > since > >> > > > > the > >> > &

Re: [DISCUSS] KIP-62: Allow consumer to send heartbeats from a background thread

2016-06-05 Thread Becket Qin
lose messages if auto commit is enabled, or the manual commit might fail after a consumer.poll() because the partitions might have been reassigned. So having a separate rebalance timeout also potentially means a big change to the users as well. Thanks, Jiangjie (Becket) Qin On Fri, Jun 3, 2016

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 <j...@confluent.io> 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 fo

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 <ism...@juma.me.uk> 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 car

[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: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-28 Thread Becket Qin
that to 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 <n...@confluent.io> wrote:

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

2016-01-29 Thread Becket Qin
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 <a...@confluent.io> wrote: > On a second thought, yes, I think we should expose record size that > represents appli

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 <a...@confluent.io> wrote: > Hi Becket, > > The use-case for CRC is end-to-end audit, rather than checking whether a > single

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 <g...@conflue

[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-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 <wangg...@gmail.com> wrote: > One motivation of my proposal is actually to

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

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: > >

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 <becket@gmail.com> wrote:

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

2016-01-29 Thread Becket Qin
h used 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 <a...@confluent.io> wrote: > Joel, thanks for your feed

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: [DISCUSS] KIP-42: Add Producer and Consumer Interceptors

2016-01-29 Thread Becket Qin
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 <jjkosh...@gmail.com> wrote: > Responding to some of the earlier comments in the thread: > > @Jay/@Neha, > > I think any one

[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: [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 <ism...@juma.me.uk> wrote: > I also changed the "Fix version" for

Re: Kafka Version 0.9.0 rebalance question

2016-02-24 Thread Becket Qin
(Becket) Qin On Wed, Feb 24, 2016 at 4:58 AM, tuyang <tuyang19890...@126.com> 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 assignm

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

2016-02-24 Thread Becket Qin
index 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 <wangg...@gmail.com> wrote:

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 <becket@gmail.com> wrote: > Hi all, > > I would like to initiate the vote for KI

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 <ka...@harsha.io> wrote: > +1 > > On Tue, Feb 23, 2016, at 02:25 PM, Christian Posta wrote: > > +1 non binding > > &g

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

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

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: [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 at 9:11 AM

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

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 <n...@confluent.io> wrote: > James, > > That is one of the many monit

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

2016-01-26 Thread Becket Qin
have concerns over the modified approach. Thanks, Jiangjie (Becket) Qin On Tue, Dec 15, 2015 at 11:45 AM, Becket Qin <becket@gmail.com> 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 serve

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

2016-01-26 Thread Becket Qin
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 <becket@gmail.com> wrote: > Bump up this thread per discussion on the KIP hangout. > > During the

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 <tgraves...@yahoo.com.invalid> wrote: > +1 > Tom > > On Tuesday, February 9, 2016 12:08 PM, Bec

[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
igh on 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 <asi...@cloudera.com> wrote: > +1 (non-binding) > > On Fri, Feb 19, 2016 at 1:03 PM, J

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 <j...@confluent.io> wrote: > Hi, Bill, >

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 <j...@confluent.io> wrote:

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

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

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 <ja...@confluent.io> wrote: > +users > > On Mon, Mar 7, 2016 at 4:09 PM, Jason Gustafson <ja...@confluent.io> > wrote: > >

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

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

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

2016-04-11 Thread Becket Qin
ify 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 <j...@confluent.io> wrote: > Hi, Jiangjie, > > Thanks for the update. Looks good to me overall

Re: Messages corrupted in kafka

2016-03-24 Thread Becket Qin
re corrupted in > kafka broker i cant read and replay them again. > > On Thu, Mar 24, 2016 at 9:42 AM, Becket Qin <becket@gmail.com> wrote: > > > Hi Sunil, > > > > The messages in Kafka has a CRC stored with each of them. When consumer > > receives a m

Re: Messages corrupted in kafka

2016-03-23 Thread Becket Qin
or some other clients? Jiangjie (Becket) Qin On Wed, Mar 23, 2016 at 8:28 PM, sunil kalva <kalva.ka...@gmail.com> wrote: > can some one help me out here. > > On Wed, Mar 23, 2016 at 7:36 PM, sunil kalva <kalva.ka...@gmail.com> > wrote: > > > Hi > > I am seeing

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

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

2016-03-01 Thread Becket Qin
. Thanks, Jiangjie (Becket) Qin On Tue, Mar 1, 2016 at 4:30 PM, Jun Rao <j...@confluent.io> 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 > . >

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

2016-03-02 Thread Becket Qin
>> > > > >> >> > >> > > > >> >> > >> On Mon, Oct 12, 2015 at 8:11 PM, Jiangjie Qin > > > >> >> <j...@linkedin.com.invalid > > > >> >> > > > > > >> >> > >> wrote: > > > >>

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

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

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 <becket@gmail.com> wrote: > Hi Jun, > > I see.

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

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 <becket@gmail.com> wrote: > Hi Jun, > > What do you think about the above solution? I am trying to include KIP-33 > into 0.10.0 because

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 <g...@confluent.io> wrote: > I think we are in agreement here. > Note that KIP-47 is already voted and merg

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 <g...@confluent.io> 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

Re: [VOTE] 0.10.0.0 RC1

2016-03-31 Thread Becket Qin
-33 to get in. How much 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 <becket@gmail.com> wrote: > > > Sure, do we have a new intended release close date?

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 <becket@gmail.com> wrote: > Hi

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Becket Qin
. The metadata of JoinGroupRequests are likely similar so the aggregated metadata should be highly compressible. Thanks, Jiangjie (Becket) Qin On Mon, May 23, 2016 at 9:17 AM, Guozhang Wang <wangg...@gmail.com> wrote: > The original concern is that regex may not be efficiently supported

Re: [DISCUSS] scalability limits in the coordinator

2016-05-23 Thread Becket Qin
. 4. SyncGroupResponse will read the message, extract the assignment part and send back the partition assignment. We can compress the partition assignment before sends it out if we want. Jiangjie (Becket) Qin On Mon, May 23, 2016 at 5:08 PM, Jason Gustafson <ja...@confluent.io> wrote: >

Re: [VOTE] KIP-58 - Make Log Compaction Point Configurable

2016-05-24 Thread Becket Qin
lag" a bad thing? Thanks, Jiangjie (Becket) Qin On Tue, May 24, 2016 at 4:21 PM, Gwen Shapira <g...@confluent.io> wrote: > +1 (binding) > > Thanks for responding to all my original concerns in the discussion thread. > > On Tue, May 24, 2016 at 1:37 PM, Eric Wasserman <

Re: [ANNOUCE] Apache Kafka 0.10.0.0 Released

2016-05-24 Thread Becket Qin
Awesome! On Tue, May 24, 2016 at 9:41 AM, Jay Kreps wrote: > Woohoo!!! :-) > > -Jay > > On Tue, May 24, 2016 at 9:24 AM, Gwen Shapira wrote: > > > The Apache Kafka community is pleased to announce the release for Apache > > Kafka 0.10.0.0. > > This is a

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 <g...@confluent.io> wrote: > Becket, > > Did you try deploying one of the 0.10.0 candidates at LinkedIn? Did > you see this issue? &

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 <j...@confluent.io> wrote: > Hi, Jiangjie, > > 11. Rebuilding all missing time indexes wil

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

2016-04-18 Thread Becket Qin
h backwards, 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 gram

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

2016-04-18 Thread Becket Qin
; > timestamp (is this only when 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. I

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

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 <ism...@juma.me.uk> wrote: > Than

Re: [DISCUSS] scalability limits in the coordinator

2016-05-24 Thread Becket Qin
mat is the following: MemberMetadata => Version Generation ClientId Host Subscription Assignment So DescribeGroupResponse will just return the entire compressed GroupMetadataMessage. SyncGroupResponse will return the corresponding inner message. Thanks, Jiangjie (Becket) Qin On Tue, May 24, 2016 a

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

2016-08-08 Thread Becket Qin
happen even without the byte limit or we disable READ from the sockets. The only difference is that the broker won't have OOM if we have the bytes limit. Thanks, Jiangjie (Becket) Qin On Mon, Aug 8, 2016 at 10:04 AM, Jun Rao <j...@confluent.io> wrote: > Radai, > > Thanks f

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

2016-08-08 Thread Becket Qin
from the sockets when the RequestChannel is full also seems hurting the memory usage control effort. Thanks, Jiangjie (Becket) Qin On Mon, Aug 8, 2016 at 4:46 PM, radai <radai.rosenbl...@gmail.com> wrote: > I agree that filling up the request queue can cause clients to time out > (an

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

2017-02-02 Thread Becket Qin
a batching blocking API is fine? Thanks, Jiangjie (Becket) Qin On Thu, Feb 2, 2017 at 5:54 PM, Dong Lin <lindon...@gmail.com> 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 be

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 <apu...@confluent.io> wrote: > Hi Becket, Thanks for the kip. > > I think one of the risks here is that when compression estimation is > disabled, y

[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-124: Request rate quotas

2017-02-20 Thread Becket Qin
is 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
tains 100 messages, and the > 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 <becket@gmail.com> wr

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

2017-02-20 Thread Becket Qin
for 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 <wangg...@gmail.com>

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

2017-02-23 Thread Becket Qin
, Jiangjie (Becket) Qin On Wed, Feb 22, 2017 at 8:57 PM, Jay Kreps <j...@confluent.io> wrote: > Hey Becket, > > I get the problem we want to solve with this, but I don't think this is > something that makes sense as a user controlled knob that everyone sending > data to kaf

Re: KIP-122: Add a tool to Reset Consumer Group Offsets

2017-02-24 Thread Becket Qin
on offset commit)? They are probably in your proof of concept code. Could you add them to the wiki as well? Thanks, Jiangjie (Becket) Qin On Fri, Feb 24, 2017 at 1:19 PM, Vahid S Hashemian < vahidhashem...@us.ibm.com> wrote: > Thanks Jorge for addressing my question/suggestion. > >

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

2017-02-24 Thread Becket Qin
there are many messages need to be split at the same time. That could potentially be an issue for some users. What do you think about this approach? Thanks, Jiangjie (Becket) Qin On Thu, Feb 23, 2017 at 1:31 PM, Jay Kreps <j...@confluent.io> wrote: > Hey Becket, > > Yeah that makes sen

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-25 Thread Becket Qin
is already complicated, I would rather leave this out of the scope and address that later when needed, e.g. after having batch level interceptors. Thanks, Jiangjie (Becket) Qin On Fri, Feb 24, 2017 at 3:56 PM, Michael Pearce <michael.pea...@ig.com> wrote: > KIP updated in response to

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

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

2017-02-19 Thread Becket Qin
Jiangjie (Becket) Qin On Sat, Feb 18, 2017 at 9:35 PM, Dong Lin <lindon...@gmail.com> 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 q

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

2017-02-21 Thread Becket Qin
to 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: [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

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

2017-01-17 Thread Becket Qin
+1. Thanks for the KIP. On Thu, Jan 12, 2017 at 10:33 AM, Joel Koshy wrote: > +1 > > (for the record, I favor the rejected alternative of not awaiting low > watermarks to go past the purge offset. I realize it offers a weaker > guarantee but it is still very useful, easier

Re: [DISCUSS] 0.10.3.0/0.11.0.0 release planning

2017-02-27 Thread Becket Qin
Hi Ismael, Thanks for volunteering on the new release. I think 0.11.0.0 makes a lot of sense given the new big features we are intended to include. Thanks, Jiangjie (Becket) Qin On Mon, Feb 27, 2017 at 7:47 PM, Ismael Juma <ism...@juma.me.uk> wrote: > Hi all, > > With 0.10.2.0

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

2017-03-01 Thread Becket Qin
Thanks for the update. The changes sound reasonable. On Wed, Mar 1, 2017 at 1:57 PM, Dong Lin wrote: > Hi all, > > I have updated the KIP to include a script that allows user to purge data > by providing a map from partition to offset. I think this script may be >

Re: [VOTE] KIP-119: Drop Support for Scala 2.10 in Kafka 0.11

2017-03-01 Thread Becket Qin
at release X+1. It seems reasonable to do the same for Scala version here. So should we consider only making Scala version 2.11 as default in Kafka 0.11.0 and drop support for Scala 2.10 in Kafka 0.11.1? Thanks, Jiangjie (Becket) Qin On Wed, Mar 1, 2017 at 4:42 PM, Apurva Mehta <apu...@confluent

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

2017-02-27 Thread Becket Qin
e batch. Thanks, Jiangjie (Becket) Qin On Mon, Feb 27, 2017 at 10:30 AM, Mayuresh Gharat < gharatmayures...@gmail.com> wrote: > Hi Becket, > > Seems like an interesting idea. > I had couple of questions : > 1) How do we decide when the batch should be split? >

Re: [DISCUSS] KIP-82 - Add Record Headers

2017-02-28 Thread Becket Qin
ice, which is probably a more common case. Thanks, Jiangjie (Becket) Qin On Tue, Feb 28, 2017 at 12:43 PM, radai <radai.rosenbl...@gmail.com> wrote: > I will settle for any API really, but just wanted to point out that as it > stands right now the API targets the most "adva

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

2016-08-31 Thread Becket Qin
s sense to me. > > Guozhang > > > On Tue, Aug 30, 2016 at 4:33 PM, Becket Qin <becket@gmail.com> wrote: > > > Hi folks, > > > > Here is another update on the change of time based log rolling. > > > > After the latest implementation, we encountered KAFK

  1   2   3   4   >