You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hbase.apache.org by Eric Owhadi <er...@esgyn.com> on 2015/09/05 02:06:24 UTC

RE: Question on hbase.client.scanner.timeout.period

OK so to answer the "is it easy to insert the patched scanner for
trafodion", the answer is no.
Was easier on .98, but on 1.0 it was quite a challenge. All about dealing
with private attributes instead of protected that are not visible to the
PatchClentScanner class that extends ClientScanner.
Currently running the regression tests to see if there is no side effect...
Was able to demonstrate with breakpoint on next() waiting more than 1 mn
(the default lease timeout value) that with the patch things gracefully
reset and all is good, no row skipped or duplicated, while without, I get
the Scanner time out exception. Patch can be turn on or off with a new key
in hbase-site.xml...
I will feel better when this will be deprecated :-).
Eric Owhadi

-----Original Message-----
From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of Stack
Sent: Friday, August 28, 2015 6:35 PM
To: HBase Dev List <de...@hbase.apache.org>
Subject: Re: Question on hbase.client.scanner.timeout.period

On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi <er...@esgyn.com> wrote:

> That sounds good, but given trafodion needs to work on current and
> future released version of HBase, unpatched, I will first implement a
> ClientScannerTrafodion (to be deprecated), inheriting from
> ClientScanner that will just overload the loadCache(),and make sure
> that the code that is picking up the right scanner based on scan
> object is bypassed to force getting the ClientScannerTrafodion when
> appropriate.
> Not very elegant, but need to take into consideration trafodion
> deployment requirements.
> Then, if we do not discover any side effect during our QA related to
> this code I will port the fix on HBase to deprecate the custom scanner
> (probably first on HBase 2.0, then will let the community decide if
> this fix is worth it for back porting...). It will be a first for me,
> but that's great, I'll take your offer to help ;-)...
>

Sweet. Suggest opening an umbrellas issue in hbase to implement this
feature. Reference HBASE-2161 (it is closed now). Link trafodion issue to
it. A subtask could have implementation in hbase 2.0, another could be
backport.

Is is easy to insert your T*ClientScanner?
St.Ack



> Regards,
> Eric
>
> -----Original Message-----
> From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> Stack
> Sent: Thursday, August 27, 2015 3:55 PM
> To: HBase Dev List <de...@hbase.apache.org>
> Subject: Re: Question on hbase.client.scanner.timeout.period
>
> On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi <er...@esgyn.com>
> wrote:
>
> > Oops, my bad, the related JIRA was :
> > https://issues.apache.org/jira/browse/HBASE-2161
> >
> > I am suggesting that the special code client side in loadCache() of
> > ClientScanner that is trapping the UnknownScannerException, then on
> > purpose check if it is coming from a lease timeout (and not by a
> > region move) to decide that it would throw a ScannerTimeoutException
> > instead of letting the code go and just reset the scanner and start
> > from last successful retrieve (the way it works for an
> > unknowScannerException due to a region moving).
> > By just removing the special handling that tries to differentiate
> > from unkownScannerException due to lease timeout, we should have a
> > resolution to JIRA 2161- And to our trafodion issue.
> >
> > We are still protecting against dead client that would cause
> > resource leak at region server, since we keep the lease timeout
> > mechanism.
> >
> > Not sure if I have overlooked something, as usually, code is here
> > for a reason :-)...
> >
> >
> Your proposal sounds good to me.
>
> Scanner works the way it does because it has always work this way (smile).
> A while back, one of the lads suggested we do like dynamodb and have
> scanner have no state on the serverside, the scan next would just
> supply all necessary context. It was argued against because serverside
> setup is so costly. Your suggestion is similar only we do it only if
> Scanner has timed out.
>
> Suggest we keep the current semantic in 1.x at least. We could flip to
> your behavior in 2.x.  Meantime, you'd have to ask for it when you set
> up your Scan object by setting a flag.
>
> Would that work? If you want to have a go at it, I could help out on
> the issue.
>
> St.Ack
>
>
>
>
> > Regards,
> > Eric
> >
> >
> >
> > -----Original Message-----
> > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> > Stack
> > Sent: Thursday, August 27, 2015 3:23 PM
> > To: HBase Dev List <de...@hbase.apache.org>
> > Subject: Re: Question on hbase.client.scanner.timeout.period
> >
> > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi <er...@esgyn.com>
> > wrote:
> >
> > > Hello St.Ack,
> > > Thanks for your pointer, but I had already investigated JIRA
> > > https://issues.apache.org/jira/browse/HBASE-13090
> > > Unfortunately, this heartbeat will protect against rpc timeout,
> > > not server side lease timeout that we are experiencing right now.
> > > I have not seen an active JIRA fixing our issue.
> > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > complaining about the exact same issue, but was never resolved.
> > >
> > >
> > Which issue? https://issues.apache.org/jira/browse/HBASE-6121 seems
> > unrelated.
> >
> >
> >
> > > The heartbeat JIRA in 13090 protect for situation where server
> > > scanner takes so long to retrieve the highly filtered information,
> > > that it exceeds the RPC timeout (hbase.rpc.timeout).
> >
> >
> >
> > > The timeout we are experiencing is the
> > > hbase.client.scanner.timeout.period,
> > > also deprecatedly known as hbase.regionserver.lease.period The
> > > mechanism is different: here, region server scanners wants to
> > > protect themselves against dead clients that would not perform
> > > "close", and allow releasing server side scanner resources. To do
> > > that, a lease mechanism is implemented, and if between 2 next()
> > > call, more than hbase.regionserver.lease.period occurs, the server
> > > side scanner will have been forced closed by this lease timeout
> > > safety mechanism. On late next() call, client will receive a
> > > DNRIOE of type unknownScannerException, and the client will assess
> > > that it is coming most likely from the lease timeout (and not from
> > > a region move), therefore throwing an exception instead of reset
> > > scanner (for the region move scenario).
> > >
> > > Hbase 1.1 does not address, as far as I have researched, the
> > > hbase.client.scanner.timeout.period issue we are facing.
> > >
> > >
> >
> > Can you not have the high-level query that is being fed by a scan do
> > HBASE-13333? That is, tickle, the ongoing scan on occasion just to
> > say that I'm still alive?
> >
> > Otherwise, what would you suggest? A scan that does not timeout? Or
> > the client being able to set a timeout in the Scan passed to the server?
> >
> > Sorry for late reply,
> > St.Ack
> >
> >
> >
> > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera and
> > > Hortonworks are having version mismatch on the next official
> > > builds trafodion will support.
> > >
> > > So my question is still open?
> > >
> > > Best regards,
> > > Eric Owhadi
> > >
> > >
> > >
> > > -----Original Message-----
> > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > Of Stack
> > > Sent: Monday, August 24, 2015 11:07 PM
> > > To: HBase Dev List
> > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > >
> > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > <er...@esgyn.com>
> > > wrote:
> > >
> > > > Hello everyone,
> > > > We have been facing a situation on trafodion, where we are
> > > > hitting the hbase.client.scanner.timeout.period scenario:
> > > > basically, when doing queries that require spilling to disk
> > > > because of high complexity of what is involved, the underlying
> > > > hbase scanner serving one of the operation involved in the
> > > > complex query cannot call the next() withing the timeout
> > > > specify... too busy taking care of other business.
> > > > This is legit scenario, and I was wondering why in the code,
> > > > special care is done to make sure that client side, if a DNRIOE
> > > > of type unknownScannerException shows up, and the
> > > > hbase.client.scanner.timeout.period time elapsed, we make sure
> > > > to throw a scannerTimeoutException, instead of just let it go
> > > > and reset scanner.
> > > >
> > > > Scanners were redone in hbase 1.1. Can Trafodion come up onto
> > > > hbase
> > 1.1?
> > > See
> > > https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1
> > > for summary.
> > > St.Ack
> > >
> > >
> > >
> > > > I imagine that the lease time out implementation on region
> > > > server side is supposed to protect from resource leak of scanner
> > > > object server side. But I am not sure why we would make it so
> > > > that client side throw this timeout exception, when in fact what
> > > > just happened was that client was too busy to call next() on time.
> > > >
> > > > I am sure there is a reason, but cannot figure it out :-).
> > > >
> > > > BTW, I found this JIRA, talking about exact same thing:
> > > > https://issues.apache.org/jira/browse/HBASE61-21 but with no
> > resolution.
> > > >
> > >
> > >
> > > > Any help understanding the reason of the timeout thrwown client
> > > > side instead of an automatic reset would be much appreciated,
> > > > Best regards, Eric Owhadi
> > > >
> > >
> >
>

Re: Question on hbase.client.scanner.timeout.period

Posted by Stack <st...@duboce.net>.
On Thu, Sep 10, 2015 at 4:26 PM, Eric Owhadi <er...@esgyn.com> wrote:

> Not sure I understand. I was not planning to do any patch on an HBase
> distro? Only on apache hbase 2.0 and get automatic distro vendor adoption?
> Reason is that for trafodion, the trafodion patch would already cover Hbase
> 1.0. (and probably 1.1 too since Hortonworks will jump to 1.1 directly if I
> understand correctly).
> So I was thinking to get a HBase 2.0 fixed, and make sure that it makes it
> in the future official distros that will be on hbase 2.0. At that point
> Trafodion patch would be removed. I am not sure Trafodion will support
> HBase
> 1.2, probably it will jump to 2.0 directly? Given that trafodion uses some
> low level non "stable" API for transactional support, there can be some
> work
> involved to support minor versions or patch that could change API ...
>
> Also fix 2.0 would be simpler to do, as it will not be optional, should
> just
> be a few line of code deletion, right?
> A 1.x fix would be enabled/disabled via hbase-site.xml? Or where you
> thinking on a per call basis with a new flag in scan object?
> Make sense?
>

Sorry. Wasn't implying you'd do distro fixes. Didn't mean to confuse. Above
sounds good.
St.Ack




