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

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

Yeah, having commitAsync() and commitSync would be more clear. Are you 
suggesting the following API?

{code}
// Synchronous commit
void commitSync();
// Synchronous commit with an offset map
void commitSync(Map<TopicPartition, OffsetAndMetadata> offsetMap);
// Asynchronous commit without a callback
void commitAsync();
// Asynchronous commit
void commitAsync(OffsetCommitCallback callback);
// Asynchronous commit with an offset map without a callback
void commitAsync(Map<TopicPartition, OffsetAndMetadata> offsetMap) 
// Asynchronous commit with an offset map
void commitAsync(Map<TopicPartition, OffsetAndMetadata> offsetMap, 
OffsetCommitCallback callback);
{code}

The problem of having commitAsync(OffsetMap) and commitAsync(callback) is that 
user can not simply pass in a null because it is ambiguous. Personally I don't 
think there is too much difference in user experience between commitAsync(null) 
and commitAsync(). So personally I think the following API might look cleaner:

{code}
// Synchronous commit
void commitSync();
// Synchronous commit with an offset map
void commitSync(Map<TopicPartition, OffsetAndMetadata> offsetMap);
// Asynchronous commit
void commitAsync(OffsetCommitCallback callback);
// Asynchronous commit with an offset map
void commitAsync(Map<TopicPartition, OffsetAndMetadata> offsetMap, 
OffsetCommitCallback callback);
{code}

If user don't want a callback, they can just put null for the callback.

> 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