incubator-s4-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Leo Neumeyer <leoneume...@gmail.com>
Subject Re: Thoughts on adding guaranteed message processing
Date Tue, 07 Aug 2012 18:37:17 GMT
Hi all. Some thoughts.

Building a fault tolerant system would require queuing N events at the
source. The length of the queue would depend on the frequency of
checkpointing. Post failure, the system would need to restore the state of
the node from checkpointing and re-apply all the events emitted since the
time of checkpointing. To increase reliability, we would also need to
replicate the nodes. One would also need to account for the peak data rate
to make sure there is enough capacity to re-process all the data and comply
with the real-time constrains.

All of this is possible but I'm not sure this is the best platform for
applications that expect zero errors. It was designed for processing large
amounts of data in application that can tolerate a small probability of
error. Improving the platform to reduce the errors is much simpler than
trying to achieve zero errors.

Perhaps a better approach is to have good error detection so the
application can handle the recovery at a higher level (not in real-time).

Rather than try to solve ALL the problems, I think that it is better to
focus on problems that involve statistical processing of massive amounts of
noisy and redundant data where a small probability of errors will not
affect the accuracy of the results. (text processing, signal processing,
sensor data, market data, etc.) The advantage of focusing is that we can
solve one problem well and keep the system as simple as possible.

Regarding the reliability at the communication layer, using TCP should work
fine, I think.

Of course sending email is easy, I wish I had more time to put my code
where my mouth is :-)

-leo

On Tue, Aug 7, 2012 at 10:15 AM, Karthik Kambatla <kkambatl@cs.purdue.edu>wrote:

> Hi Flavio,
>
> We are in agreement. I was trying to push the discussion further, and get
> your inputs to decide on a plausible approach in S4.
>
> Regarding exactly-once semantics, I understand we need to plug multiple
> holes in a failing environment. To ensure we actually can support reliable
> delivery, should we outline possible failures and how they can be
> addressed. I might be able to think threw and list the steps (and possible
> failures) later today/tomorrow.
>
> On a side note, I remember Kishore and Leo initiating a conversation on
> using ZeroMQ in the comm-layer. ZeroMQ, I have heard, supports reliable
> delivery, and they claim to be faster than TCP.
>
> Thanks
> Karthik
>
> On Tue, Aug 7, 2012 at 6:53 AM, Flavio Junqueira <fpj@yahoo-inc.com>
> wrote:
>
> > I didn't mean to suggest a different way, I was trying to understand the
> > definition of exactly-once. As for the use case Benjamin has posted, he
> > says that not even a single tag scan can be lost, but can we guarantee
> that
> > events are reliably delivered at-least-once with S4?
> >
> > -Flavio
> >
> > On Aug 7, 2012, at 7:38 AM, Karthik Kambatla wrote:
> >
> > > Given that nodes crash, it seems essential to build reliability like it
> > is
> > > in lower layers - think TCP. One trivial approach could be to use
> > > monotonically increasing sequence numbers to identify events in a
> stream.
> > >
> > >   1. Event ordering: Hold events until all the previous sequence
> numbers
> > >   have been received.
> > >   2. Exactly-once: If a sequence number is smaller than previous one,
> it
> > >   is a duplicate.
> > >   3. Fault-tolerance: Store the latest sequence number along with the
> > >   checkpoint, replay events from there onwards.
> > >
> > > This, of course, comes at a performance overhead and should be
> optional.
> > >
> > > As I said, this is the first approach that comes to mind. It is indeed
> an
> > > interesting problem, and I feel we should not need to re-do stuff that
> is
> > > done at lower layers.
> > >
> > > Please suggest improvements/alternatives as you see fit.
> > >
> > > Thanks
> > > Karthik
> > >
> > > On Mon, Aug 6, 2012 at 10:01 PM, Flavio Junqueira <fpj@yahoo-inc.com>
> > wrote:
> > >
> > >>
> > >> On Aug 6, 2012, at 10:11 PM, Karthik Kambatla wrote:
> > >>
> > >> Flavio - it is indeed tricky to offer exactly-once semantics. My
> > >> understanding is that the underlying comm-layer could filter out
> > subsequent
> > >> duplicate events; however, we need to sacrifice ordering.
> > >>
> > >>
> > >> I was also thinking that if a node crash and we recover from
> > checkpoints,
> > >> we may end up having messages applied twice.
> > >>
> > >> -Flavio
> > >>
> > >> Thanks
> > >> Karthik
> > >>
> > >> On Mon, Aug 6, 2012 at 6:43 AM, "Benjamin Süß" <Gothic13@gmx.de>
> wrote:
> > >>
> > >>> Hi Matthieu,
> > >>>
> > >>> thank you for your reply. I had a specific use case in mind, indeed:
> > >>>
> > >>> I am trying to track RFID tags in distributed systems. This means,
> that
> > >>> not even a single tag scan may get lost. And of course, none are to
> be
> > sent
> > >>> twice or even more often as this would heavily confuse any
> surveillance
> > >>> routines I am going to implement.
> > >>>
> > >>> Regarding your answers, especially point 3, I do not think this can
> be
> > >>> done with S4 at the moment, can it?
> > >>>
> > >>> Regards,
> > >>> Benjamin
> > >>>
> > >>> -------- Original-Nachricht --------
> > >>>> Datum: Tue, 31 Jul 2012 17:30:27 +0200
> > >>>> Von: Matthieu Morel <mmorel@apache.org>
> > >>>> An: s4-user@incubator.apache.org
> > >>>> Betreff: Re: Thoughts on adding guaranteed message processing
> > >>>
> > >>>> On 7/31/12 2:54 PM, "Benjamin Süß" wrote:
> > >>>>> Hi there,
> > >>>>>
> > >>>>> it is stated in several places that S4 does not include guaranteed
> > >>>> one-time message processing. So my question is: are there currently
> > any
> > >>> plans on
> > >>>> adding this to S4? Or is it certain this is not going to happen?
If
> > >>> there
> > >>>> are any plans on this, can I find further information somewhere?
> > >>>>>
> > >>>>
> > >>>> There are typically 3 requirements for guaranteeing one-time message
> > >>>> processing:
> > >>>>
> > >>>> 1. reliable communication channels
> > >>>>
> > >>>> 2. replayable input stream: you need an upstream component that
is
> > able
> > >>>> to store/bufferize the whole stream and replay on demand.
> > >>>>
> > >>>> 3. tracking of messages, using some sort of piggybacking, possibly
> > >>>> requiring manual input from the user.
> > >>>>
> > >>>>
> > >>>> In S4 0.5.0, we already address 1. by providing communications
> through
> > >>>> TCP by default. Requirement 2. is quite straightforward to
> implement,
> > by
> > >>>> adding some machinery to connect to a component such as Apache
Kafka
> > for
> > >>>> instance. We are considering options for 3.
> > >>>>
> > >>>>
> > >>>> Do you have a specific use case in mind?
> > >>>>
> > >>>> Regards,
> > >>>>
> > >>>> Matthieu
> > >>>>
> > >>>
> > >>
> > >>
> > >>
> >
> >
>



-- 
*Leo Neumeyer*
Software, data, algorithms.
leoneumeyer@gmail.com
*http://www.linkedin.com/in/leoneu*

See who we know in
common<http://www.linkedin.com/e/wwk/27360/?hs=false&tok=3SHIsP1c6dURk1>Want
a signature like
this?<http://www.linkedin.com/e/sig/27360/?hs=false&tok=0r9XKRwP2dURk1>

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