> Eric
>
> -----Original Message-----
> From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of Stack
> Sent: Thursday, September 10, 2015 5:49 PM
> To: HBase Dev List <de...@hbase.apache.org>
> Subject: Re: Question on hbase.client.scanner.timeout.period
>
> Suggest you do apache hbase first. You have a chance of getting it into 1.2
> if you do it soon. You might get a review too. Vendors generally want patch
> upstream first before backporting. If up in apache hbase, they might pull
> it
> back in a patch release if it doesn't break anything and it makes user or
> partner life smoother.
>
> Just a suggestion,
> St.Ack
>
>
> On Thu, Sep 10, 2015 at 3:14 PM, Eric Owhadi <er...@esgyn.com>
> wrote:
>
> > Not yes, I am waiting our QA validation on the patched trafodion and
> > see if we are not finding side effects to then work on an hbase 2.0
> > patch and create the umbrella jira as you suggested.
> > Except if you think I should rush on 2.0 to make sure it makes it to
> > any future official Hbase distro shipping with 2.0?
> >
> > Eric
> >
> >
> > -----Original Message-----
> > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> > Stack
> > Sent: Thursday, September 10, 2015 4:23 PM
> > To: HBase Dev List <de...@hbase.apache.org>
> > Subject: Re: Question on hbase.client.scanner.timeout.period
> >
> > You have a patch for apache hbase Eric? Is there an apache hbase issue
> > to add this in?
> > St.Ack
> >
> > On Thu, Sep 10, 2015 at 10:21 AM, Eric Owhadi <er...@esgyn.com>
> > wrote:
> >
> > > Thanks for pointing me to HBase-13333, it is indeed supposed to
> > > address the very same problem. With the drawback of requiring client
> > > side involvement, of asynchronous nature. I still have not
> > > discovered any reason why just doing it the way I proposed would
> > > lead to any negative side effect. Must admit I feel uncomfortable
> > > since the patch is just about removing code that usually is added with
> a
> > > purpose :-).
> > > We have not yet run full QA, but at least 100% of trafodion
> > > regression test pass.
> > > As for when the patch will make it to trafodion, given that I did it
> > > only for a CDH build of Trafodion with HBase 1.0 support, I still
> > > cannot check it in (trafodion is still on .98 and builds OK for
> > > Cloudera,Hortonworks,Mapr and Apache). Trafodion would first need to
> > > have full support for HBase 1.0 for all Hadoop distro we support,
> > > then I will need to redo the patch that is distro specific, and make
> > > sure the build process deals with this... It is my plan to do so...
> > > Hoping that I do not discover any issue with other distro (like
> > > private attributes or functions that I cannot circumvent... but that
> > > would just mean that the patch would not be available for a specific
> > > distro)
> > > Eric
> > >
> > >
> > > -----Original Message-----
> > > From: Jerry He [mailto:jerryjch@gmail.com]
> > > Sent: Saturday, September 5, 2015 1:47 PM
> > > To: dev <de...@hbase.apache.org>
> > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > >
> > > You can take a look at HBASE-13333: Renew Scanner Lease without
> > > advancing the RegionScanner, which may be helpful in this kind of
> > > case Your proposal sounds like a good alternative approach as well.
> > > We should add that JIRA to the blog link Stack mentioned.
> > >
> > > Jerry
> > >
> > > On Sat, Sep 5, 2015 at 9:07 AM, Stack <st...@duboce.net> wrote:
> > >
> > > > On Fri, Sep 4, 2015 at 5:06 PM, Eric Owhadi
> > > > <er...@esgyn.com>
> > > wrote:
> > > >
> > > > > OK so to answer the "is it easy to insert the patched scanner
> > > > > for trafodion", the answer is no.
> > > > >
> > > >
> > > > I suspected this.
> > > >
> > > >
> > > >
> > > > > Was easier on .98, but on 1.0 it was quite a challenge. All
> > > > > about dealing with private attributes instead of protected that
> > > > > are not visible to the PatchClentScanner class that extends
> > > > > ClientScanner.
> > > > > Currently running the regression tests to see if there is no
> > > > > side
> > > > effect...
> > > > > Was able to demonstrate with breakpoint on next() waiting more
> > > > > than
> > > > > 1 mn (the default lease timeout value) that with the patch
> > > > > things gracefully reset and all is good, no row skipped or
> > > > > duplicated, while without, I get the Scanner time out exception.
> > > > > Patch can be turn on or off with a new
> > > > key
> > > > > in hbase-site.xml...
> > > > > I will feel better when this will be deprecated :-).
> > > > >
> > > >
> > > > Smile.
> > > >
> > > > Excellent. You have a patch for us then Eric?  Sounds like the
> > > > interjection of your new Scanner would be for pre-2.0. For 2.0 we
> > > > should just turn on this behavior as the default.
> > > >
> > > > Thanks,
> > > > St.Ack
> > > >
> > > >
> > > >
> > > > > Eric Owhadi
> > > > >
> > > > > -----Original Message-----
> > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > > Of
> > > > Stack
> > > > > Sent: Friday, August 28, 2015 6:35 PM
> > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > >
> > > > > On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi
> > > > > <er...@esgyn.com>
> > > > > wrote:
> > > > >
> > > > > > That sounds good, but given trafodion needs to work on current
> > > > > > and future released version of HBase, unpatched, I will first
> > > > > > implement a ClientScannerTrafodion (to be deprecated),
> > > > > > inheriting from ClientScanner that will just overload the
> > > > > > loadCache(),and make sure that the code that is picking up the
> > > > > > right scanner based on scan object is bypassed to force
> > > > > > getting the ClientScannerTrafodion when appropriate.
> > > > > > Not very elegant, but need to take into consideration
> > > > > > trafodion deployment requirements.
> > > > > > Then, if we do not discover any side effect during our QA
> > > > > > related to this code I will port the fix on HBase to deprecate
> > > > > > the custom scanner (probably first on HBase 2.0, then will let
> > > > > > the community decide if this fix is worth it for back
> > > > > > porting...). It will be a first for me, but that's great, I'll
> > > > > > take your offer to help ;-)...
> > > > > >
> > > > >
> > > > > Sweet. Suggest opening an umbrellas issue in hbase to implement
> > > > > this feature. Reference HBASE-2161 (it is closed now). Link
> > > > > trafodion issue to it. A subtask could have implementation in
> > > > > hbase 2.0, another could be backport.
> > > > >
> > > > > Is is easy to insert your T*ClientScanner?
> > > > > St.Ack
> > > > >
> > > > >
> > > > >
> > > > > > Regards,
> > > > > > Eric
> > > > > >
> > > > > > -----Original Message-----
> > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > Behalf Of Stack
> > > > > > Sent: Thursday, August 27, 2015 3:55 PM
> > > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > > >
> > > > > > On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi
> > > > > > <er...@esgyn.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Oops, my bad, the related JIRA was :
> > > > > > > https://issues.apache.org/jira/browse/HBASE-2161
> > > > > > >
> > > > > > > I am suggesting that the special code client side in
> > > > > > > loadCache() of ClientScanner that is trapping the
> > > > > > > UnknownScannerException, then on purpose check if it is
> > > > > > > coming from a lease timeout (and not by a region move) to
> > > > > > > decide that it would throw a ScannerTimeoutException instead
> > > > > > > of letting the code go and just reset the scanner and start
> > > > > > > from last successful retrieve (the way it works for an
> > > > > > > unknowScannerException due to a region moving).
> > > > > > > By just removing the special handling that tries to
> > > > > > > differentiate from unkownScannerException due to lease
> > > > > > > timeout, we should have a resolution to JIRA 2161- And to
> > > > > > > our trafodion issue.
> > > > > > >
> > > > > > > We are still protecting against dead client that would cause
> > > > > > > resource leak at region server, since we keep the lease
> > > > > > > timeout mechanism.
> > > > > > >
> > > > > > > Not sure if I have overlooked something, as usually, code is
> > > > > > > here for a reason :-)...
> > > > > > >
> > > > > > >
> > > > > > Your proposal sounds good to me.
> > > > > >
> > > > > > Scanner works the way it does because it has always work this
> > > > > > way
> > > > > (smile).
> > > > > > A while back, one of the lads suggested we do like dynamodb
> > > > > > and have scanner have no state on the serverside, the scan
> > > > > > next would just supply all necessary context. It was argued
> > > > > > against because serverside setup is so costly. Your suggestion
> > > > > > is similar only we do it only if Scanner has timed out.
> > > > > >
> > > > > > Suggest we keep the current semantic in 1.x at least. We could
> > > > > > flip to your behavior in 2.x.  Meantime, you'd have to ask for
> > > > > > it when you set up your Scan object by setting a flag.
> > > > > >
> > > > > > Would that work? If you want to have a go at it, I could help
> > > > > > out on the issue.
> > > > > >
> > > > > > St.Ack
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > > Regards,
> > > > > > > Eric
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > -----Original Message-----
> > > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > > Behalf Of Stack
> > > > > > > Sent: Thursday, August 27, 2015 3:23 PM
> > > > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > > > >
> > > > > > > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi
> > > > > > > <er...@esgyn.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hello St.Ack,
> > > > > > > > Thanks for your pointer, but I had already investigated
> > > > > > > > JIRA
> > > > > > > > https://issues.apache.org/jira/browse/HBASE-13090
> > > > > > > > Unfortunately, this heartbeat will protect against rpc
> > > > > > > > timeout, not server side lease timeout that we are
> > > > > > > > experiencing right now.
> > > > > > > > I have not seen an active JIRA fixing our issue.
> > > > > > > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > > > > > > complaining about the exact same issue, but was never
> > > > > > > > resolved.
> > > > > > > >
> > > > > > > >
> > > > > > > Which issue?
> > > > > > > https://issues.apache.org/jira/browse/HBASE-6121
> > > > > > > seems unrelated.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > > The heartbeat JIRA in 13090 protect for situation where
> > > > > > > > server scanner takes so long to retrieve the highly
> > > > > > > > filtered information, that it exceeds the RPC timeout
> > > > > > > > (hbase.rpc.timeout).
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > > The timeout we are experiencing is the
> > > > > > > > hbase.client.scanner.timeout.period,
> > > > > > > > also deprecatedly known as hbase.regionserver.lease.period
> > > > > > > > The mechanism is different: here, region server scanners
> > > > > > > > wants to protect themselves against dead clients that
> > > > > > > > would not perform "close", and allow releasing server side
> > > > > > > > scanner resources. To do that, a lease mechanism is
> > > > > > > > implemented, and if between 2
> > > > > > > > next() call, more than hbase.regionserver.lease.period
> > > > > > > > occurs, the server side scanner will have been forced
> > > > > > > > closed by this lease timeout safety mechanism. On late
> > > > > > > > next() call, client will receive a DNRIOE of type
> > > > > > > > unknownScannerException, and the client will assess that
> > > > > > > > it is coming most likely from the lease timeout (and not
> > > > > > > > from a region move), therefore throwing an exception
> > > > > > > > instead of reset scanner (for the region move scenario).
> > > > > > > >
> > > > > > > > Hbase 1.1 does not address, as far as I have researched,
> > > > > > > > the hbase.client.scanner.timeout.period issue we are facing.
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > > > Can you not have the high-level query that is being fed by a
> > > > > > > scan do HBASE-13333? That is, tickle, the ongoing scan on
> > > > > > > occasion just to say that I'm still alive?
> > > > > > >
> > > > > > > Otherwise, what would you suggest? A scan that does not
> timeout?
> > > > > > > Or the client being able to set a timeout in the Scan passed
> > > > > > > to the
> > > > > server?
> > > > > > >
> > > > > > > Sorry for late reply,
> > > > > > > St.Ack
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera
> > > > > > > > and Hortonworks are having version mismatch on the next
> > > > > > > > official builds trafodion will support.
> > > > > > > >
> > > > > > > > So my question is still open?
> > > > > > > >
> > > > > > > > Best regards,
> > > > > > > > Eric Owhadi
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > -----Original Message-----
> > > > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > > > Behalf Of Stack
> > > > > > > > Sent: Monday, August 24, 2015 11:07 PM
> > > > > > > > To: HBase Dev List
> > > > > > > > Subject: Re: Question on
> > > > > > > > hbase.client.scanner.timeout.period
> > > > > > > >
> > > > > > > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > > > > > > <er...@esgyn.com>
> > > > > > > > wrote:
> > > > > > > >
> > > > > > > > > Hello everyone,
> > > > > > > > > We have been facing a situation on trafodion, where we
> > > > > > > > > are hitting the hbase.client.scanner.timeout.period
> > > > > > > > > scenario:
> > > > > > > > > basically, when doing queries that require spilling to
> > > > > > > > > disk because of high complexity of what is involved, the
> > > > > > > > > underlying hbase scanner serving one of the operation
> > > > > > > > > involved in the complex query cannot call the next()
> > > > > > > > > withing the timeout specify... too busy taking care of
> > > > > > > > > other business.
> > > > > > > > > This is legit scenario, and I was wondering why in the
> > > > > > > > > code, special care is done to make sure that client
> > > > > > > > > side, if a DNRIOE of type unknownScannerException shows
> > > > > > > > > up, and the hbase.client.scanner.timeout.period time
> > > > > > > > > elapsed, we make sure to throw a
> > > > > > > > > scannerTimeoutException, instead of just let it go and
> reset
> > > > > > > > > scanner.
> > > > > > > > >
> > > > > > > > > Scanners were redone in hbase 1.1. Can Trafodion come up
> > > > > > > > > onto hbase
> > > > > > > 1.1?
> > > > > > > > See
> > > > > > > > https://blogs.apache.org/hbase/entry/scan_improvements_in_
> > > > > > > > hb
> > > > > > > > as
> > > > > > > > e_1
> > > > > > > > for summary.
> > > > > > > > St.Ack
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > > I imagine that the lease time out implementation on
> > > > > > > > > region server side is supposed to protect from resource
> > > > > > > > > leak of scanner object server side. But I am not sure
> > > > > > > > > why we would make it so that client side throw this
> > > > > > > > > timeout exception, when in fact what just happened was
> > > > > > > > > that client was too busy to call next() on
> > > > time.
> > > > > > > > >
> > > > > > > > > I am sure there is a reason, but cannot figure it out :-).
> > > > > > > > >
> > > > > > > > > BTW, I found this JIRA, talking about exact same thing:
> > > > > > > > > https://issues.apache.org/jira/browse/HBASE61-21 but
> > > > > > > > > with no
> > > > > > > resolution.
> > > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > > Any help understanding the reason of the timeout thrwown
> > > > > > > > > client side instead of an automatic reset would be much
> > > > > > > > > appreciated, Best regards, Eric Owhadi
> > > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

RE: Question on hbase.client.scanner.timeout.period

Posted by Eric Owhadi <er...@esgyn.com>.
Not sure I understand. I was not planning to do any patch on an HBase
distro? Only on apache hbase 2.0 and get automatic distro vendor adoption?
Reason is that for trafodion, the trafodion patch would already cover Hbase
1.0. (and probably 1.1 too since Hortonworks will jump to 1.1 directly if I
understand correctly).
So I was thinking to get a HBase 2.0 fixed, and make sure that it makes it
in the future official distros that will be on hbase 2.0. At that point
Trafodion patch would be removed. I am not sure Trafodion will support HBase
1.2, probably it will jump to 2.0 directly? Given that trafodion uses some
low level non "stable" API for transactional support, there can be some work
involved to support minor versions or patch that could change API ...

Also fix 2.0 would be simpler to do, as it will not be optional, should just
be a few line of code deletion, right?
A 1.x fix would be enabled/disabled via hbase-site.xml? Or where you
thinking on a per call basis with a new flag in scan object?
Make sense?
Eric

-----Original Message-----
From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of Stack
Sent: Thursday, September 10, 2015 5:49 PM
To: HBase Dev List <de...@hbase.apache.org>
Subject: Re: Question on hbase.client.scanner.timeout.period

Suggest you do apache hbase first. You have a chance of getting it into 1.2
if you do it soon. You might get a review too. Vendors generally want patch
upstream first before backporting. If up in apache hbase, they might pull it
back in a patch release if it doesn't break anything and it makes user or
partner life smoother.

Just a suggestion,
St.Ack


On Thu, Sep 10, 2015 at 3:14 PM, Eric Owhadi <er...@esgyn.com> wrote:

