Cameron:
Another thing we've discussed but haven't really thought through -
We might be able to support some kind of epoch write request, where the
epoch is guaranteed to have changed if the writer has changed or the ledger
was ever fenced off. Writes include an epoch and are rejected if the epoch
has changed.
With a mechanism like this, fencing the ledger off after a failure would
ensure any pending writes had either been written or would be rejected.


On Sat, Oct 8, 2016 at 7:10 AM, Sijie Guo <si...@apache.org> wrote:

> Cameron,
>
> I think both Leigh and Xi had made a few good points about your question.
>
> To add one more point to your question - "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?"
>
> The current proxy and client logic doesn't do perfectly on handling
> failures (duplicates) - the strategy now is the client will retry as best
> at it can before throwing exceptions to users. The code you are looking for
> - it is on BKLogSegmentWriter for the proxy handling writes and it is on
> DistributedLogClientImpl for the proxy client handling responses from
> proxies. Does this help you?
>
> And also, you are welcome to contribute the pull requests.
>
> - Sijie
>
>
>
> On Tue, Oct 4, 2016 at 3:39 PM, Cameron Hatfield <kin...@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
> >
>

Reply via email to