distributedlog-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Cameron Hatfield <kin...@gmail.com>
Subject Re: Proxy Client - Batch Ordering / Commit
Date Wed, 05 Oct 2016 21:58:26 GMT
Basically, its point 6 above. What ends up being the timeout for me to
ensure that a result is committed or not, so that I guarantee a read
actually gives me the last value??


-Cameron

On Wed, Oct 5, 2016 at 2:51 PM, Leigh Stewart <lstewart@twitter.com.invalid>
wrote:

> There is no automatic dedup or key based update.
>
> In failure scenarios you would need to recover by reading the latest id,
> and retry based on this.
>
> Would that work?
>
> On Wed, Oct 5, 2016 at 2:02 PM, Cameron Hatfield <kinguy@gmail.com> wrote:
>
> > Main question I have then is what happens in the case of retry due to
> > network failure, where the client doesn't get a success response, but it
> > succeeds / will succeed proxy server side?
> > What do I have to ensure that no duplicates are written to distributedlog
> > in such a case?
> >
> > -Cameron
> >
> > On Wed, Oct 5, 2016 at 11:33 AM, Leigh Stewart
> > <lstewart@twitter.com.invalid
> > > wrote:
> >
> > > >
> > > > So, my basic question is if this is currently possible in the proxy?
> I
> > > > don't believe it gives these guarantees as it stands today, but I am
> > not
> > > > 100% of how all of the futures in the code handle failures.
> > > >
> > >
> > > As long as you use this method
> > > <http://distributedlog.incubator.apache.org/docs/
> > latest/tutorials/basic-4>
> > > to write this is possible.
> > >
> > > The writeBulk is not atomic and we will probably deprecated it at some
> > > point.
> > >
> > > If not, where in the code would be the relevant places to add the
> ability
> > > > to do this, and would the project be interested in a pull request?
> > >
> > >
> > > Does the example linked above meet your requirements?
> > >
> > > Thx
> > >
> > > On Tue, Oct 4, 2016 at 12:39 PM, Cameron Hatfield <kinguy@gmail.com>
> > > wrote:
> > >
> > > > I have a question about the Proxy Client. Basically, for our use
> cases,
> > > we
> > > > want to guarantee ordering at the key level, irrespective of the
> > ordering
> > > > of the partition it may be assigned to as a whole. Due to the source
> of
> > > the
> > > > data (HBase Replication), we cannot guarantee that a single partition
> > > will
> > > > be owned for writes by the same client. This means the proxy client
> > works
> > > > well (since we don't care which proxy owns the partition we are
> writing
> > > > to).
> > > >
> > > >
> > > > However, the guarantees we need when writing a batch consists of:
> > > > Definition of a Batch: The set of records sent to the writeBatch
> > endpoint
> > > > on the proxy
> > > >
> > > > 1. Batch success: If the client receives a success from the proxy,
> then
> > > > that batch is successfully written
> > > >
> > > > 2. Inter-Batch ordering : Once a batch has been written successfully
> by
> > > the
> > > > client, when another batch is written, it will be guaranteed to be
> > > ordered
> > > > after the last batch (if it is the same stream).
> > > >
> > > > 3. Intra-Batch ordering: Within a batch of writes, the records will
> be
> > > > committed in order
> > > >
> > > > 4. Intra-Batch failure ordering: If an individual record fails to
> write
> > > > within a batch, all records after that record will not be written.
> > > >
> > > > 5. Batch Commit: Guarantee that if a batch returns a success, it will
> > be
> > > > written
> > > >
> > > > 6. Read-after-write: Once a batch is committed, within a limited
> > > time-frame
> > > > it will be able to be read. This is required in the case of failure,
> so
> > > > that the client can see what actually got committed. I believe the
> > > > time-frame part could be removed if the client can send in the same
> > > > sequence number that was written previously, since it would then fail
> > and
> > > > we would know that a read needs to occur.
> > > >
> > > >
> > > > So, my basic question is if this is currently possible in the proxy?
> I
> > > > don't believe it gives these guarantees as it stands today, but I am
> > not
> > > > 100% of how all of the futures in the code handle failures.
> > > > If not, where in the code would be the relevant places to add the
> > ability
> > > > to do this, and would the project be interested in a pull request?
> > > >
> > > >
> > > > Thanks,
> > > > Cameron
> > > >
> > >
> >
>

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