> Not yes, I am waiting our QA validation on the patched trafodion and
> see if we are not finding side effects to then work on an hbase 2.0
> patch and create the umbrella jira as you suggested.
> Except if you think I should rush on 2.0 to make sure it makes it to
> any future official Hbase distro shipping with 2.0?
>
> Eric
>
>
> -----Original Message-----
> From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> Stack
> Sent: Thursday, September 10, 2015 4:23 PM
> To: HBase Dev List <de...@hbase.apache.org>
> Subject: Re: Question on hbase.client.scanner.timeout.period
>
> You have a patch for apache hbase Eric? Is there an apache hbase issue
> to add this in?
> St.Ack
>
> On Thu, Sep 10, 2015 at 10:21 AM, Eric Owhadi <er...@esgyn.com>
> wrote:
>
> > Thanks for pointing me to HBase-13333, it is indeed supposed to
> > address the very same problem. With the drawback of requiring client
> > side involvement, of asynchronous nature. I still have not
> > discovered any reason why just doing it the way I proposed would
> > lead to any negative side effect. Must admit I feel uncomfortable
> > since the patch is just about removing code that usually is added with a
> > purpose :-).
> > We have not yet run full QA, but at least 100% of trafodion
> > regression test pass.
> > As for when the patch will make it to trafodion, given that I did it
> > only for a CDH build of Trafodion with HBase 1.0 support, I still
> > cannot check it in (trafodion is still on .98 and builds OK for
> > Cloudera,Hortonworks,Mapr and Apache). Trafodion would first need to
> > have full support for HBase 1.0 for all Hadoop distro we support,
> > then I will need to redo the patch that is distro specific, and make
> > sure the build process deals with this... It is my plan to do so...
> > Hoping that I do not discover any issue with other distro (like
> > private attributes or functions that I cannot circumvent... but that
> > would just mean that the patch would not be available for a specific
> > distro)
> > Eric
> >
> >
> > -----Original Message-----
> > From: Jerry He [mailto:jerryjch@gmail.com]
> > Sent: Saturday, September 5, 2015 1:47 PM
> > To: dev <de...@hbase.apache.org>
> > Subject: Re: Question on hbase.client.scanner.timeout.period
> >
> > You can take a look at HBASE-13333: Renew Scanner Lease without
> > advancing the RegionScanner, which may be helpful in this kind of
> > case Your proposal sounds like a good alternative approach as well.
> > We should add that JIRA to the blog link Stack mentioned.
> >
> > Jerry
> >
> > On Sat, Sep 5, 2015 at 9:07 AM, Stack <st...@duboce.net> wrote:
> >
> > > On Fri, Sep 4, 2015 at 5:06 PM, Eric Owhadi
> > > <er...@esgyn.com>
> > wrote:
> > >
> > > > OK so to answer the "is it easy to insert the patched scanner
> > > > for trafodion", the answer is no.
> > > >
> > >
> > > I suspected this.
> > >
> > >
> > >
> > > > Was easier on .98, but on 1.0 it was quite a challenge. All
> > > > about dealing with private attributes instead of protected that
> > > > are not visible to the PatchClentScanner class that extends
> > > > ClientScanner.
> > > > Currently running the regression tests to see if there is no
> > > > side
> > > effect...
> > > > Was able to demonstrate with breakpoint on next() waiting more
> > > > than
> > > > 1 mn (the default lease timeout value) that with the patch
> > > > things gracefully reset and all is good, no row skipped or
> > > > duplicated, while without, I get the Scanner time out exception.
> > > > Patch can be turn on or off with a new
> > > key
> > > > in hbase-site.xml...
> > > > I will feel better when this will be deprecated :-).
> > > >
> > >
> > > Smile.
> > >
> > > Excellent. You have a patch for us then Eric?  Sounds like the
> > > interjection of your new Scanner would be for pre-2.0. For 2.0 we
> > > should just turn on this behavior as the default.
> > >
> > > Thanks,
> > > St.Ack
> > >
> > >
> > >
> > > > Eric Owhadi
> > > >
> > > > -----Original Message-----
> > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > Of
> > > Stack
> > > > Sent: Friday, August 28, 2015 6:35 PM
> > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > >
> > > > On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi
> > > > <er...@esgyn.com>
> > > > wrote:
> > > >
> > > > > That sounds good, but given trafodion needs to work on current
> > > > > and future released version of HBase, unpatched, I will first
> > > > > implement a ClientScannerTrafodion (to be deprecated),
> > > > > inheriting from ClientScanner that will just overload the
> > > > > loadCache(),and make sure that the code that is picking up the
> > > > > right scanner based on scan object is bypassed to force
> > > > > getting the ClientScannerTrafodion when appropriate.
> > > > > Not very elegant, but need to take into consideration
> > > > > trafodion deployment requirements.
> > > > > Then, if we do not discover any side effect during our QA
> > > > > related to this code I will port the fix on HBase to deprecate
> > > > > the custom scanner (probably first on HBase 2.0, then will let
> > > > > the community decide if this fix is worth it for back
> > > > > porting...). It will be a first for me, but that's great, I'll
> > > > > take your offer to help ;-)...
> > > > >
> > > >
> > > > Sweet. Suggest opening an umbrellas issue in hbase to implement
> > > > this feature. Reference HBASE-2161 (it is closed now). Link
> > > > trafodion issue to it. A subtask could have implementation in
> > > > hbase 2.0, another could be backport.
> > > >
> > > > Is is easy to insert your T*ClientScanner?
> > > > St.Ack
> > > >
> > > >
> > > >
> > > > > Regards,
> > > > > Eric
> > > > >
> > > > > -----Original Message-----
> > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > Behalf Of Stack
> > > > > Sent: Thursday, August 27, 2015 3:55 PM
> > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > >
> > > > > On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi
> > > > > <er...@esgyn.com>
> > > > > wrote:
> > > > >
> > > > > > Oops, my bad, the related JIRA was :
> > > > > > https://issues.apache.org/jira/browse/HBASE-2161
> > > > > >
> > > > > > I am suggesting that the special code client side in
> > > > > > loadCache() of ClientScanner that is trapping the
> > > > > > UnknownScannerException, then on purpose check if it is
> > > > > > coming from a lease timeout (and not by a region move) to
> > > > > > decide that it would throw a ScannerTimeoutException instead
> > > > > > of letting the code go and just reset the scanner and start
> > > > > > from last successful retrieve (the way it works for an
> > > > > > unknowScannerException due to a region moving).
> > > > > > By just removing the special handling that tries to
> > > > > > differentiate from unkownScannerException due to lease
> > > > > > timeout, we should have a resolution to JIRA 2161- And to
> > > > > > our trafodion issue.
> > > > > >
> > > > > > We are still protecting against dead client that would cause
> > > > > > resource leak at region server, since we keep the lease
> > > > > > timeout mechanism.
> > > > > >
> > > > > > Not sure if I have overlooked something, as usually, code is
> > > > > > here for a reason :-)...
> > > > > >
> > > > > >
> > > > > Your proposal sounds good to me.
> > > > >
> > > > > Scanner works the way it does because it has always work this
> > > > > way
> > > > (smile).
> > > > > A while back, one of the lads suggested we do like dynamodb
> > > > > and have scanner have no state on the serverside, the scan
> > > > > next would just supply all necessary context. It was argued
> > > > > against because serverside setup is so costly. Your suggestion
> > > > > is similar only we do it only if Scanner has timed out.
> > > > >
> > > > > Suggest we keep the current semantic in 1.x at least. We could
> > > > > flip to your behavior in 2.x.  Meantime, you'd have to ask for
> > > > > it when you set up your Scan object by setting a flag.
> > > > >
> > > > > Would that work? If you want to have a go at it, I could help
> > > > > out on the issue.
> > > > >
> > > > > St.Ack
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > > Regards,
> > > > > > Eric
> > > > > >
> > > > > >
> > > > > >
> > > > > > -----Original Message-----
> > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > Behalf Of Stack
> > > > > > Sent: Thursday, August 27, 2015 3:23 PM
> > > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > > >
> > > > > > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi
> > > > > > <er...@esgyn.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hello St.Ack,
> > > > > > > Thanks for your pointer, but I had already investigated
> > > > > > > JIRA
> > > > > > > https://issues.apache.org/jira/browse/HBASE-13090
> > > > > > > Unfortunately, this heartbeat will protect against rpc
> > > > > > > timeout, not server side lease timeout that we are
> > > > > > > experiencing right now.
> > > > > > > I have not seen an active JIRA fixing our issue.
> > > > > > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > > > > > complaining about the exact same issue, but was never
> > > > > > > resolved.
> > > > > > >
> > > > > > >
> > > > > > Which issue?
> > > > > > https://issues.apache.org/jira/browse/HBASE-6121
> > > > > > seems unrelated.
> > > > > >
> > > > > >
> > > > > >
> > > > > > > The heartbeat JIRA in 13090 protect for situation where
> > > > > > > server scanner takes so long to retrieve the highly
> > > > > > > filtered information, that it exceeds the RPC timeout
> > > > > > > (hbase.rpc.timeout).
> > > > > >
> > > > > >
> > > > > >
> > > > > > > The timeout we are experiencing is the
> > > > > > > hbase.client.scanner.timeout.period,
> > > > > > > also deprecatedly known as hbase.regionserver.lease.period
> > > > > > > The mechanism is different: here, region server scanners
> > > > > > > wants to protect themselves against dead clients that
> > > > > > > would not perform "close", and allow releasing server side
> > > > > > > scanner resources. To do that, a lease mechanism is
> > > > > > > implemented, and if between 2
> > > > > > > next() call, more than hbase.regionserver.lease.period
> > > > > > > occurs, the server side scanner will have been forced
> > > > > > > closed by this lease timeout safety mechanism. On late
> > > > > > > next() call, client will receive a DNRIOE of type
> > > > > > > unknownScannerException, and the client will assess that
> > > > > > > it is coming most likely from the lease timeout (and not
> > > > > > > from a region move), therefore throwing an exception
> > > > > > > instead of reset scanner (for the region move scenario).
> > > > > > >
> > > > > > > Hbase 1.1 does not address, as far as I have researched,
> > > > > > > the hbase.client.scanner.timeout.period issue we are facing.
> > > > > > >
> > > > > > >
> > > > > >
> > > > > > Can you not have the high-level query that is being fed by a
> > > > > > scan do HBASE-13333? That is, tickle, the ongoing scan on
> > > > > > occasion just to say that I'm still alive?
> > > > > >
> > > > > > Otherwise, what would you suggest? A scan that does not timeout?
> > > > > > Or the client being able to set a timeout in the Scan passed
> > > > > > to the
> > > > server?
> > > > > >
> > > > > > Sorry for late reply,
> > > > > > St.Ack
> > > > > >
> > > > > >
> > > > > >
> > > > > > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera
> > > > > > > and Hortonworks are having version mismatch on the next
> > > > > > > official builds trafodion will support.
> > > > > > >
> > > > > > > So my question is still open?
> > > > > > >
> > > > > > > Best regards,
> > > > > > > Eric Owhadi
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > -----Original Message-----
> > > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > > Behalf Of Stack
> > > > > > > Sent: Monday, August 24, 2015 11:07 PM
> > > > > > > To: HBase Dev List
> > > > > > > Subject: Re: Question on
> > > > > > > hbase.client.scanner.timeout.period
> > > > > > >
> > > > > > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > > > > > <er...@esgyn.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hello everyone,
> > > > > > > > We have been facing a situation on trafodion, where we
> > > > > > > > are hitting the hbase.client.scanner.timeout.period
> > > > > > > > scenario:
> > > > > > > > basically, when doing queries that require spilling to
> > > > > > > > disk because of high complexity of what is involved, the
> > > > > > > > underlying hbase scanner serving one of the operation
> > > > > > > > involved in the complex query cannot call the next()
> > > > > > > > withing the timeout specify... too busy taking care of
> > > > > > > > other business.
> > > > > > > > This is legit scenario, and I was wondering why in the
> > > > > > > > code, special care is done to make sure that client
> > > > > > > > side, if a DNRIOE of type unknownScannerException shows
> > > > > > > > up, and the hbase.client.scanner.timeout.period time
> > > > > > > > elapsed, we make sure to throw a
> > > > > > > > scannerTimeoutException, instead of just let it go and reset
> > > > > > > > scanner.
> > > > > > > >
> > > > > > > > Scanners were redone in hbase 1.1. Can Trafodion come up
> > > > > > > > onto hbase
> > > > > > 1.1?
> > > > > > > See
> > > > > > > https://blogs.apache.org/hbase/entry/scan_improvements_in_
> > > > > > > hb
> > > > > > > as
> > > > > > > e_1
> > > > > > > for summary.
> > > > > > > St.Ack
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > > I imagine that the lease time out implementation on
> > > > > > > > region server side is supposed to protect from resource
> > > > > > > > leak of scanner object server side. But I am not sure
> > > > > > > > why we would make it so that client side throw this
> > > > > > > > timeout exception, when in fact what just happened was
> > > > > > > > that client was too busy to call next() on
> > > time.
> > > > > > > >
> > > > > > > > I am sure there is a reason, but cannot figure it out :-).
> > > > > > > >
> > > > > > > > BTW, I found this JIRA, talking about exact same thing:
> > > > > > > > https://issues.apache.org/jira/browse/HBASE61-21 but
> > > > > > > > with no
> > > > > > resolution.
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > > Any help understanding the reason of the timeout thrwown
> > > > > > > > client side instead of an automatic reset would be much
> > > > > > > > appreciated, Best regards, Eric Owhadi
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Question on hbase.client.scanner.timeout.period

Posted by Stack <st...@duboce.net>.
Suggest you do apache hbase first. You have a chance of getting it into 1.2
if you do it soon. You might get a review too. Vendors generally want patch
upstream first before backporting. If up in apache hbase, they might pull
it back in a patch release if it doesn't break anything and it makes user
or partner life smoother.

Just a suggestion,
St.Ack


On Thu, Sep 10, 2015 at 3:14 PM, Eric Owhadi <er...@esgyn.com> wrote:

