incubator-s4-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From kishore g <g.kish...@gmail.com>
Subject Re: Handling connection failures in Netty
Date Sat, 12 Nov 2011 01:18:39 GMT
Sounds good to me. The buffer per partition must be small.

On Fri, Nov 11, 2011 at 4:57 PM, Karthik Kambatla <kkambatl@cs.purdue.edu>wrote:

> Okay.
>
> I ll implement the following simple fix -
>
> 1. If connection not open, try connecting.
> 2. If can't connect, queue to the buffer. If buffer full, return false
> 3. Have another thread which works on re-connections.
> 4. If connection open, check if the buffer has any messages queued, send
> them and then send the current one.
>
> Let me know if you think it is okay.
>
> Thanks
> Karthik
>
> On Fri, Nov 11, 2011 at 6:18 PM, kishore g <g.kishore@gmail.com> wrote:
>
> > It boils down to the simple question if one node(partition) is down do we
> > want the system to halt or continue sending data to other partition.
> Other
> > option is to provide zeromq implementation and it takes care of buffering
> > on the sender side per destination.
> >
> > We dont have to complicate the api, the api remains the same. This logic
> is
> > much simpler "if something fails while sending put in the retry buffer
> and
> > retry when we are able to connect again". Also this is only for the netty
> > implementation.
> >
> > thanks,
> > Kishore G
> >
> > On Fri, Nov 11, 2011 at 2:40 PM, Leo Neumeyer <leoneumeyer@gmail.com>
> > wrote:
> >
> > > Not sure I understand how this would help. We already have a queue in
> the
> > > stream for events. Once the framework gives up trying, blocking the
> queue
> > > seems like a natural solution, the queue will fill up and block. The
> app
> > > will remain blocked until the underlying problem is resolved. We don't
> > want
> > > to complicate the API unless, of course, there is a compelling reason.
> > > Let's wait until someone can make the case that this is required. More
> > > API-level features, means more complexity, more states, more potential
> > > bugs, more documentation, etc. What do you think?
> > >
> > > On Fri, Nov 11, 2011 at 2:04 PM, Karthik Kambatla <
> > kkambatl@cs.purdue.edu
> > > >wrote:
> > >
> > > > Kishore,
> > > >
> > > > One buffer per destination node seems fair enough. If we are
> > maintaining
> > > > buffers, how about just queue messages into this buffer on call to
> > > send(),
> > > > we can have another thread sending messages on these buffers across.
> > > >
> > > > Leo,
> > > >
> > > > It is true that the application can't do much, however the
> application
> > > can
> > > > choose to forget the old messages and see if the new messages can be
> > sent
> > > > across. The buffers that Kishore suggested can be circular,
> indicating
> > to
> > > > the application a full buffer from time to time.
> > > >
> > > > Thanks
> > > > Karthik
> > > >
> > > > On Fri, Nov 11, 2011 at 4:29 PM, Leo Neumeyer <leoneumeyer@gmail.com
> >
> > > > wrote:
> > > >
> > > > > Seems to me that the application shouldn't handle the failure
> because
> > > it
> > > > is
> > > > > pretty much a fatal error. The framework operator needs to do
> > something
> > > > > while the apps are blocked, the failure would affect all loaded
> > > > > applications. Once the framework gives up, there is no more
> recovery,
> > > > > stream queues will fill up and block, and the whole system will
> stop
> > > > until
> > > > > the operator resolves the problem. Is there any other possible
> > > scenario?
> > > > > What can an app do once it is informed other than log more errors?
> > > > >
> > > > > -leo
> > > > >
> > > > > On Fri, Nov 11, 2011 at 11:52 AM, Karthik Kambatla
> > > > > <kkambatl@cs.purdue.edu>wrote:
> > > > >
> > > > > > Yes, we have to retry connecting to the node but only a bounded
> > > number
> > > > of
> > > > > > times, after which we give up. If someone decides to use TCP,
it
> is
> > > for
> > > > > > guaranteed message delivery in which case the application needs
> to
> > be
> > > > > > informed.
> > > > > >
> > > > > > What we need to decide is whether to let the application handle
> an
> > > > > > exception or a return value?
> > > > > >
> > > > > > Karthik
> > > > > >
> > > > > > On Fri, Nov 11, 2011 at 2:41 PM, Leo Neumeyer <
> > leoneumeyer@gmail.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Presumably the scenario is that we will retry so I guess
we
> > should
> > > > log
> > > > > > each
> > > > > > > connection error when the exception is thrown.
> > > > > > >
> > > > > > > What can we do if we were to return false? We can always
do
> this
> > > > later
> > > > > > once
> > > > > > > we decide how to handle the error, no?
> > > > > > >
> > > > > > > -leo
> > > > > > >
> > > > > > > On Fri, Nov 11, 2011 at 11:25 AM, Karthik Kambatla
> > > > > > > <kkambatl@cs.purdue.edu>wrote:
> > > > > > >
> > > > > > > > How should we go about handling connection failures
(not
> being
> > > able
> > > > > to
> > > > > > > > connect to) for a send().
> > > > > > > >
> > > > > > > > We can either (1) throw java.net.ConnectException
or (2)
> return
> > > > > false.
> > > > > > In
> > > > > > > > the second case, we need to modify the Emitter.send
to return
> > > > > boolean.
> > > > > > > > Comments?
> > > > > > > >
> > > > > > > > Thanks
> > > > > > > > Karthik
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > >
> > > > > > > Leo Neumeyer (@leoneu)
> > > > > > >
> > > > > >
> > > > >
> > > > >
> > > > >
> > > > > --
> > > > >
> > > > > Leo Neumeyer (@leoneu)
> > > > >
> > > >
> > >
> > >
> > >
> > > --
> > >
> > > Leo Neumeyer (@leoneu)
> > >
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message