Re: Proxy Client - Batch Ordering / Commit

2016-12-08 Thread Jon Derrick
On Fri, Nov 18, 2016 at 2:37 PM, Sijie Guo wrote: > On Thu, Nov 17, 2016 at 2:30 AM, Xi Liu wrote: > > > Cameron, > > > > Thank you for your comments. It's very helpful. My replies are inline. > > > > On Wed, Nov 16, 2016 at 11:59 AM, Cameron Hatfield

Re: Proxy Client - Batch Ordering / Commit

2016-11-17 Thread Xi Liu
Cameron, Thank you for your comments. It's very helpful. My replies are inline. On Wed, Nov 16, 2016 at 11:59 AM, Cameron Hatfield wrote: > "A couple of questions" is what I originally wrote, and then the following > happened. Sorry about the large swath of them, making sure

Re: Proxy Client - Batch Ordering / Commit

2016-11-16 Thread Sijie Guo
Cameron, I just granted you the permissions. You should be able to edit the wiki pages now. Let me know if you encountered any issues. - Sijie On Wed, Nov 16, 2016 at 12:20 PM, Cameron Hatfield wrote: > I believe it is: >

Re: Proxy Client - Batch Ordering / Commit

2016-11-16 Thread Cameron Hatfield
I believe it is: https://cwiki.apache.org/confluence/display/~cameron.hatfield On Wed, Nov 16, 2016 at 12:14 PM, Sijie Guo wrote: > Cameron, > > Can you send me your wiki account name? I can grant you the permission to > edit it. > > - Sijie > > On Wed, Nov 16, 2016

Re: Proxy Client - Batch Ordering / Commit

2016-11-16 Thread Cameron Hatfield
Also, would it be possible for me to get wiki access so I will be able to update it / etc? -Cameron On Wed, Nov 16, 2016 at 11:59 AM, Cameron Hatfield wrote: > "A couple of questions" is what I originally wrote, and then the following > happened. Sorry about the large swath

Re: Proxy Client - Batch Ordering / Commit

2016-11-09 Thread Xi Liu
Cameron, Have you started any work for this? I just updated the proposal page - https://cwiki.apache.org/confluence/display/DL/DP-2+-+Epoch+Write+Support Maybe we can work together with this. Sijie, Leigh, can you guys help review this to make sure our proposal is in the right direction? - Xi

Re: Proxy Client - Batch Ordering / Commit

2016-11-01 Thread Sijie Guo
I created https://issues.apache.org/jira/browse/DL-63 for tracking the proposed idea here. On Wed, Oct 26, 2016 at 4:53 PM, Sijie Guo wrote: > On Tue, Oct 25, 2016 at 11:30 AM, Cameron Hatfield > wrote: > > > Yes, we are reading the HBase WAL

Re: Proxy Client - Batch Ordering / Commit

2016-10-26 Thread Sijie Guo
On Tue, Oct 25, 2016 at 11:30 AM, Cameron Hatfield wrote: > Yes, we are reading the HBase WAL (from their replication plugin support), > and writing that into DL. > Gotcha. > > From the sounds of it, yes, it would. Only thing I would say is make the > epoch requirement

Re: Proxy Client - Batch Ordering / Commit

2016-10-19 Thread Sijie Guo
On Wed, Oct 19, 2016 at 7:17 PM, Sijie Guo wrote: > > > On Monday, October 17, 2016, Cameron Hatfield wrote: > >> Answer inline: >> >> On Mon, Oct 17, 2016 at 11:46 AM, Sijie Guo wrote: >> >> > Cameron, >> > >> > Thank you for your

Re: Proxy Client - Batch Ordering / Commit

2016-10-19 Thread Xi Liu
On Mon, Oct 17, 2016 at 3:14 PM, Cameron Hatfield wrote: > Answer inline: > > On Mon, Oct 17, 2016 at 11:46 AM, Sijie Guo wrote: > > > Cameron, > > > > Thank you for your summary. I liked the discussion here. I also liked the > > summary of your requirement -

Re: Proxy Client - Batch Ordering / Commit

2016-10-14 Thread Cameron Hatfield
> Ah- yes good point (to be clear we're not using the proxy this way today). > > 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. > Do you mean you *need* to support multiple writers issuing

Re: Proxy Client - Batch Ordering / Commit

2016-10-07 Thread Xi Liu
We investigated DL for a similar use case. I believed 1-5 are already met with current proxy with atomic-writes. However there is a limitation about how large a batch can be. The limitation is 1 megabytes, which I believe it is the limitation of the size of a bookkeeper entry. 6 is guaranteed if

Re: Proxy Client - Batch Ordering / Commit

2016-10-05 Thread Leigh Stewart
> > 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