> Not yes, I am waiting our QA validation on the patched trafodion and see if
> we are not finding side effects to then work on an hbase 2.0 patch and
> create the umbrella jira as you suggested.
> Except if you think I should rush on 2.0 to make sure it makes it to any
> future official Hbase distro shipping with 2.0?
>
> Eric
>
>
> -----Original Message-----
> From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of Stack
> Sent: Thursday, September 10, 2015 4:23 PM
> To: HBase Dev List <de...@hbase.apache.org>
> Subject: Re: Question on hbase.client.scanner.timeout.period
>
> You have a patch for apache hbase Eric? Is there an apache hbase issue to
> add this in?
> St.Ack
>
> On Thu, Sep 10, 2015 at 10:21 AM, Eric Owhadi <er...@esgyn.com>
> wrote:
>
> > Thanks for pointing me to HBase-13333, it is indeed supposed to
> > address the very same problem. With the drawback of requiring client
> > side involvement, of asynchronous nature. I still have not discovered
> > any reason why just doing it the way I proposed would lead to any
> > negative side effect. Must admit I feel uncomfortable since the patch
> > is just about removing code that usually is added with a purpose :-).
> > We have not yet run full QA, but at least 100% of trafodion regression
> > test pass.
> > As for when the patch will make it to trafodion, given that I did it
> > only for a CDH build of Trafodion with HBase 1.0 support, I still
> > cannot check it in (trafodion is still on .98 and builds OK for
> > Cloudera,Hortonworks,Mapr and Apache). Trafodion would first need to
> > have full support for HBase 1.0 for all Hadoop distro we support, then
> > I will need to redo the patch that is distro specific, and make sure
> > the build process deals with this... It is my plan to do so... Hoping
> > that I do not discover any issue with other distro (like private
> > attributes or functions that I cannot circumvent... but that would
> > just mean that the patch would not be available for a specific
> > distro)
> > Eric
> >
> >
> > -----Original Message-----
> > From: Jerry He [mailto:jerryjch@gmail.com]
> > Sent: Saturday, September 5, 2015 1:47 PM
> > To: dev <de...@hbase.apache.org>
> > Subject: Re: Question on hbase.client.scanner.timeout.period
> >
> > You can take a look at HBASE-13333: Renew Scanner Lease without
> > advancing the RegionScanner, which may be helpful in this kind of case
> > Your proposal sounds like a good alternative approach as well.
> > We should add that JIRA to the blog link Stack mentioned.
> >
> > Jerry
> >
> > On Sat, Sep 5, 2015 at 9:07 AM, Stack <st...@duboce.net> wrote:
> >
> > > On Fri, Sep 4, 2015 at 5:06 PM, Eric Owhadi <er...@esgyn.com>
> > wrote:
> > >
> > > > OK so to answer the "is it easy to insert the patched scanner for
> > > > trafodion", the answer is no.
> > > >
> > >
> > > I suspected this.
> > >
> > >
> > >
> > > > Was easier on .98, but on 1.0 it was quite a challenge. All about
> > > > dealing with private attributes instead of protected that are not
> > > > visible to the PatchClentScanner class that extends ClientScanner.
> > > > Currently running the regression tests to see if there is no side
> > > effect...
> > > > Was able to demonstrate with breakpoint on next() waiting more
> > > > than
> > > > 1 mn (the default lease timeout value) that with the patch things
> > > > gracefully reset and all is good, no row skipped or duplicated,
> > > > while without, I get the Scanner time out exception. Patch can be
> > > > turn on or off with a new
> > > key
> > > > in hbase-site.xml...
> > > > I will feel better when this will be deprecated :-).
> > > >
> > >
> > > Smile.
> > >
> > > Excellent. You have a patch for us then Eric?  Sounds like the
> > > interjection of your new Scanner would be for pre-2.0. For 2.0 we
> > > should just turn on this behavior as the default.
> > >
> > > Thanks,
> > > St.Ack
> > >
> > >
> > >
> > > > Eric Owhadi
> > > >
> > > > -----Original Message-----
> > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > Of
> > > Stack
> > > > Sent: Friday, August 28, 2015 6:35 PM
> > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > >
> > > > On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi
> > > > <er...@esgyn.com>
> > > > wrote:
> > > >
> > > > > That sounds good, but given trafodion needs to work on current
> > > > > and future released version of HBase, unpatched, I will first
> > > > > implement a ClientScannerTrafodion (to be deprecated),
> > > > > inheriting from ClientScanner that will just overload the
> > > > > loadCache(),and make sure that the code that is picking up the
> > > > > right scanner based on scan object is bypassed to force getting
> > > > > the ClientScannerTrafodion when appropriate.
> > > > > Not very elegant, but need to take into consideration trafodion
> > > > > deployment requirements.
> > > > > Then, if we do not discover any side effect during our QA
> > > > > related to this code I will port the fix on HBase to deprecate
> > > > > the custom scanner (probably first on HBase 2.0, then will let
> > > > > the community decide if this fix is worth it for back
> > > > > porting...). It will be a first for me, but that's great, I'll take
> > > > > your offer to help ;-)...
> > > > >
> > > >
> > > > Sweet. Suggest opening an umbrellas issue in hbase to implement
> > > > this feature. Reference HBASE-2161 (it is closed now). Link
> > > > trafodion issue to it. A subtask could have implementation in
> > > > hbase 2.0, another could be backport.
> > > >
> > > > Is is easy to insert your T*ClientScanner?
> > > > St.Ack
> > > >
> > > >
> > > >
> > > > > Regards,
> > > > > Eric
> > > > >
> > > > > -----Original Message-----
> > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > > Of Stack
> > > > > Sent: Thursday, August 27, 2015 3:55 PM
> > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > >
> > > > > On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi
> > > > > <er...@esgyn.com>
> > > > > wrote:
> > > > >
> > > > > > Oops, my bad, the related JIRA was :
> > > > > > https://issues.apache.org/jira/browse/HBASE-2161
> > > > > >
> > > > > > I am suggesting that the special code client side in
> > > > > > loadCache() of ClientScanner that is trapping the
> > > > > > UnknownScannerException, then on purpose check if it is coming
> > > > > > from a lease timeout (and not by a region move) to decide that
> > > > > > it would throw a ScannerTimeoutException instead of letting
> > > > > > the code go and just reset the scanner and start from last
> > > > > > successful retrieve (the way it works for an
> > > > > > unknowScannerException due to a region moving).
> > > > > > By just removing the special handling that tries to
> > > > > > differentiate from unkownScannerException due to lease
> > > > > > timeout, we should have a resolution to JIRA 2161- And to our
> > > > > > trafodion issue.
> > > > > >
> > > > > > We are still protecting against dead client that would cause
> > > > > > resource leak at region server, since we keep the lease
> > > > > > timeout mechanism.
> > > > > >
> > > > > > Not sure if I have overlooked something, as usually, code is
> > > > > > here for a reason :-)...
> > > > > >
> > > > > >
> > > > > Your proposal sounds good to me.
> > > > >
> > > > > Scanner works the way it does because it has always work this
> > > > > way
> > > > (smile).
> > > > > A while back, one of the lads suggested we do like dynamodb and
> > > > > have scanner have no state on the serverside, the scan next
> > > > > would just supply all necessary context. It was argued against
> > > > > because serverside setup is so costly. Your suggestion is
> > > > > similar only we do it only if Scanner has timed out.
> > > > >
> > > > > Suggest we keep the current semantic in 1.x at least. We could
> > > > > flip to your behavior in 2.x.  Meantime, you'd have to ask for
> > > > > it when you set up your Scan object by setting a flag.
> > > > >
> > > > > Would that work? If you want to have a go at it, I could help
> > > > > out on the issue.
> > > > >
> > > > > St.Ack
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > > Regards,
> > > > > > Eric
> > > > > >
> > > > > >
> > > > > >
> > > > > > -----Original Message-----
> > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > Behalf Of Stack
> > > > > > Sent: Thursday, August 27, 2015 3:23 PM
> > > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > > >
> > > > > > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi
> > > > > > <er...@esgyn.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hello St.Ack,
> > > > > > > Thanks for your pointer, but I had already investigated JIRA
> > > > > > > https://issues.apache.org/jira/browse/HBASE-13090
> > > > > > > Unfortunately, this heartbeat will protect against rpc
> > > > > > > timeout, not server side lease timeout that we are
> > > > > > > experiencing right now.
> > > > > > > I have not seen an active JIRA fixing our issue.
> > > > > > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > > > > > complaining about the exact same issue, but was never resolved.
> > > > > > >
> > > > > > >
> > > > > > Which issue? https://issues.apache.org/jira/browse/HBASE-6121
> > > > > > seems unrelated.
> > > > > >
> > > > > >
> > > > > >
> > > > > > > The heartbeat JIRA in 13090 protect for situation where
> > > > > > > server scanner takes so long to retrieve the highly filtered
> > > > > > > information, that it exceeds the RPC timeout
> > > > > > > (hbase.rpc.timeout).
> > > > > >
> > > > > >
> > > > > >
> > > > > > > The timeout we are experiencing is the
> > > > > > > hbase.client.scanner.timeout.period,
> > > > > > > also deprecatedly known as hbase.regionserver.lease.period
> > > > > > > The mechanism is different: here, region server scanners
> > > > > > > wants to protect themselves against dead clients that would
> > > > > > > not perform "close", and allow releasing server side scanner
> > > > > > > resources. To do that, a lease mechanism is implemented, and
> > > > > > > if between 2
> > > > > > > next() call, more than hbase.regionserver.lease.period
> > > > > > > occurs, the server side scanner will have been forced closed
> > > > > > > by this lease timeout safety mechanism. On late next() call,
> > > > > > > client will receive a DNRIOE of type
> > > > > > > unknownScannerException, and the client will assess that it
> > > > > > > is coming most likely from the lease timeout (and not from a
> > > > > > > region move), therefore throwing an exception instead of
> > > > > > > reset scanner (for the region move scenario).
> > > > > > >
> > > > > > > Hbase 1.1 does not address, as far as I have researched, the
> > > > > > > hbase.client.scanner.timeout.period issue we are facing.
> > > > > > >
> > > > > > >
> > > > > >
> > > > > > Can you not have the high-level query that is being fed by a
> > > > > > scan do HBASE-13333? That is, tickle, the ongoing scan on
> > > > > > occasion just to say that I'm still alive?
> > > > > >
> > > > > > Otherwise, what would you suggest? A scan that does not timeout?
> > > > > > Or the client being able to set a timeout in the Scan passed
> > > > > > to the
> > > > server?
> > > > > >
> > > > > > Sorry for late reply,
> > > > > > St.Ack
> > > > > >
> > > > > >
> > > > > >
> > > > > > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera and
> > > > > > > Hortonworks are having version mismatch on the next official
> > > > > > > builds trafodion will support.
> > > > > > >
> > > > > > > So my question is still open?
> > > > > > >
> > > > > > > Best regards,
> > > > > > > Eric Owhadi
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > -----Original Message-----
> > > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > > Behalf Of Stack
> > > > > > > Sent: Monday, August 24, 2015 11:07 PM
> > > > > > > To: HBase Dev List
> > > > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > > > >
> > > > > > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > > > > > <er...@esgyn.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > > > Hello everyone,
> > > > > > > > We have been facing a situation on trafodion, where we are
> > > > > > > > hitting the hbase.client.scanner.timeout.period scenario:
> > > > > > > > basically, when doing queries that require spilling to
> > > > > > > > disk because of high complexity of what is involved, the
> > > > > > > > underlying hbase scanner serving one of the operation
> > > > > > > > involved in the complex query cannot call the next()
> > > > > > > > withing the timeout specify... too busy taking care of other
> > > > > > > > business.
> > > > > > > > This is legit scenario, and I was wondering why in the
> > > > > > > > code, special care is done to make sure that client side,
> > > > > > > > if a DNRIOE of type unknownScannerException shows up, and
> > > > > > > > the hbase.client.scanner.timeout.period time elapsed, we
> > > > > > > > make sure to throw a scannerTimeoutException, instead of
> > > > > > > > just let it go and reset scanner.
> > > > > > > >
> > > > > > > > Scanners were redone in hbase 1.1. Can Trafodion come up
> > > > > > > > onto hbase
> > > > > > 1.1?
> > > > > > > See
> > > > > > > https://blogs.apache.org/hbase/entry/scan_improvements_in_hb
> > > > > > > as
> > > > > > > e_1
> > > > > > > for summary.
> > > > > > > St.Ack
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > > I imagine that the lease time out implementation on region
> > > > > > > > server side is supposed to protect from resource leak of
> > > > > > > > scanner object server side. But I am not sure why we would
> > > > > > > > make it so that client side throw this timeout exception,
> > > > > > > > when in fact what just happened was that client was too
> > > > > > > > busy to call next() on
> > > time.
> > > > > > > >
> > > > > > > > I am sure there is a reason, but cannot figure it out :-).
> > > > > > > >
> > > > > > > > BTW, I found this JIRA, talking about exact same thing:
> > > > > > > > https://issues.apache.org/jira/browse/HBASE61-21 but with
> > > > > > > > no
> > > > > > resolution.
> > > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > > Any help understanding the reason of the timeout thrwown
> > > > > > > > client side instead of an automatic reset would be much
> > > > > > > > appreciated, Best regards, Eric Owhadi
> > > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

RE: Question on hbase.client.scanner.timeout.period

Posted by Eric Owhadi <er...@esgyn.com>.
Not yes, I am waiting our QA validation on the patched trafodion and see if
we are not finding side effects to then work on an hbase 2.0 patch and
create the umbrella jira as you suggested.
Except if you think I should rush on 2.0 to make sure it makes it to any
future official Hbase distro shipping with 2.0?

Eric


-----Original Message-----
From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of Stack
Sent: Thursday, September 10, 2015 4:23 PM
To: HBase Dev List <de...@hbase.apache.org>
Subject: Re: Question on hbase.client.scanner.timeout.period

You have a patch for apache hbase Eric? Is there an apache hbase issue to
add this in?
St.Ack

On Thu, Sep 10, 2015 at 10:21 AM, Eric Owhadi <er...@esgyn.com> wrote:

