[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-15 Thread ASF GitHub Bot (JIRA)
uest at: https://github.com/apache/kafka/pull/79 > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-15 Thread ASF GitHub Bot (JIRA)
(at least) the following in the commit message: This closes #79 commit 6eb7ec648fdd95e9c73cf6c452c425527e6c800d Author: Jason Gustafson Date: 2015-07-16T00:10:12Z [Minor] fix new consumer heartbeat reschedule bug > Make new consumer offset commit API use callback +

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-15 Thread Guozhang Wang (JIRA)
king on this long-dragging patch! This is great work and I have just committed to trunk. > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://iss

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-15 Thread Guozhang Wang (JIRA)
: Resolved (was: Patch Available) > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jira/browse/KAFKA-2123 >

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-14 Thread Jason Gustafson (JIRA)
ttps://reviews.apache.org/r/36333/diff/ against branch upstream/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2123: --- Attachment: KAFKA-2123_2015-07-14_18:21:38.patch > Make new consumer offset commit API

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-14 Thread Jason Gustafson (JIRA)
ttps://reviews.apache.org/r/36333/diff/ against branch upstream/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-14 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2123: --- Attachment: KAFKA-2123_2015-07-14_13:20:25.patch > Make new consumer offset commit API

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-14 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2123: --- Reviewer: Guozhang Wang (was: Jay Kreps) > Make new consumer offset commit API use callback + fut

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-13 Thread Jason Gustafson (JIRA)
ttps://reviews.apache.org/r/36333/diff/ against branch upstream/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-13 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2123: --- Attachment: KAFKA-2123_2015-07-13_18:45:08.patch > Make new consumer offset commit API

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-11 Thread Jason Gustafson (JIRA)
ttps://reviews.apache.org/r/36333/diff/ against branch upstream/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-11 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2123: --- Attachment: KAFKA-2123_2015-07-11_17:33:59.patch > Make new consumer offset commit API

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-08 Thread Jason Gustafson (JIRA)
can bring it back if we think it adds a lot of value. > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jira/browse/KAFKA-2123

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-08 Thread Jason Gustafson (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jason Gustafson updated KAFKA-2123: --- Attachment: KAFKA-2123.patch > Make new consumer offset commit API use callback + fut

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-07-08 Thread Jason Gustafson (JIRA)
ttps://reviews.apache.org/r/36333/diff/ against branch upstream/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-06-16 Thread Guozhang Wang (JIRA)
5-05-04_09:39:50.patch, > KAFKA-2123_2015-05-04_22:51:48.patch, KAFKA-2123_2015-05-29_11:11:05.patch > > > The current version of the offset commit API in the new consumer is > void commit(offsets, commit type) > where the commit type is either sync or async. This means you need

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-06-16 Thread Guozhang Wang (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Guozhang Wang updated KAFKA-2123: - Priority: Critical (was: Major) > Make new consumer offset commit API use callback + fut

Re: [DISCUSS] New consumer offset commit API

2015-06-09 Thread Joel Koshy
is: > > > > > > > > > > > > void commit(Map offsets, CommitType type, > > > > > > ConsumerCommitCallback callback); > > > > > > > > > > > > For case 1) people call "commit(offsets)" which will block foreve

Re: [DISCUSS] New consumer offset commit API

2015-06-09 Thread Joel Koshy
(offsets, async)" which will return > > > > > immediately, with not callback upon finishes; > > > > > > > > > > For case 3) people call "commit(offsets, async, callback)", and the > > > > > callback will be executed when it

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-29 Thread Ewen Cheslack-Postava (JIRA)
5-05-04_09:39:50.patch, > KAFKA-2123_2015-05-04_22:51:48.patch, KAFKA-2123_2015-05-29_11:11:05.patch > > > The current version of the offset commit API in the new consumer is > void commit(offsets, commit type) > where the commit type is either sync or async. This means you need

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-29 Thread Ewen Cheslack-Postava (JIRA)
ttps://reviews.apache.org/r/33196/diff/ against branch origin/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-05 Thread Jun Rao (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jun Rao updated KAFKA-2123: --- Reviewer: Jay Kreps > Make new consumer offset commit API use callback + fut

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-04 Thread Ewen Cheslack-Postava (JIRA)
5-05-04_09:39:50.patch, > KAFKA-2123_2015-05-04_22:51:48.patch > > > The current version of the offset commit API in the new consumer is > void commit(offsets, commit type) > where the commit type is either sync or async. This means you need to use > sync if you ever want co

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-04 Thread Ewen Cheslack-Postava (JIRA)
ttps://reviews.apache.org/r/33196/diff/ against branch origin/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-04 Thread Ewen Cheslack-Postava (JIRA)
015-05-04_09:39:50.patch > > > The current version of the offset commit API in the new consumer is > void commit(offsets, commit type) > where the commit type is either sync or async. This means you need to use > sync if you ever want confirmation that the commit succeeded. Some >

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-04 Thread Ewen Cheslack-Postava (JIRA)
ttps://reviews.apache.org/r/33196/diff/ against branch origin/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-01 Thread Ewen Cheslack-Postava (JIRA)
keep this complexity in check. > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jira/browse/KAFKA-2123 > Project:

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-01 Thread Ewen Cheslack-Postava (JIRA)
ttps://reviews.apache.org/r/33196/diff/ against branch origin/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-05-01 Thread Ewen Cheslack-Postava (JIRA)
nents: clients, consumer >Reporter: Ewen Cheslack-Postava >Assignee: Ewen Cheslack-Postava > Fix For: 0.8.3 > > Attachments: KAFKA-2123.patch, KAFKA-2123_2015-04-30_11:23:05.patch, > KAFKA-2123_2015-05-01_19:33:19.patch > > >

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
d async to commit all offsets automatically). Only unusual cases where you're submitting the offsets map and doing partial commits might care about smarter behavior. > Make new consumer offset commit API use callback + future > -

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2123: - Status: Patch Available (was: In Progress) > Make new consumer offset commit

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
nents: clients, consumer >Reporter: Ewen Cheslack-Postava >Assignee: Ewen Cheslack-Postava > Fix For: 0.8.3 > > Attachments: KAFKA-2123.patch, KAFKA-2123_2015-04-30_11:23:05.patch > > > The current version of the offset commit

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-30 Thread Ewen Cheslack-Postava (JIRA)
ttps://reviews.apache.org/r/33196/diff/ against branch origin/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

Re: [DISCUSS] New consumer offset commit API

2015-04-27 Thread Guozhang Wang
t; > > > > > > This API will make much smaller changes to the current > implementations > > as > > > > well. Of course if we have a common scenario where users would really > > > care > > > > about the exact timeout for async commits, then Futur

Re: [DISCUSS] New consumer offset commit API

2015-04-23 Thread Ewen Cheslack-Postava
> > > > On Thu, Apr 16, 2015 at 1:00 PM, Jiangjie Qin > > > > > > wrote: > > > > > > > Hey Ewen, > > > > > > > > This makes sense. People usually do not want to stop consuming when > > > > committing offsets. >

Re: [DISCUSS] New consumer offset commit API

2015-04-22 Thread Bhavesh Mistry
ious > one. > > > > > > Another thing is that whether the future mechanism will only be applied > > to > > > auto commit or it will also be used in manual commit? Because in new > > > consumer we allow user to provide an offset map for offset commit. &

Re: [DISCUSS] New consumer offset commit API

2015-04-22 Thread Jay Kreps
used in manual commit? Because in new > > consumer we allow user to provide an offset map for offset commit. Simply > > canceling a previous pending offset commit does not seem to be ideal in > > this case because the two commits could be for different partitions. > > &g

Re: [DISCUSS] New consumer offset commit API

2015-04-22 Thread Guozhang Wang
e two commits could be for different partitions. > > Thanks. > > Jiangjie (Becket) Qin > > On 4/14/15, 4:31 PM, "Ewen Cheslack-Postava" wrote: > > >I'd like to get some feedback on changing the offset commit API in the new > >consumer. Since this is user

Re: [DISCUSS] New consumer offset commit API

2015-04-16 Thread Jiangjie Qin
be ideal in this case because the two commits could be for different partitions. Thanks. Jiangjie (Becket) Qin On 4/14/15, 4:31 PM, "Ewen Cheslack-Postava" wrote: >I'd like to get some feedback on changing the offset commit API in the new >consumer. Since this is user-f

[DISCUSS] New consumer offset commit API

2015-04-14 Thread Ewen Cheslack-Postava
I'd like to get some feedback on changing the offset commit API in the new consumer. Since this is user-facing API I wanted to make sure this gets better visibility than the JIRA ( https://issues.apache.org/jira/browse/KAFKA-2123) might. The motivation is to make it possible to do async co

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-14 Thread Ewen Cheslack-Postava (JIRA)
n the existing tests back into functional shape. The exact set will probably depend on how we decide to handle retries. > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2

[jira] [Commented] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-14 Thread Ewen Cheslack-Postava (JIRA)
ttps://reviews.apache.org/r/33196/diff/ against branch origin/trunk > Make new consumer offset commit API use callback + future > - > > Key: KAFKA-2123 > URL: https://issues.apache.org/jir

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2123: - Status: Patch Available (was: Open) > Make new consumer offset commit API

[jira] [Updated] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-14 Thread Ewen Cheslack-Postava (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-2123?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ewen Cheslack-Postava updated KAFKA-2123: - Attachment: KAFKA-2123.patch > Make new consumer offset commit API use callb

[jira] [Created] (KAFKA-2123) Make new consumer offset commit API use callback + future

2015-04-14 Thread Ewen Cheslack-Postava (JIRA)
Ewen Cheslack-Postava created KAFKA-2123: Summary: Make new consumer offset commit API use callback + future Key: KAFKA-2123 URL: https://issues.apache.org/jira/browse/KAFKA-2123 Project

[jira] [Commented] (KAFKA-1306) Offset commit API, does it work?

2014-03-19 Thread korebantic2 (JIRA)
correctly. I change the ID to the request as you indicated above, and I was able to submit an OffsetCommit request. I also confirmed the data was stored in zookeeper (/consumers/{consumer_name}/offsets/{topic}. So it looks good. > Offset commit API,

[jira] [Comment Edited] (KAFKA-1306) Offset commit API, does it work?

2014-03-19 Thread korebantic2 (JIRA)
7;t documented correctly. I change the ID to the request as you indicated above, and I was able to submit an OffsetCommit request. I also confirmed the data was stored in zookeeper /consumers/{consumer_name}/offsets/{topic} . So it looks good. > Offset commit

[jira] [Comment Edited] (KAFKA-1306) Offset commit API, does it work?

2014-03-19 Thread korebantic2 (JIRA)
rebantic2): Yup. As you said, it wasn't documented correctly. I change the ID to the request as you indicated above, and I was able to submit an OffsetCommit request. I also confirmed the data was stored in zookeeper ( /consumers/{consumer_name}/offsets/{topic} ). So it looks good. >

[jira] [Comment Edited] (KAFKA-1306) Offset commit API, does it work?

2014-03-19 Thread korebantic2 (JIRA)
s (Author: korebantic2): Yup. As you said, it wasn't documented correctly. I change the ID to the request as you indicated above, and I was able to submit an OffsetCommit request. I also confirmed the data was stored in zookeeper (/consumers/{consumer_name}/offsets/{topic}. So it looks good. >

[jira] [Resolved] (KAFKA-1306) Offset commit API, does it work?

2014-03-19 Thread korebantic2 (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1306?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] korebantic2 resolved KAFKA-1306. Resolution: Fixed > Offset commit API, does it w

[jira] [Commented] (KAFKA-1306) Offset commit API, does it work?

2014-03-19 Thread korebantic2 (JIRA)
been if a 6 was an 8. http://www.youtube.com/watch?v=VNXWMHu9An0 Thanks for the update, I'll work on testing it out on my end and update the issue once I verify. > Offset commit API, does it work? > > > Key: KAFKA-1306 >

[jira] [Commented] (KAFKA-1306) Offset commit API, does it work?

2014-03-18 Thread Jun Rao (JIRA)
for OffsetCommit is documented incorrectly in the wiki. Updated the wiki. > Offset commit API, does it work? > > > Key: KAFKA-1306 > URL: https://issues.apache.org/jira/browse/KAFKA-1306 > Project: Kafka >

[jira] [Commented] (KAFKA-1306) Offset commit API, does it work?

2014-03-18 Thread Jun Rao (JIRA)
e id for OffsetCommit is 8, instead of 6. > Offset commit API, does it work? > > > Key: KAFKA-1306 > URL: https://issues.apache.org/jira/browse/KAFKA-1306 > Project: Kafka > Issue Type: Bug >

[jira] [Updated] (KAFKA-1306) Offset commit API, does it work?

2014-03-17 Thread korebantic2 (JIRA)
[ https://issues.apache.org/jira/browse/KAFKA-1306?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] korebantic2 updated KAFKA-1306: --- Affects Version/s: 0.8.1 > Offset commit API, does it w

[jira] [Created] (KAFKA-1306) Offset commit API, does it work?

2014-03-16 Thread korebantic2 (JIRA)
korebantic2 created KAFKA-1306: -- Summary: Offset commit API, does it work? Key: KAFKA-1306 URL: https://issues.apache.org/jira/browse/KAFKA-1306 Project: Kafka Issue Type: Bug

Re: Offset commit api

2012-12-20 Thread Jay Kreps
ference to any associated state (say an HDFS > > >>> file > > >>> name) so that if the consumption fails over the new consumer can > start > > up > > >>> with the same state, this would be a place to store that. It would > not > > be

Re: Offset commit api

2012-12-20 Thread Milind Parikh
t;>> with the same state, this would be a place to store that. It would not > be > >>> intended to support large stuff (we could enforce a 1k limit or > >>> something, > >>> just something small or a reference on where to find the state (say a &g

Re: Offset commit api

2012-12-20 Thread David Arthur
ys, David has made a bunch of progress on the offset commit api implementation. Since this is a public API it would be good to do as much thinking up-front as possible to minimize future iterations. It would be great if folks could do the following: 1. Read the wiki here: https://cwiki.apache.org/**con

Re: Offset commit api

2012-12-20 Thread Jay Kreps
art up >>> with the same state, this would be a place to store that. It would not be >>> intended to support large stuff (we could enforce a 1k limit or >>> something, >>> just something small or a reference on where to find the state (say a >>> file >>>

Re: Offset commit api

2012-12-20 Thread Jay Kreps
ff (we could enforce a 1k limit or something, >> just something small or a reference on where to find the state (say a file >> name). >> >> Objections? >> >> -Jay >> >> >> On Mon, Dec 17, 2012 at 10:45 AM, Jay Kreps wrote: >> >>

Re: Offset commit api

2012-12-20 Thread David Arthur
ed to support large stuff (we could enforce a 1k limit or something, just something small or a reference on where to find the state (say a file name). Objections? -Jay On Mon, Dec 17, 2012 at 10:45 AM, Jay Kreps wrote: Hey Guys, David has made a bunch of progress on the offset commit api im

Re: Offset commit api

2012-12-20 Thread Jay Kreps
AM, Jay Kreps wrote: > Hey Guys, > > David has made a bunch of progress on the offset commit api implementation. > > Since this is a public API it would be good to do as much thinking > up-front as possible to minimize future iterations. > > It would be great if folks coul

Re: Offset commit api

2012-12-18 Thread Jay Kreps
Guys, > > > > David has made a bunch of progress on the offset commit api > implementation. > > > > Since this is a public API it would be good to do as much thinking > up-front > > as possible to minimize future iterations. > > > > It would be great

Re: Offset commit api

2012-12-18 Thread Jun Rao
Thanks for the proposal. Added a couple of comments to the wiki. Thanks, Jun On Mon, Dec 17, 2012 at 10:45 AM, Jay Kreps wrote: > Hey Guys, > > David has made a bunch of progress on the offset commit api implementation. > > Since this is a public API it would be good to do as m

Re: Offset commit api

2012-12-17 Thread Jay Kreps
to absolutely roll their own? > > Regards > Milind > > > > > > > On Mon, Dec 17, 2012 at 10:45 AM, Jay Kreps wrote: > > > Hey Guys, > > > > David has made a bunch of progress on the offset commit api > implementation. > > > > Since t

Re: Offset commit api

2012-12-17 Thread David Arthur
, Jay Kreps <mailto:jay.kr...@gmail.com>> wrote: Hey Guys, David has made a bunch of progress on the offset commit api implementation. Since this is a public API it would be good to do as much thinking up-front as possible to minimize future iterations. It w

Re: Offset commit api

2012-12-17 Thread Milind Parikh
against.Any other use cases for the clients to absolutely roll their own? Regards Milind On Mon, Dec 17, 2012 at 10:45 AM, Jay Kreps wrote: > Hey Guys, > > David has made a bunch of progress on the offset commit api implementation. > > Since this is a public API it would be

Offset commit api

2012-12-17 Thread Jay Kreps
Hey Guys, David has made a bunch of progress on the offset commit api implementation. Since this is a public API it would be good to do as much thinking up-front as possible to minimize future iterations. It would be great if folks could do the following: 1. Read the wiki here: https