[ 
https://issues.apache.org/jira/browse/KAFKA-2389?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14708541#comment-14708541
 ] 

Jiangjie Qin commented on KAFKA-2389:
-------------------------------------

[~jkreps] I think a synchronous call taking a callback sounds a little bit 
weird. If it is synchronous, what is the point of having a callback? It looks 
intuitive to me that if the synchronous commit succeeds, the method return 
without any error, otherwise it throws exception.

So for commit, I think there are three cases:
1. Synchronous commit - commit(), commit(OffsetMap)
2. Asynchronous commit without a callback - commit(NoOpCallback), 
commit(OffsetMap, NoOpCallback)
3. Asynchronous commit with a callback - commit(callback), commit(OffsetMap, 
callback).

In this case, we only need the following commit methods:
{code}
// Synchronous commit
void commit();
// Synchronous commit with an offset map
void commit(Map<TopicPartition, OffsetAndMetadata> offsetMap);
// Asynchronous commit
void commit(OffsetCommitCallback callback);
// Asynchronous commit with an offset map
void commit(Map<TopicPartition, OffsetAndMetadata> offsetMap, 
OffsetCommitCallback callback);
{code}
We can make the name more explicit if we want.


> CommitType seems not necessary in commit().
> -------------------------------------------
>
>                 Key: KAFKA-2389
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2389
>             Project: Kafka
>          Issue Type: Sub-task
>            Reporter: Jiangjie Qin
>            Assignee: Jiangjie Qin
>
> The CommitType does not seem to be necessary in for commit(), it can be 
> inferred from whether user passed in a callback or not.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to