> Thanks for pointing me to HBase-13333, it is indeed supposed to
> address the very same problem. With the drawback of requiring client
> side involvement, of asynchronous nature. I still have not discovered
> any reason why just doing it the way I proposed would lead to any
> negative side effect. Must admit I feel uncomfortable since the patch
> is just about removing code that usually is added with a purpose :-).
> We have not yet run full QA, but at least 100% of trafodion regression
> test pass.
> As for when the patch will make it to trafodion, given that I did it
> only for a CDH build of Trafodion with HBase 1.0 support, I still
> cannot check it in (trafodion is still on .98 and builds OK for
> Cloudera,Hortonworks,Mapr and Apache). Trafodion would first need to
> have full support for HBase 1.0 for all Hadoop distro we support, then
> I will need to redo the patch that is distro specific, and make sure
> the build process deals with this... It is my plan to do so... Hoping
> that I do not discover any issue with other distro (like private
> attributes or functions that I cannot circumvent... but that would
> just mean that the patch would not be available for a specific
> distro)
> Eric
>
>
> -----Original Message-----
> From: Jerry He [mailto:jerryjch@gmail.com]
> Sent: Saturday, September 5, 2015 1:47 PM
> To: dev <de...@hbase.apache.org>
> Subject: Re: Question on hbase.client.scanner.timeout.period
>
> You can take a look at HBASE-13333: Renew Scanner Lease without
> advancing the RegionScanner, which may be helpful in this kind of case
> Your proposal sounds like a good alternative approach as well.
> We should add that JIRA to the blog link Stack mentioned.
>
> Jerry
>
> On Sat, Sep 5, 2015 at 9:07 AM, Stack <st...@duboce.net> wrote:
>
> > On Fri, Sep 4, 2015 at 5:06 PM, Eric Owhadi <er...@esgyn.com>
> wrote:
> >
> > > OK so to answer the "is it easy to insert the patched scanner for
> > > trafodion", the answer is no.
> > >
> >
> > I suspected this.
> >
> >
> >
> > > Was easier on .98, but on 1.0 it was quite a challenge. All about
> > > dealing with private attributes instead of protected that are not
> > > visible to the PatchClentScanner class that extends ClientScanner.
> > > Currently running the regression tests to see if there is no side
> > effect...
> > > Was able to demonstrate with breakpoint on next() waiting more
> > > than
> > > 1 mn (the default lease timeout value) that with the patch things
> > > gracefully reset and all is good, no row skipped or duplicated,
> > > while without, I get the Scanner time out exception. Patch can be
> > > turn on or off with a new
> > key
> > > in hbase-site.xml...
> > > I will feel better when this will be deprecated :-).
> > >
> >
> > Smile.
> >
> > Excellent. You have a patch for us then Eric?  Sounds like the
> > interjection of your new Scanner would be for pre-2.0. For 2.0 we
> > should just turn on this behavior as the default.
> >
> > Thanks,
> > St.Ack
> >
> >
> >
> > > Eric Owhadi
> > >
> > > -----Original Message-----
> > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > Of
> > Stack
> > > Sent: Friday, August 28, 2015 6:35 PM
> > > To: HBase Dev List <de...@hbase.apache.org>
> > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > >
> > > On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi
> > > <er...@esgyn.com>
> > > wrote:
> > >
> > > > That sounds good, but given trafodion needs to work on current
> > > > and future released version of HBase, unpatched, I will first
> > > > implement a ClientScannerTrafodion (to be deprecated),
> > > > inheriting from ClientScanner that will just overload the
> > > > loadCache(),and make sure that the code that is picking up the
> > > > right scanner based on scan object is bypassed to force getting
> > > > the ClientScannerTrafodion when appropriate.
> > > > Not very elegant, but need to take into consideration trafodion
> > > > deployment requirements.
> > > > Then, if we do not discover any side effect during our QA
> > > > related to this code I will port the fix on HBase to deprecate
> > > > the custom scanner (probably first on HBase 2.0, then will let
> > > > the community decide if this fix is worth it for back
> > > > porting...). It will be a first for me, but that's great, I'll take
> > > > your offer to help ;-)...
> > > >
> > >
> > > Sweet. Suggest opening an umbrellas issue in hbase to implement
> > > this feature. Reference HBASE-2161 (it is closed now). Link
> > > trafodion issue to it. A subtask could have implementation in
> > > hbase 2.0, another could be backport.
> > >
> > > Is is easy to insert your T*ClientScanner?
> > > St.Ack
> > >
> > >
> > >
> > > > Regards,
> > > > Eric
> > > >
> > > > -----Original Message-----
> > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > Of Stack
> > > > Sent: Thursday, August 27, 2015 3:55 PM
> > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > >
> > > > On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi
> > > > <er...@esgyn.com>
> > > > wrote:
> > > >
> > > > > Oops, my bad, the related JIRA was :
> > > > > https://issues.apache.org/jira/browse/HBASE-2161
> > > > >
> > > > > I am suggesting that the special code client side in
> > > > > loadCache() of ClientScanner that is trapping the
> > > > > UnknownScannerException, then on purpose check if it is coming
> > > > > from a lease timeout (and not by a region move) to decide that
> > > > > it would throw a ScannerTimeoutException instead of letting
> > > > > the code go and just reset the scanner and start from last
> > > > > successful retrieve (the way it works for an
> > > > > unknowScannerException due to a region moving).
> > > > > By just removing the special handling that tries to
> > > > > differentiate from unkownScannerException due to lease
> > > > > timeout, we should have a resolution to JIRA 2161- And to our
> > > > > trafodion issue.
> > > > >
> > > > > We are still protecting against dead client that would cause
> > > > > resource leak at region server, since we keep the lease
> > > > > timeout mechanism.
> > > > >
> > > > > Not sure if I have overlooked something, as usually, code is
> > > > > here for a reason :-)...
> > > > >
> > > > >
> > > > Your proposal sounds good to me.
> > > >
> > > > Scanner works the way it does because it has always work this
> > > > way
> > > (smile).
> > > > A while back, one of the lads suggested we do like dynamodb and
> > > > have scanner have no state on the serverside, the scan next
> > > > would just supply all necessary context. It was argued against
> > > > because serverside setup is so costly. Your suggestion is
> > > > similar only we do it only if Scanner has timed out.
> > > >
> > > > Suggest we keep the current semantic in 1.x at least. We could
> > > > flip to your behavior in 2.x.  Meantime, you'd have to ask for
> > > > it when you set up your Scan object by setting a flag.
> > > >
> > > > Would that work? If you want to have a go at it, I could help
> > > > out on the issue.
> > > >
> > > > St.Ack
> > > >
> > > >
> > > >
> > > >
> > > > > Regards,
> > > > > Eric
> > > > >
> > > > >
> > > > >
> > > > > -----Original Message-----
> > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > Behalf Of Stack
> > > > > Sent: Thursday, August 27, 2015 3:23 PM
> > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > >
> > > > > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi
> > > > > <er...@esgyn.com>
> > > > > wrote:
> > > > >
> > > > > > Hello St.Ack,
> > > > > > Thanks for your pointer, but I had already investigated JIRA
> > > > > > https://issues.apache.org/jira/browse/HBASE-13090
> > > > > > Unfortunately, this heartbeat will protect against rpc
> > > > > > timeout, not server side lease timeout that we are
> > > > > > experiencing right now.
> > > > > > I have not seen an active JIRA fixing our issue.
> > > > > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > > > > complaining about the exact same issue, but was never resolved.
> > > > > >
> > > > > >
> > > > > Which issue? https://issues.apache.org/jira/browse/HBASE-6121
> > > > > seems unrelated.
> > > > >
> > > > >
> > > > >
> > > > > > The heartbeat JIRA in 13090 protect for situation where
> > > > > > server scanner takes so long to retrieve the highly filtered
> > > > > > information, that it exceeds the RPC timeout
> > > > > > (hbase.rpc.timeout).
> > > > >
> > > > >
> > > > >
> > > > > > The timeout we are experiencing is the
> > > > > > hbase.client.scanner.timeout.period,
> > > > > > also deprecatedly known as hbase.regionserver.lease.period
> > > > > > The mechanism is different: here, region server scanners
> > > > > > wants to protect themselves against dead clients that would
> > > > > > not perform "close", and allow releasing server side scanner
> > > > > > resources. To do that, a lease mechanism is implemented, and
> > > > > > if between 2
> > > > > > next() call, more than hbase.regionserver.lease.period
> > > > > > occurs, the server side scanner will have been forced closed
> > > > > > by this lease timeout safety mechanism. On late next() call,
> > > > > > client will receive a DNRIOE of type
> > > > > > unknownScannerException, and the client will assess that it
> > > > > > is coming most likely from the lease timeout (and not from a
> > > > > > region move), therefore throwing an exception instead of
> > > > > > reset scanner (for the region move scenario).
> > > > > >
> > > > > > Hbase 1.1 does not address, as far as I have researched, the
> > > > > > hbase.client.scanner.timeout.period issue we are facing.
> > > > > >
> > > > > >
> > > > >
> > > > > Can you not have the high-level query that is being fed by a
> > > > > scan do HBASE-13333? That is, tickle, the ongoing scan on
> > > > > occasion just to say that I'm still alive?
> > > > >
> > > > > Otherwise, what would you suggest? A scan that does not timeout?
> > > > > Or the client being able to set a timeout in the Scan passed
> > > > > to the
> > > server?
> > > > >
> > > > > Sorry for late reply,
> > > > > St.Ack
> > > > >
> > > > >
> > > > >
> > > > > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera and
> > > > > > Hortonworks are having version mismatch on the next official
> > > > > > builds trafodion will support.
> > > > > >
> > > > > > So my question is still open?
> > > > > >
> > > > > > Best regards,
> > > > > > Eric Owhadi
> > > > > >
> > > > > >
> > > > > >
> > > > > > -----Original Message-----
> > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > Behalf Of Stack
> > > > > > Sent: Monday, August 24, 2015 11:07 PM
> > > > > > To: HBase Dev List
> > > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > > >
> > > > > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > > > > <er...@esgyn.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hello everyone,
> > > > > > > We have been facing a situation on trafodion, where we are
> > > > > > > hitting the hbase.client.scanner.timeout.period scenario:
> > > > > > > basically, when doing queries that require spilling to
> > > > > > > disk because of high complexity of what is involved, the
> > > > > > > underlying hbase scanner serving one of the operation
> > > > > > > involved in the complex query cannot call the next()
> > > > > > > withing the timeout specify... too busy taking care of other
> > > > > > > business.
> > > > > > > This is legit scenario, and I was wondering why in the
> > > > > > > code, special care is done to make sure that client side,
> > > > > > > if a DNRIOE of type unknownScannerException shows up, and
> > > > > > > the hbase.client.scanner.timeout.period time elapsed, we
> > > > > > > make sure to throw a scannerTimeoutException, instead of
> > > > > > > just let it go and reset scanner.
> > > > > > >
> > > > > > > Scanners were redone in hbase 1.1. Can Trafodion come up
> > > > > > > onto hbase
> > > > > 1.1?
> > > > > > See
> > > > > > https://blogs.apache.org/hbase/entry/scan_improvements_in_hb
> > > > > > as
> > > > > > e_1
> > > > > > for summary.
> > > > > > St.Ack
> > > > > >
> > > > > >
> > > > > >
> > > > > > > I imagine that the lease time out implementation on region
> > > > > > > server side is supposed to protect from resource leak of
> > > > > > > scanner object server side. But I am not sure why we would
> > > > > > > make it so that client side throw this timeout exception,
> > > > > > > when in fact what just happened was that client was too
> > > > > > > busy to call next() on
> > time.
> > > > > > >
> > > > > > > I am sure there is a reason, but cannot figure it out :-).
> > > > > > >
> > > > > > > BTW, I found this JIRA, talking about exact same thing:
> > > > > > > https://issues.apache.org/jira/browse/HBASE61-21 but with
> > > > > > > no
> > > > > resolution.
> > > > > > >
> > > > > >
> > > > > >
> > > > > > > Any help understanding the reason of the timeout thrwown
> > > > > > > client side instead of an automatic reset would be much
> > > > > > > appreciated, Best regards, Eric Owhadi
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Question on hbase.client.scanner.timeout.period

Posted by Stack <st...@duboce.net>.
You have a patch for apache hbase Eric? Is there an apache hbase issue to
add this in?
St.Ack

On Thu, Sep 10, 2015 at 10:21 AM, Eric Owhadi <er...@esgyn.com> wrote:

