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
(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 +
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
: 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
>
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
[
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
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
[
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
[
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
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
[
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
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
[
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
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
[
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
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
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
[
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
is:
> > > > > >
> > > > > > void commit(Map offsets, CommitType type,
> > > > > > ConsumerCommitCallback callback);
> > > > > >
> > > > > > For case 1) people call "commit(offsets)" which will block foreve
(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
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
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
[
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
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
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
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
>
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
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:
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
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
>
>
>
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
> -
[
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
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
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
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
>
> > > 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.
>
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.
&
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
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
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
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
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
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
[
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
[
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
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
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,
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
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.
>
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.
>
[
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
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
>
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
>
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
>
[
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
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
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
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
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
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
>>>
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:
>>
>>
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
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
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
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
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
, 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
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
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
69 matches
Mail list logo