[ 
https://issues.apache.org/jira/browse/KAFKA-1863?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14278104#comment-14278104
 ] 
Jay Kreps commented on KAFKA-1863:
----------------------------------

[~guozhang] Makes sense. You're correct that the docs on this suck, the send 
method doesn't even list the exceptions it throws so there is no way to figure 
this out except by reading the code.

[~criccomini] Your understanding is correct. Setting 
max.in.flight.requests.per.connection=1 and retries=infinity should give you 
what you want.

send() can throw exceptions in certain cases. Specifically if:
1. The serialization of the message fails.
2. You time out waiting for metadata about the cluster/topic on the very first 
request
3. You configure the client to fail rather than block if it's internal queue is 
full and the internal queue is full
4. You set a partition on the ProducerRecord which is invalid (i.e. larger than 
the largest) and various other IllegalArgumentExceptions

I think none of these should be an issue since (1) is a bug, (2) should be 
configured to infinite for you, (3) you won't do, and (4) is a bug.

The mental model here is that we throw the exception directly if we can't even 
enqueue the message. 

The next question is, great, you have given me information about each 
individual send but how can I know that all my stuff is sent correctly. The 
easiest thing here would be to hang on to all the futures and just call get on 
them as you suggest.

So I think the take aways are:
1. We need to document the errors (and rename that one)
2. We need to document the retry behavior with multiple in-flight requests
3. It might be nice to add a flush() call that waits for all the currently 
buffered or in-flight requests to complete assuming this doesn't add 
inefficiency in the normal case.







> Exception categories / hierarchy in clients
> -------------------------------------------
>
>                 Key: KAFKA-1863
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1863
>             Project: Kafka
>          Issue Type: Bug
>            Reporter: Guozhang Wang
>            Assignee: Guozhang Wang
>             Fix For: 0.9.0
>
>
> In the new clients package we introduces a new set of exceptions, but its 
> hierarchy is not very clear as of today:
> {code}
> RuntimeException -> KafkaException -> BufferExhastedException
>                                                            -> ConfigException
>                                                            -> 
> SerializationException
>                                                            -> 
> QuotaViolationException
>                                                            -> SchemaException
>                                                            -> ApiException
> ApiException -> InvalidTopicException
>                      -> OffsetMetadataTooLarge (probabaly need to be renamed)
>                      -> RecordBatchTooLargeException
>                      -> RecordTooLargeException
>                      -> UnknownServerException
>                      -> RetriableException
> RetriableException -> CorruptRecordException
>                                -> InvalidMetadataException
>                                -> NotEnoughtReplicasAfterAppendException
>                                -> NotEnoughReplicasException
>                                -> OffsetOutOfRangeException
>                                -> TimeoutException
>                                -> UnknownTopicOrPartitionException
> {code}
> KafkaProducer.send() may throw KafkaExceptions that are not ApiExceptions; 
> other exceptions will be set in the returned future metadata.
> We need better to
> 1. Re-examine the hierarchy. For example, for producers only exceptions that 
> are thrown directly from the caller thread before it is appended to the batch 
> buffer should be ApiExceptions; some exceptions could be renamed / merged.
> 2. Clearly document the exception category / hierarchy as part of the release.
> [~criccomini] may have some more feedbacks for this issue from Samza's usage 
> experience. [~jkreps]



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

Reply via email to