> Thanks for pointing me to HBase-13333, it is indeed supposed to address the
> very same problem. With the drawback of requiring client side involvement,
> of asynchronous nature. I still have not discovered any reason why just
> doing it the way I proposed would lead to any negative side effect. Must
> admit I feel uncomfortable since the patch is just about removing code that
> usually is added with a purpose :-).
> We have not yet run full QA, but at least 100% of trafodion regression test
> pass.
> As for when the patch will make it to trafodion, given that I did it only
> for a CDH build of Trafodion with HBase 1.0 support, I still cannot check
> it
> in (trafodion is still on .98 and builds OK for Cloudera,Hortonworks,Mapr
> and Apache). Trafodion would first need to have full support for HBase 1.0
> for all Hadoop distro we support, then I will need to redo the patch that
> is
> distro specific, and make sure the build process deals with this... It is
> my
> plan to do so... Hoping that I do not discover any issue with other distro
> (like private attributes or functions that I cannot circumvent... but that
> would just mean that the patch would not be available for a specific
> distro)
> Eric
>
>
> -----Original Message-----
> From: Jerry He [mailto:jerryjch@gmail.com]
> Sent: Saturday, September 5, 2015 1:47 PM
> To: dev <de...@hbase.apache.org>
> Subject: Re: Question on hbase.client.scanner.timeout.period
>
> You can take a look at HBASE-13333: Renew Scanner Lease without advancing
> the RegionScanner, which may be helpful in this kind of case  Your proposal
> sounds like a good alternative approach as well.
> We should add that JIRA to the blog link Stack mentioned.
>
> Jerry
>
> On Sat, Sep 5, 2015 at 9:07 AM, Stack <st...@duboce.net> wrote:
>
> > On Fri, Sep 4, 2015 at 5:06 PM, Eric Owhadi <er...@esgyn.com>
> wrote:
> >
> > > OK so to answer the "is it easy to insert the patched scanner for
> > > trafodion", the answer is no.
> > >
> >
> > I suspected this.
> >
> >
> >
> > > Was easier on .98, but on 1.0 it was quite a challenge. All about
> > > dealing with private attributes instead of protected that are not
> > > visible to the PatchClentScanner class that extends ClientScanner.
> > > Currently running the regression tests to see if there is no side
> > effect...
> > > Was able to demonstrate with breakpoint on next() waiting more than
> > > 1 mn (the default lease timeout value) that with the patch things
> > > gracefully reset and all is good, no row skipped or duplicated,
> > > while without, I get the Scanner time out exception. Patch can be
> > > turn on or off with a new
> > key
> > > in hbase-site.xml...
> > > I will feel better when this will be deprecated :-).
> > >
> >
> > Smile.
> >
> > Excellent. You have a patch for us then Eric?  Sounds like the
> > interjection of your new Scanner would be for pre-2.0. For 2.0 we
> > should just turn on this behavior as the default.
> >
> > Thanks,
> > St.Ack
> >
> >
> >
> > > Eric Owhadi
> > >
> > > -----Original Message-----
> > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> > Stack
> > > Sent: Friday, August 28, 2015 6:35 PM
> > > To: HBase Dev List <de...@hbase.apache.org>
> > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > >
> > > On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi
> > > <er...@esgyn.com>
> > > wrote:
> > >
> > > > That sounds good, but given trafodion needs to work on current and
> > > > future released version of HBase, unpatched, I will first
> > > > implement a ClientScannerTrafodion (to be deprecated), inheriting
> > > > from ClientScanner that will just overload the loadCache(),and
> > > > make sure that the code that is picking up the right scanner based
> > > > on scan object is bypassed to force getting the
> > > > ClientScannerTrafodion when appropriate.
> > > > Not very elegant, but need to take into consideration trafodion
> > > > deployment requirements.
> > > > Then, if we do not discover any side effect during our QA related
> > > > to this code I will port the fix on HBase to deprecate the custom
> > > > scanner (probably first on HBase 2.0, then will let the community
> > > > decide if this fix is worth it for back porting...). It will be a
> > > > first for me, but that's great, I'll take your offer to help ;-)...
> > > >
> > >
> > > Sweet. Suggest opening an umbrellas issue in hbase to implement this
> > > feature. Reference HBASE-2161 (it is closed now). Link trafodion
> > > issue to it. A subtask could have implementation in hbase 2.0,
> > > another could be backport.
> > >
> > > Is is easy to insert your T*ClientScanner?
> > > St.Ack
> > >
> > >
> > >
> > > > Regards,
> > > > Eric
> > > >
> > > > -----Original Message-----
> > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > Of Stack
> > > > Sent: Thursday, August 27, 2015 3:55 PM
> > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > >
> > > > On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi
> > > > <er...@esgyn.com>
> > > > wrote:
> > > >
> > > > > Oops, my bad, the related JIRA was :
> > > > > https://issues.apache.org/jira/browse/HBASE-2161
> > > > >
> > > > > I am suggesting that the special code client side in loadCache()
> > > > > of ClientScanner that is trapping the UnknownScannerException,
> > > > > then on purpose check if it is coming from a lease timeout (and
> > > > > not by a region move) to decide that it would throw a
> > > > > ScannerTimeoutException instead of letting the code go and just
> > > > > reset the scanner and start from last successful retrieve (the
> > > > > way it works for an unknowScannerException due to a region moving).
> > > > > By just removing the special handling that tries to
> > > > > differentiate from unkownScannerException due to lease timeout,
> > > > > we should have a resolution to JIRA 2161- And to our trafodion
> > > > > issue.
> > > > >
> > > > > We are still protecting against dead client that would cause
> > > > > resource leak at region server, since we keep the lease timeout
> > > > > mechanism.
> > > > >
> > > > > Not sure if I have overlooked something, as usually, code is
> > > > > here for a reason :-)...
> > > > >
> > > > >
> > > > Your proposal sounds good to me.
> > > >
> > > > Scanner works the way it does because it has always work this way
> > > (smile).
> > > > A while back, one of the lads suggested we do like dynamodb and
> > > > have scanner have no state on the serverside, the scan next would
> > > > just supply all necessary context. It was argued against because
> > > > serverside setup is so costly. Your suggestion is similar only we
> > > > do it only if Scanner has timed out.
> > > >
> > > > Suggest we keep the current semantic in 1.x at least. We could
> > > > flip to your behavior in 2.x.  Meantime, you'd have to ask for it
> > > > when you set up your Scan object by setting a flag.
> > > >
> > > > Would that work? If you want to have a go at it, I could help out
> > > > on the issue.
> > > >
> > > > St.Ack
> > > >
> > > >
> > > >
> > > >
> > > > > Regards,
> > > > > Eric
> > > > >
> > > > >
> > > > >
> > > > > -----Original Message-----
> > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > > Of Stack
> > > > > Sent: Thursday, August 27, 2015 3:23 PM
> > > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > >
> > > > > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi
> > > > > <er...@esgyn.com>
> > > > > wrote:
> > > > >
> > > > > > Hello St.Ack,
> > > > > > Thanks for your pointer, but I had already investigated JIRA
> > > > > > https://issues.apache.org/jira/browse/HBASE-13090
> > > > > > Unfortunately, this heartbeat will protect against rpc
> > > > > > timeout, not server side lease timeout that we are experiencing
> > > > > > right now.
> > > > > > I have not seen an active JIRA fixing our issue.
> > > > > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > > > > complaining about the exact same issue, but was never resolved.
> > > > > >
> > > > > >
> > > > > Which issue? https://issues.apache.org/jira/browse/HBASE-6121
> > > > > seems unrelated.
> > > > >
> > > > >
> > > > >
> > > > > > The heartbeat JIRA in 13090 protect for situation where server
> > > > > > scanner takes so long to retrieve the highly filtered
> > > > > > information, that it exceeds the RPC timeout (hbase.rpc.timeout).
> > > > >
> > > > >
> > > > >
> > > > > > The timeout we are experiencing is the
> > > > > > hbase.client.scanner.timeout.period,
> > > > > > also deprecatedly known as hbase.regionserver.lease.period The
> > > > > > mechanism is different: here, region server scanners wants to
> > > > > > protect themselves against dead clients that would not perform
> > > > > > "close", and allow releasing server side scanner resources. To
> > > > > > do that, a lease mechanism is implemented, and if between 2
> > > > > > next() call, more than hbase.regionserver.lease.period occurs,
> > > > > > the server side scanner will have been forced closed by this
> > > > > > lease timeout safety mechanism. On late next() call, client
> > > > > > will receive a DNRIOE of type unknownScannerException, and the
> > > > > > client will assess that it is coming most likely from the
> > > > > > lease timeout (and not from a region move), therefore throwing
> > > > > > an exception instead of reset scanner (for the region move
> > > > > > scenario).
> > > > > >
> > > > > > Hbase 1.1 does not address, as far as I have researched, the
> > > > > > hbase.client.scanner.timeout.period issue we are facing.
> > > > > >
> > > > > >
> > > > >
> > > > > Can you not have the high-level query that is being fed by a
> > > > > scan do HBASE-13333? That is, tickle, the ongoing scan on
> > > > > occasion just to say that I'm still alive?
> > > > >
> > > > > Otherwise, what would you suggest? A scan that does not timeout?
> > > > > Or the client being able to set a timeout in the Scan passed to
> > > > > the
> > > server?
> > > > >
> > > > > Sorry for late reply,
> > > > > St.Ack
> > > > >
> > > > >
> > > > >
> > > > > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera and
> > > > > > Hortonworks are having version mismatch on the next official
> > > > > > builds trafodion will support.
> > > > > >
> > > > > > So my question is still open?
> > > > > >
> > > > > > Best regards,
> > > > > > Eric Owhadi
> > > > > >
> > > > > >
> > > > > >
> > > > > > -----Original Message-----
> > > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > > Behalf Of Stack
> > > > > > Sent: Monday, August 24, 2015 11:07 PM
> > > > > > To: HBase Dev List
> > > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > > >
> > > > > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > > > > <er...@esgyn.com>
> > > > > > wrote:
> > > > > >
> > > > > > > Hello everyone,
> > > > > > > We have been facing a situation on trafodion, where we are
> > > > > > > hitting the hbase.client.scanner.timeout.period scenario:
> > > > > > > basically, when doing queries that require spilling to disk
> > > > > > > because of high complexity of what is involved, the
> > > > > > > underlying hbase scanner serving one of the operation
> > > > > > > involved in the complex query cannot call the next() withing
> > > > > > > the timeout specify... too busy taking care of other business.
> > > > > > > This is legit scenario, and I was wondering why in the code,
> > > > > > > special care is done to make sure that client side, if a
> > > > > > > DNRIOE of type unknownScannerException shows up, and the
> > > > > > > hbase.client.scanner.timeout.period time elapsed, we make
> > > > > > > sure to throw a scannerTimeoutException, instead of just let
> > > > > > > it go and reset scanner.
> > > > > > >
> > > > > > > Scanners were redone in hbase 1.1. Can Trafodion come up
> > > > > > > onto hbase
> > > > > 1.1?
> > > > > > See
> > > > > > https://blogs.apache.org/hbase/entry/scan_improvements_in_hbas
> > > > > > e_1
> > > > > > for summary.
> > > > > > St.Ack
> > > > > >
> > > > > >
> > > > > >
> > > > > > > I imagine that the lease time out implementation on region
> > > > > > > server side is supposed to protect from resource leak of
> > > > > > > scanner object server side. But I am not sure why we would
> > > > > > > make it so that client side throw this timeout exception,
> > > > > > > when in fact what just happened was that client was too busy
> > > > > > > to call next() on
> > time.
> > > > > > >
> > > > > > > I am sure there is a reason, but cannot figure it out :-).
> > > > > > >
> > > > > > > BTW, I found this JIRA, talking about exact same thing:
> > > > > > > https://issues.apache.org/jira/browse/HBASE61-21 but with no
> > > > > resolution.
> > > > > > >
> > > > > >
> > > > > >
> > > > > > > Any help understanding the reason of the timeout thrwown
> > > > > > > client side instead of an automatic reset would be much
> > > > > > > appreciated, Best regards, Eric Owhadi
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

RE: Question on hbase.client.scanner.timeout.period

Posted by Eric Owhadi <er...@esgyn.com>.
Thanks for pointing me to HBase-13333, it is indeed supposed to address the
very same problem. With the drawback of requiring client side involvement,
of asynchronous nature. I still have not discovered any reason why just
doing it the way I proposed would lead to any negative side effect. Must
admit I feel uncomfortable since the patch is just about removing code that
usually is added with a purpose :-).
We have not yet run full QA, but at least 100% of trafodion regression test
pass.
As for when the patch will make it to trafodion, given that I did it only
for a CDH build of Trafodion with HBase 1.0 support, I still cannot check it
in (trafodion is still on .98 and builds OK for Cloudera,Hortonworks,Mapr
and Apache). Trafodion would first need to have full support for HBase 1.0
for all Hadoop distro we support, then I will need to redo the patch that is
distro specific, and make sure the build process deals with this... It is my
plan to do so... Hoping that I do not discover any issue with other distro
(like private attributes or functions that I cannot circumvent... but that
would just mean that the patch would not be available for a specific distro)
Eric


-----Original Message-----
From: Jerry He [mailto:jerryjch@gmail.com]
Sent: Saturday, September 5, 2015 1:47 PM
To: dev <de...@hbase.apache.org>
Subject: Re: Question on hbase.client.scanner.timeout.period

You can take a look at HBASE-13333: Renew Scanner Lease without advancing
the RegionScanner, which may be helpful in this kind of case  Your proposal
sounds like a good alternative approach as well.
We should add that JIRA to the blog link Stack mentioned.

Jerry

On Sat, Sep 5, 2015 at 9:07 AM, Stack <st...@duboce.net> wrote:

