Home | About | Sematext search-lucene.com search-hadoop.com
 Search Hadoop and all its subprojects:

Switch to Threaded View
Kafka, mail # user - Producer.send questions


Copy link to this message
-
Re: Producer.send questions
Jason Rosenberg 2013-08-24, 16:54
Jun,

Thanks, this is helpful.

So, can QueueFullException occur in either sync or async mode (or just
async mode)?

If there's a MessageSizeTooLargeException, is there any visibility of this
to the caller?  Or will it just be a FailedToSendMessageException.  I
gathered from one of your previous responses, that a
MessageSizeTooLargeException can be rectified with a smaller batch size.
 If so, does that imply that the message size limit is measured on the
broker by the cumulative size of the batch, and not of any one message?
 (makes sense if the broker doesn't unwrap a batch of messages before
storing on the server).

If I want to implement guaranteed delivery semantics, using the new
request.required.acks configuration, I need to expose retry logic beyond
that built into the producer?  And to do this, I need to indicate to the
caller whether it's possible to retry, or whether it will be fruitless.  I
suppose allowing message.max.send.retries to allow infinite retries (e.g.
by setting it to -1) might be useful.  But optionally, I'd like the caller
to be able to handle this retry logic itself.

Jason
On Sat, Aug 24, 2013 at 8:22 AM, Jun Rao <[EMAIL PROTECTED]> wrote:

> You don't need to restart the producer. The producer currently handles all
> error/exceptions by refreshing the metadata and retrying. If it fails all
> retries, it throws a FailedToSendMessageException to the caller (in sync
> mode). The original cause is not included in this exception. We have
> thought about being a bit smarter in the producer retry logic such that it
> only retries on recoverable errors and could implement this at some point.
> Other than FailedToSendMessageException, the producer can also throw
> QueueFullException.
> This is an indication that the producer is sending data at a rate faster
> than the broker can handle. This may or may not be recoverable since it
> depends on the load.
>
> Thanks,
>
> Jun
>
>
> On Sat, Aug 24, 2013 at 1:44 AM, Jason Rosenberg <[EMAIL PROTECTED]> wrote:
>
> > Jun,
> >
> > There are several others I've seen that I would have thought would be
> > retryable (possibly after an exponential backoff delay).  I'm curious
> > about:
> >
> > BrokerNotAvailableException
> > FailedToSendMessageException
> > QueueFullException (happens if producerType is 'async')
> > KafkaException (this seems to wrap lots of base conditions, does one have
> > to sort through the different wrapped exception types?)
> > LeaderNotAvailableException
> > MessageSizeTooLargeException (does a batch of messages get treated as a
> > single message, when checking for message size too large?)
> > ReplicaNotAvailableException
> > UnavailableProducerException
> > UnknownException
> >
> > Also, what about my first question, regarding whether it makes sense to
> > refresh a producer by closing it and restarting it after a failure?
> >
> > Thanks,
> >
> > Jason
> >
> >
> > On Fri, Aug 23, 2013 at 9:07 PM, Jun Rao <[EMAIL PROTECTED]> wrote:
> >
> > > For the most part, only SocketExceptions and
> > NotLeaderForPartitionException
> > > are recoverable. MessageSizeTooLargeException may be recoverable with a
> > > smaller batch size.
> > >
> > > Thanks,
> > >
> > > Jun
> > >
> > >
> > > On Fri, Aug 23, 2013 at 4:09 PM, Jason Rosenberg <[EMAIL PROTECTED]>
> > wrote:
> > >
> > > > I'm using the kafka.javaapi.producer.Producer class from a java
> client.
> > > >  I'm wondering if it ever makes sense to refresh a producer by
> stopping
> > > it
> > > > and creating a new one, for example in response to a downstream IO
> > error
> > > > (e.g. a broker got restarted, or a stale socket, etc.).
> > > >
> > > > Or should it always be safe to rely on the producer's implementation
> to
> > > > manage it's pool of BlockingChannel connections, etc.
> > > >
> > > > I'm also interested in trying to understand which exceptions
> indicate a
> > > > failed send() request might be retryable (basically anything that
> > doesn't
> > > > involve a data-dependent problem, like a malformed message, or a