> On Fri, Sep 4, 2015 at 5:06 PM, Eric Owhadi <er...@esgyn.com> wrote:
>
> > OK so to answer the "is it easy to insert the patched scanner for
> > trafodion", the answer is no.
> >
>
> I suspected this.
>
>
>
> > Was easier on .98, but on 1.0 it was quite a challenge. All about
> > dealing with private attributes instead of protected that are not
> > visible to the PatchClentScanner class that extends ClientScanner.
> > Currently running the regression tests to see if there is no side
> effect...
> > Was able to demonstrate with breakpoint on next() waiting more than
> > 1 mn (the default lease timeout value) that with the patch things
> > gracefully reset and all is good, no row skipped or duplicated,
> > while without, I get the Scanner time out exception. Patch can be
> > turn on or off with a new
> key
> > in hbase-site.xml...
> > I will feel better when this will be deprecated :-).
> >
>
> Smile.
>
> Excellent. You have a patch for us then Eric?  Sounds like the
> interjection of your new Scanner would be for pre-2.0. For 2.0 we
> should just turn on this behavior as the default.
>
> Thanks,
> St.Ack
>
>
>
> > Eric Owhadi
> >
> > -----Original Message-----
> > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> Stack
> > Sent: Friday, August 28, 2015 6:35 PM
> > To: HBase Dev List <de...@hbase.apache.org>
> > Subject: Re: Question on hbase.client.scanner.timeout.period
> >
> > On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi
> > <er...@esgyn.com>
> > wrote:
> >
> > > That sounds good, but given trafodion needs to work on current and
> > > future released version of HBase, unpatched, I will first
> > > implement a ClientScannerTrafodion (to be deprecated), inheriting
> > > from ClientScanner that will just overload the loadCache(),and
> > > make sure that the code that is picking up the right scanner based
> > > on scan object is bypassed to force getting the
> > > ClientScannerTrafodion when appropriate.
> > > Not very elegant, but need to take into consideration trafodion
> > > deployment requirements.
> > > Then, if we do not discover any side effect during our QA related
> > > to this code I will port the fix on HBase to deprecate the custom
> > > scanner (probably first on HBase 2.0, then will let the community
> > > decide if this fix is worth it for back porting...). It will be a
> > > first for me, but that's great, I'll take your offer to help ;-)...
> > >
> >
> > Sweet. Suggest opening an umbrellas issue in hbase to implement this
> > feature. Reference HBASE-2161 (it is closed now). Link trafodion
> > issue to it. A subtask could have implementation in hbase 2.0,
> > another could be backport.
> >
> > Is is easy to insert your T*ClientScanner?
> > St.Ack
> >
> >
> >
> > > Regards,
> > > Eric
> > >
> > > -----Original Message-----
> > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > Of Stack
> > > Sent: Thursday, August 27, 2015 3:55 PM
> > > To: HBase Dev List <de...@hbase.apache.org>
> > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > >
> > > On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi
> > > <er...@esgyn.com>
> > > wrote:
> > >
> > > > Oops, my bad, the related JIRA was :
> > > > https://issues.apache.org/jira/browse/HBASE-2161
> > > >
> > > > I am suggesting that the special code client side in loadCache()
> > > > of ClientScanner that is trapping the UnknownScannerException,
> > > > then on purpose check if it is coming from a lease timeout (and
> > > > not by a region move) to decide that it would throw a
> > > > ScannerTimeoutException instead of letting the code go and just
> > > > reset the scanner and start from last successful retrieve (the
> > > > way it works for an unknowScannerException due to a region moving).
> > > > By just removing the special handling that tries to
> > > > differentiate from unkownScannerException due to lease timeout,
> > > > we should have a resolution to JIRA 2161- And to our trafodion
> > > > issue.
> > > >
> > > > We are still protecting against dead client that would cause
> > > > resource leak at region server, since we keep the lease timeout
> > > > mechanism.
> > > >
> > > > Not sure if I have overlooked something, as usually, code is
> > > > here for a reason :-)...
> > > >
> > > >
> > > Your proposal sounds good to me.
> > >
> > > Scanner works the way it does because it has always work this way
> > (smile).
> > > A while back, one of the lads suggested we do like dynamodb and
> > > have scanner have no state on the serverside, the scan next would
> > > just supply all necessary context. It was argued against because
> > > serverside setup is so costly. Your suggestion is similar only we
> > > do it only if Scanner has timed out.
> > >
> > > Suggest we keep the current semantic in 1.x at least. We could
> > > flip to your behavior in 2.x.  Meantime, you'd have to ask for it
> > > when you set up your Scan object by setting a flag.
> > >
> > > Would that work? If you want to have a go at it, I could help out
> > > on the issue.
> > >
> > > St.Ack
> > >
> > >
> > >
> > >
> > > > Regards,
> > > > Eric
> > > >
> > > >
> > > >
> > > > -----Original Message-----
> > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > Of Stack
> > > > Sent: Thursday, August 27, 2015 3:23 PM
> > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > >
> > > > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi
> > > > <er...@esgyn.com>
> > > > wrote:
> > > >
> > > > > Hello St.Ack,
> > > > > Thanks for your pointer, but I had already investigated JIRA
> > > > > https://issues.apache.org/jira/browse/HBASE-13090
> > > > > Unfortunately, this heartbeat will protect against rpc
> > > > > timeout, not server side lease timeout that we are experiencing
> > > > > right now.
> > > > > I have not seen an active JIRA fixing our issue.
> > > > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > > > complaining about the exact same issue, but was never resolved.
> > > > >
> > > > >
> > > > Which issue? https://issues.apache.org/jira/browse/HBASE-6121
> > > > seems unrelated.
> > > >
> > > >
> > > >
> > > > > The heartbeat JIRA in 13090 protect for situation where server
> > > > > scanner takes so long to retrieve the highly filtered
> > > > > information, that it exceeds the RPC timeout (hbase.rpc.timeout).
> > > >
> > > >
> > > >
> > > > > The timeout we are experiencing is the
> > > > > hbase.client.scanner.timeout.period,
> > > > > also deprecatedly known as hbase.regionserver.lease.period The
> > > > > mechanism is different: here, region server scanners wants to
> > > > > protect themselves against dead clients that would not perform
> > > > > "close", and allow releasing server side scanner resources. To
> > > > > do that, a lease mechanism is implemented, and if between 2
> > > > > next() call, more than hbase.regionserver.lease.period occurs,
> > > > > the server side scanner will have been forced closed by this
> > > > > lease timeout safety mechanism. On late next() call, client
> > > > > will receive a DNRIOE of type unknownScannerException, and the
> > > > > client will assess that it is coming most likely from the
> > > > > lease timeout (and not from a region move), therefore throwing
> > > > > an exception instead of reset scanner (for the region move
> > > > > scenario).
> > > > >
> > > > > Hbase 1.1 does not address, as far as I have researched, the
> > > > > hbase.client.scanner.timeout.period issue we are facing.
> > > > >
> > > > >
> > > >
> > > > Can you not have the high-level query that is being fed by a
> > > > scan do HBASE-13333? That is, tickle, the ongoing scan on
> > > > occasion just to say that I'm still alive?
> > > >
> > > > Otherwise, what would you suggest? A scan that does not timeout?
> > > > Or the client being able to set a timeout in the Scan passed to
> > > > the
> > server?
> > > >
> > > > Sorry for late reply,
> > > > St.Ack
> > > >
> > > >
> > > >
> > > > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera and
> > > > > Hortonworks are having version mismatch on the next official
> > > > > builds trafodion will support.
> > > > >
> > > > > So my question is still open?
> > > > >
> > > > > Best regards,
> > > > > Eric Owhadi
> > > > >
> > > > >
> > > > >
> > > > > -----Original Message-----
> > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On
> > > > > Behalf Of Stack
> > > > > Sent: Monday, August 24, 2015 11:07 PM
> > > > > To: HBase Dev List
> > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > >
> > > > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > > > <er...@esgyn.com>
> > > > > wrote:
> > > > >
> > > > > > Hello everyone,
> > > > > > We have been facing a situation on trafodion, where we are
> > > > > > hitting the hbase.client.scanner.timeout.period scenario:
> > > > > > basically, when doing queries that require spilling to disk
> > > > > > because of high complexity of what is involved, the
> > > > > > underlying hbase scanner serving one of the operation
> > > > > > involved in the complex query cannot call the next() withing
> > > > > > the timeout specify... too busy taking care of other business.
> > > > > > This is legit scenario, and I was wondering why in the code,
> > > > > > special care is done to make sure that client side, if a
> > > > > > DNRIOE of type unknownScannerException shows up, and the
> > > > > > hbase.client.scanner.timeout.period time elapsed, we make
> > > > > > sure to throw a scannerTimeoutException, instead of just let
> > > > > > it go and reset scanner.
> > > > > >
> > > > > > Scanners were redone in hbase 1.1. Can Trafodion come up
> > > > > > onto hbase
> > > > 1.1?
> > > > > See
> > > > > https://blogs.apache.org/hbase/entry/scan_improvements_in_hbas
> > > > > e_1
> > > > > for summary.
> > > > > St.Ack
> > > > >
> > > > >
> > > > >
> > > > > > I imagine that the lease time out implementation on region
> > > > > > server side is supposed to protect from resource leak of
> > > > > > scanner object server side. But I am not sure why we would
> > > > > > make it so that client side throw this timeout exception,
> > > > > > when in fact what just happened was that client was too busy
> > > > > > to call next() on
> time.
> > > > > >
> > > > > > I am sure there is a reason, but cannot figure it out :-).
> > > > > >
> > > > > > BTW, I found this JIRA, talking about exact same thing:
> > > > > > https://issues.apache.org/jira/browse/HBASE61-21 but with no
> > > > resolution.
> > > > > >
> > > > >
> > > > >
> > > > > > Any help understanding the reason of the timeout thrwown
> > > > > > client side instead of an automatic reset would be much
> > > > > > appreciated, Best regards, Eric Owhadi
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Question on hbase.client.scanner.timeout.period

Posted by Jerry He <je...@gmail.com>.
You can take a look at HBASE-13333: Renew Scanner Lease without advancing
the RegionScanner, which may be helpful in this kind of case  Your proposal
sounds like a good alternative approach as well.
We should add that JIRA to the blog link Stack mentioned.

Jerry

On Sat, Sep 5, 2015 at 9:07 AM, Stack <st...@duboce.net> wrote:

> On Fri, Sep 4, 2015 at 5:06 PM, Eric Owhadi <er...@esgyn.com> wrote:
>
> > OK so to answer the "is it easy to insert the patched scanner for
> > trafodion", the answer is no.
> >
>
> I suspected this.
>
>
>
> > Was easier on .98, but on 1.0 it was quite a challenge. All about dealing
> > with private attributes instead of protected that are not visible to the
> > PatchClentScanner class that extends ClientScanner.
> > Currently running the regression tests to see if there is no side
> effect...
> > Was able to demonstrate with breakpoint on next() waiting more than 1 mn
> > (the default lease timeout value) that with the patch things gracefully
> > reset and all is good, no row skipped or duplicated, while without, I get
> > the Scanner time out exception. Patch can be turn on or off with a new
> key
> > in hbase-site.xml...
> > I will feel better when this will be deprecated :-).
> >
>
> Smile.
>
> Excellent. You have a patch for us then Eric?  Sounds like the interjection
> of your new Scanner would be for pre-2.0. For 2.0 we should just turn on
> this behavior as the default.
>
> Thanks,
> St.Ack
>
>
>
> > Eric Owhadi
> >
> > -----Original Message-----
> > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> Stack
> > Sent: Friday, August 28, 2015 6:35 PM
> > To: HBase Dev List <de...@hbase.apache.org>
> > Subject: Re: Question on hbase.client.scanner.timeout.period
> >
> > On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi <er...@esgyn.com>
> > wrote:
> >
> > > That sounds good, but given trafodion needs to work on current and
> > > future released version of HBase, unpatched, I will first implement a
> > > ClientScannerTrafodion (to be deprecated), inheriting from
> > > ClientScanner that will just overload the loadCache(),and make sure
> > > that the code that is picking up the right scanner based on scan
> > > object is bypassed to force getting the ClientScannerTrafodion when
> > > appropriate.
> > > Not very elegant, but need to take into consideration trafodion
> > > deployment requirements.
> > > Then, if we do not discover any side effect during our QA related to
> > > this code I will port the fix on HBase to deprecate the custom scanner
> > > (probably first on HBase 2.0, then will let the community decide if
> > > this fix is worth it for back porting...). It will be a first for me,
> > > but that's great, I'll take your offer to help ;-)...
> > >
> >
> > Sweet. Suggest opening an umbrellas issue in hbase to implement this
> > feature. Reference HBASE-2161 (it is closed now). Link trafodion issue to
> > it. A subtask could have implementation in hbase 2.0, another could be
> > backport.
> >
> > Is is easy to insert your T*ClientScanner?
> > St.Ack
> >
> >
> >
> > > Regards,
> > > Eric
> > >
> > > -----Original Message-----
> > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> > > Stack
> > > Sent: Thursday, August 27, 2015 3:55 PM
> > > To: HBase Dev List <de...@hbase.apache.org>
> > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > >
> > > On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi <er...@esgyn.com>
> > > wrote:
> > >
> > > > Oops, my bad, the related JIRA was :
> > > > https://issues.apache.org/jira/browse/HBASE-2161
> > > >
> > > > I am suggesting that the special code client side in loadCache() of
> > > > ClientScanner that is trapping the UnknownScannerException, then on
> > > > purpose check if it is coming from a lease timeout (and not by a
> > > > region move) to decide that it would throw a ScannerTimeoutException
> > > > instead of letting the code go and just reset the scanner and start
> > > > from last successful retrieve (the way it works for an
> > > > unknowScannerException due to a region moving).
> > > > By just removing the special handling that tries to differentiate
> > > > from unkownScannerException due to lease timeout, we should have a
> > > > resolution to JIRA 2161- And to our trafodion issue.
> > > >
> > > > We are still protecting against dead client that would cause
> > > > resource leak at region server, since we keep the lease timeout
> > > > mechanism.
> > > >
> > > > Not sure if I have overlooked something, as usually, code is here
> > > > for a reason :-)...
> > > >
> > > >
> > > Your proposal sounds good to me.
> > >
> > > Scanner works the way it does because it has always work this way
> > (smile).
> > > A while back, one of the lads suggested we do like dynamodb and have
> > > scanner have no state on the serverside, the scan next would just
> > > supply all necessary context. It was argued against because serverside
> > > setup is so costly. Your suggestion is similar only we do it only if
> > > Scanner has timed out.
> > >
> > > Suggest we keep the current semantic in 1.x at least. We could flip to
> > > your behavior in 2.x.  Meantime, you'd have to ask for it when you set
> > > up your Scan object by setting a flag.
> > >
> > > Would that work? If you want to have a go at it, I could help out on
> > > the issue.
> > >
> > > St.Ack
> > >
> > >
> > >
> > >
> > > > Regards,
> > > > Eric
> > > >
> > > >
> > > >
> > > > -----Original Message-----
> > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> > > > Stack
> > > > Sent: Thursday, August 27, 2015 3:23 PM
> > > > To: HBase Dev List <de...@hbase.apache.org>
> > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > >
> > > > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi <er...@esgyn.com>
> > > > wrote:
> > > >
> > > > > Hello St.Ack,
> > > > > Thanks for your pointer, but I had already investigated JIRA
> > > > > https://issues.apache.org/jira/browse/HBASE-13090
> > > > > Unfortunately, this heartbeat will protect against rpc timeout,
> > > > > not server side lease timeout that we are experiencing right now.
> > > > > I have not seen an active JIRA fixing our issue.
> > > > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > > > complaining about the exact same issue, but was never resolved.
> > > > >
> > > > >
> > > > Which issue? https://issues.apache.org/jira/browse/HBASE-6121 seems
> > > > unrelated.
> > > >
> > > >
> > > >
> > > > > The heartbeat JIRA in 13090 protect for situation where server
> > > > > scanner takes so long to retrieve the highly filtered information,
> > > > > that it exceeds the RPC timeout (hbase.rpc.timeout).
> > > >
> > > >
> > > >
> > > > > The timeout we are experiencing is the
> > > > > hbase.client.scanner.timeout.period,
> > > > > also deprecatedly known as hbase.regionserver.lease.period The
> > > > > mechanism is different: here, region server scanners wants to
> > > > > protect themselves against dead clients that would not perform
> > > > > "close", and allow releasing server side scanner resources. To do
> > > > > that, a lease mechanism is implemented, and if between 2 next()
> > > > > call, more than hbase.regionserver.lease.period occurs, the server
> > > > > side scanner will have been forced closed by this lease timeout
> > > > > safety mechanism. On late next() call, client will receive a
> > > > > DNRIOE of type unknownScannerException, and the client will assess
> > > > > that it is coming most likely from the lease timeout (and not from
> > > > > a region move), therefore throwing an exception instead of reset
> > > > > scanner (for the region move scenario).
> > > > >
> > > > > Hbase 1.1 does not address, as far as I have researched, the
> > > > > hbase.client.scanner.timeout.period issue we are facing.
> > > > >
> > > > >
> > > >
> > > > Can you not have the high-level query that is being fed by a scan do
> > > > HBASE-13333? That is, tickle, the ongoing scan on occasion just to
> > > > say that I'm still alive?
> > > >
> > > > Otherwise, what would you suggest? A scan that does not timeout? Or
> > > > the client being able to set a timeout in the Scan passed to the
> > server?
> > > >
> > > > Sorry for late reply,
> > > > St.Ack
> > > >
> > > >
> > > >
> > > > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera and
> > > > > Hortonworks are having version mismatch on the next official
> > > > > builds trafodion will support.
> > > > >
> > > > > So my question is still open?
> > > > >
> > > > > Best regards,
> > > > > Eric Owhadi
> > > > >
> > > > >
> > > > >
> > > > > -----Original Message-----
> > > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > > Of Stack
> > > > > Sent: Monday, August 24, 2015 11:07 PM
> > > > > To: HBase Dev List
> > > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > > >
> > > > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > > > <er...@esgyn.com>
> > > > > wrote:
> > > > >
> > > > > > Hello everyone,
> > > > > > We have been facing a situation on trafodion, where we are
> > > > > > hitting the hbase.client.scanner.timeout.period scenario:
> > > > > > basically, when doing queries that require spilling to disk
> > > > > > because of high complexity of what is involved, the underlying
> > > > > > hbase scanner serving one of the operation involved in the
> > > > > > complex query cannot call the next() withing the timeout
> > > > > > specify... too busy taking care of other business.
> > > > > > This is legit scenario, and I was wondering why in the code,
> > > > > > special care is done to make sure that client side, if a DNRIOE
> > > > > > of type unknownScannerException shows up, and the
> > > > > > hbase.client.scanner.timeout.period time elapsed, we make sure
> > > > > > to throw a scannerTimeoutException, instead of just let it go
> > > > > > and reset scanner.
> > > > > >
> > > > > > Scanners were redone in hbase 1.1. Can Trafodion come up onto
> > > > > > hbase
> > > > 1.1?
> > > > > See
> > > > > https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1
> > > > > for summary.
> > > > > St.Ack
> > > > >
> > > > >
> > > > >
> > > > > > I imagine that the lease time out implementation on region
> > > > > > server side is supposed to protect from resource leak of scanner
> > > > > > object server side. But I am not sure why we would make it so
> > > > > > that client side throw this timeout exception, when in fact what
> > > > > > just happened was that client was too busy to call next() on
> time.
> > > > > >
> > > > > > I am sure there is a reason, but cannot figure it out :-).
> > > > > >
> > > > > > BTW, I found this JIRA, talking about exact same thing:
> > > > > > https://issues.apache.org/jira/browse/HBASE61-21 but with no
> > > > resolution.
> > > > > >
> > > > >
> > > > >
> > > > > > Any help understanding the reason of the timeout thrwown client
> > > > > > side instead of an automatic reset would be much appreciated,
> > > > > > Best regards, Eric Owhadi
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Question on hbase.client.scanner.timeout.period

Posted by Stack <st...@duboce.net>.
On Fri, Sep 4, 2015 at 5:06 PM, Eric Owhadi <er...@esgyn.com> wrote:

> OK so to answer the "is it easy to insert the patched scanner for
> trafodion", the answer is no.
>

I suspected this.



> Was easier on .98, but on 1.0 it was quite a challenge. All about dealing
> with private attributes instead of protected that are not visible to the
> PatchClentScanner class that extends ClientScanner.
> Currently running the regression tests to see if there is no side effect...
> Was able to demonstrate with breakpoint on next() waiting more than 1 mn
> (the default lease timeout value) that with the patch things gracefully
> reset and all is good, no row skipped or duplicated, while without, I get
> the Scanner time out exception. Patch can be turn on or off with a new key
> in hbase-site.xml...
> I will feel better when this will be deprecated :-).
>

Smile.

Excellent. You have a patch for us then Eric?  Sounds like the interjection
of your new Scanner would be for pre-2.0. For 2.0 we should just turn on
this behavior as the default.

Thanks,
St.Ack



> Eric Owhadi
>
> -----Original Message-----
> From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of Stack
> Sent: Friday, August 28, 2015 6:35 PM
> To: HBase Dev List <de...@hbase.apache.org>
> Subject: Re: Question on hbase.client.scanner.timeout.period
>
> On Fri, Aug 28, 2015 at 11:31 AM, Eric Owhadi <er...@esgyn.com>
> wrote:
>
> > That sounds good, but given trafodion needs to work on current and
> > future released version of HBase, unpatched, I will first implement a
> > ClientScannerTrafodion (to be deprecated), inheriting from
> > ClientScanner that will just overload the loadCache(),and make sure
> > that the code that is picking up the right scanner based on scan
> > object is bypassed to force getting the ClientScannerTrafodion when
> > appropriate.
> > Not very elegant, but need to take into consideration trafodion
> > deployment requirements.
> > Then, if we do not discover any side effect during our QA related to
> > this code I will port the fix on HBase to deprecate the custom scanner
> > (probably first on HBase 2.0, then will let the community decide if
> > this fix is worth it for back porting...). It will be a first for me,
> > but that's great, I'll take your offer to help ;-)...
> >
>
> Sweet. Suggest opening an umbrellas issue in hbase to implement this
> feature. Reference HBASE-2161 (it is closed now). Link trafodion issue to
> it. A subtask could have implementation in hbase 2.0, another could be
> backport.
>
> Is is easy to insert your T*ClientScanner?
> St.Ack
>
>
>
> > Regards,
> > Eric
> >
> > -----Original Message-----
> > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> > Stack
> > Sent: Thursday, August 27, 2015 3:55 PM
> > To: HBase Dev List <de...@hbase.apache.org>
> > Subject: Re: Question on hbase.client.scanner.timeout.period
> >
> > On Thu, Aug 27, 2015 at 1:39 PM, Eric Owhadi <er...@esgyn.com>
> > wrote:
> >
> > > Oops, my bad, the related JIRA was :
> > > https://issues.apache.org/jira/browse/HBASE-2161
> > >
> > > I am suggesting that the special code client side in loadCache() of
> > > ClientScanner that is trapping the UnknownScannerException, then on
> > > purpose check if it is coming from a lease timeout (and not by a
> > > region move) to decide that it would throw a ScannerTimeoutException
> > > instead of letting the code go and just reset the scanner and start
> > > from last successful retrieve (the way it works for an
> > > unknowScannerException due to a region moving).
> > > By just removing the special handling that tries to differentiate
> > > from unkownScannerException due to lease timeout, we should have a
> > > resolution to JIRA 2161- And to our trafodion issue.
> > >
> > > We are still protecting against dead client that would cause
> > > resource leak at region server, since we keep the lease timeout
> > > mechanism.
> > >
> > > Not sure if I have overlooked something, as usually, code is here
> > > for a reason :-)...
> > >
> > >
> > Your proposal sounds good to me.
> >
> > Scanner works the way it does because it has always work this way
> (smile).
> > A while back, one of the lads suggested we do like dynamodb and have
> > scanner have no state on the serverside, the scan next would just
> > supply all necessary context. It was argued against because serverside
> > setup is so costly. Your suggestion is similar only we do it only if
> > Scanner has timed out.
> >
> > Suggest we keep the current semantic in 1.x at least. We could flip to
> > your behavior in 2.x.  Meantime, you'd have to ask for it when you set
> > up your Scan object by setting a flag.
> >
> > Would that work? If you want to have a go at it, I could help out on
> > the issue.
> >
> > St.Ack
> >
> >
> >
> >
> > > Regards,
> > > Eric
> > >
> > >
> > >
> > > -----Original Message-----
> > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf Of
> > > Stack
> > > Sent: Thursday, August 27, 2015 3:23 PM
> > > To: HBase Dev List <de...@hbase.apache.org>
> > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > >
> > > On Tue, Aug 25, 2015 at 8:03 AM, Eric Owhadi <er...@esgyn.com>
> > > wrote:
> > >
> > > > Hello St.Ack,
> > > > Thanks for your pointer, but I had already investigated JIRA
> > > > https://issues.apache.org/jira/browse/HBASE-13090
> > > > Unfortunately, this heartbeat will protect against rpc timeout,
> > > > not server side lease timeout that we are experiencing right now.
> > > > I have not seen an active JIRA fixing our issue.
> > > > Only https://issues.apache.org/jira/browse/HBASE6121 is
> > > > complaining about the exact same issue, but was never resolved.
> > > >
> > > >
> > > Which issue? https://issues.apache.org/jira/browse/HBASE-6121 seems
> > > unrelated.
> > >
> > >
> > >
> > > > The heartbeat JIRA in 13090 protect for situation where server
> > > > scanner takes so long to retrieve the highly filtered information,
> > > > that it exceeds the RPC timeout (hbase.rpc.timeout).
> > >
> > >
> > >
> > > > The timeout we are experiencing is the
> > > > hbase.client.scanner.timeout.period,
> > > > also deprecatedly known as hbase.regionserver.lease.period The
> > > > mechanism is different: here, region server scanners wants to
> > > > protect themselves against dead clients that would not perform
> > > > "close", and allow releasing server side scanner resources. To do
> > > > that, a lease mechanism is implemented, and if between 2 next()
> > > > call, more than hbase.regionserver.lease.period occurs, the server
> > > > side scanner will have been forced closed by this lease timeout
> > > > safety mechanism. On late next() call, client will receive a
> > > > DNRIOE of type unknownScannerException, and the client will assess
> > > > that it is coming most likely from the lease timeout (and not from
> > > > a region move), therefore throwing an exception instead of reset
> > > > scanner (for the region move scenario).
> > > >
> > > > Hbase 1.1 does not address, as far as I have researched, the
> > > > hbase.client.scanner.timeout.period issue we are facing.
> > > >
> > > >
> > >
> > > Can you not have the high-level query that is being fed by a scan do
> > > HBASE-13333? That is, tickle, the ongoing scan on occasion just to
> > > say that I'm still alive?
> > >
> > > Otherwise, what would you suggest? A scan that does not timeout? Or
> > > the client being able to set a timeout in the Scan passed to the
> server?
> > >
> > > Sorry for late reply,
> > > St.Ack
> > >
> > >
> > >
> > > > And yes, we will move to Hbase 1.1, and 1.0 as Cloudera and
> > > > Hortonworks are having version mismatch on the next official
> > > > builds trafodion will support.
> > > >
> > > > So my question is still open?
> > > >
> > > > Best regards,
> > > > Eric Owhadi
> > > >
> > > >
> > > >
> > > > -----Original Message-----
> > > > From: saint.ack@gmail.com [mailto:saint.ack@gmail.com] On Behalf
> > > > Of Stack
> > > > Sent: Monday, August 24, 2015 11:07 PM
> > > > To: HBase Dev List
> > > > Subject: Re: Question on hbase.client.scanner.timeout.period
> > > >
> > > > On Mon, Aug 24, 2015 at 4:48 PM, Eric Owhadi
> > > > <er...@esgyn.com>
> > > > wrote:
> > > >
> > > > > Hello everyone,
> > > > > We have been facing a situation on trafodion, where we are
> > > > > hitting the hbase.client.scanner.timeout.period scenario:
> > > > > basically, when doing queries that require spilling to disk
> > > > > because of high complexity of what is involved, the underlying
> > > > > hbase scanner serving one of the operation involved in the
> > > > > complex query cannot call the next() withing the timeout
> > > > > specify... too busy taking care of other business.
> > > > > This is legit scenario, and I was wondering why in the code,
> > > > > special care is done to make sure that client side, if a DNRIOE
> > > > > of type unknownScannerException shows up, and the
> > > > > hbase.client.scanner.timeout.period time elapsed, we make sure
> > > > > to throw a scannerTimeoutException, instead of just let it go
> > > > > and reset scanner.
> > > > >
> > > > > Scanners were redone in hbase 1.1. Can Trafodion come up onto
> > > > > hbase
> > > 1.1?
> > > > See
> > > > https://blogs.apache.org/hbase/entry/scan_improvements_in_hbase_1
> > > > for summary.
> > > > St.Ack
> > > >
> > > >
> > > >
> > > > > I imagine that the lease time out implementation on region
> > > > > server side is supposed to protect from resource leak of scanner
> > > > > object server side. But I am not sure why we would make it so
> > > > > that client side throw this timeout exception, when in fact what
> > > > > just happened was that client was too busy to call next() on time.
> > > > >
> > > > > I am sure there is a reason, but cannot figure it out :-).
> > > > >
> > > > > BTW, I found this JIRA, talking about exact same thing:
> > > > > https://issues.apache.org/jira/browse/HBASE61-21 but with no
> > > resolution.
> > > > >
> > > >
> > > >
> > > > > Any help understanding the reason of the timeout thrwown client
> > > > > side instead of an automatic reset would be much appreciated,
> > > > > Best regards, Eric Owhadi
> > > > >
> > > >
> > >
> >
>