Re: How Long Will HBase Hold A Row Write Lock?

2018-03-10 Thread Anoop John
Hi Saad
   In your initial mail you mentioned that there are lots
of checkAndPut ops but on different rows. The failure in obtaining
locks (write lock as it is checkAndPut) means there is contention on
the same row key.  If that is the case , ya that is the 1st step
before BC reads and it make sense.

On the Q on why not caching the compacted file content, yes it is this
way. Even if cache on write is true. This is because some times the
compacted result file could be so large (what is major compaction) and
that will exhaust the BC if written. Also it might contain some data
which are very old.  There is a jira recently raised jira which
discuss abt this.  Pls see HBASE-20045


-Anoop-

On Sun, Mar 11, 2018 at 7:57 AM, Saad Mufti  wrote:
> Although now that I think about this a bit more, all the failures we saw
> were failure to obtain a row lock, and in the thread stack traces we always
> saw it somewhere inside getRowLockInternal and similar. Never saw any
> contention on bucket cache lock that I could see.
>
> Cheers.
>
> 
> Saad
>
>
> On Sat, Mar 10, 2018 at 8:04 PM, Saad Mufti  wrote:
>
>> Also, for now we have mitigated this problem by using the new setting in
>> HBase 1.4.0 that prevents one slow region server from blocking all client
>> requests. Of course it causes some timeouts but our overall ecosystem
>> contains Kafka queues for retries, so we can live with that. From what I
>> can see, it looks like this setting also has the good effect of preventing
>> clients from hammering a region server that is slow because its IPC queues
>> are backed up, allowing it to recover faster.
>>
>> Does that make sense?
>>
>> Cheers.
>>
>> 
>> Saad
>>
>>
>> On Sat, Mar 10, 2018 at 7:04 PM, Saad Mufti  wrote:
>>
>>> So if I understand correctly, we would mitigate the problem by not
>>> evicting blocks for archived files immediately? Wouldn't this potentially
>>> lead to problems later if the LRU algo chooses to evict blocks for active
>>> files and leave blocks for archived files in there?
>>>
>>> I would definitely love to test this!!! Unfortunately we are running on
>>> EMR and the details of how to patch HBase under EMR are not clear to me :-(
>>>
>>> What we would really love would be a setting for actually immediately
>>> caching blocks for a new compacted file. I have seen in the code that even
>>> is we have the cache on write setting set to true, it will refuse to cache
>>> blocks for a file that is a newly compacted one. In our case we have sized
>>> the bucket cache to be big enough to hold all our data, and really want to
>>> avoid having to go to S3 until the last possible moment. A config setting
>>> to test this would be great.
>>>
>>> But thanks everyone for your feedback. Any more would also be welcome on
>>> the idea to let a user cache all newly compacted files.
>>>
>>> 
>>> Saad
>>>
>>>
>>> On Wed, Mar 7, 2018 at 12:00 AM, Anoop John 
>>> wrote:
>>>
 >>a) it was indeed one of the regions that was being compacted, major
 compaction in one case, minor compaction in another, the issue started
 just
 after compaction completed blowing away bucket cached blocks for the
 older
 HFile's

 About this part.Ya after the compaction, there is a step where the
 compacted away HFile's blocks getting removed from cache. This op takes a
 write lock for this region (In Bucket Cache layer)..  Every read op which
 is part of checkAndPut will try read from BC and that in turn need a read
 lock for this region.  So there is chances that the read locks starve
 because of so many frequent write locks .  Each block evict will attain
 the
 write lock one after other.  Will it be possible for you to patch this
 evict and test once? We can avoid the immediate evict from BC after
 compaction. I can help you with a patch if you wish

 Anoop



 On Mon, Mar 5, 2018 at 11:07 AM, ramkrishna vasudevan <
 ramkrishna.s.vasude...@gmail.com> wrote:
 > Hi Saad
 >
 > Your argument here
 >>> The
 >>>theory is that since prefetch is an async operation, a lot of the
 reads
 in
 >>>the checkAndPut for the region in question start reading from S3
 which is
 >>>slow. So the write lock obtained for the checkAndPut is held for a
 longer
 >>>duration than normal. This has cascading upstream effects. Does that
 sound
 >>>plausible?
 >
 > Seems very much plausible. So before even the prefetch happens say for
 > 'block 1' - and you have already issues N checkAndPut calls for the
 rows
 in
 > that 'block 1' -  all those checkAndPut will have to read that block
 from
 > S3 to perform the get() and then apply the mutation.
 >
 > This may happen for multiple threads at the same time because we are
 not
 > sure when the prefetch would have actually been completed. I don know
 what
 > are the general read characteris

Re: TableSnapshotInputFormat Behavior In HBase 1.4.0

2018-03-10 Thread Saad Mufti
The question remain though of why it is even accessing a column family's
files that should be excluded based on the Scan. And that column family
does NOT specify prefetch on open in its schema. Only the one we want to
read specifies prefetch on open, which we want to override if possible for
the Spark job.


Saad

On Sat, Mar 10, 2018 at 9:51 PM, Saad Mufti  wrote:

> See below more I found on item 3.
>
> Cheers.
>
> 
> Saad
>
> On Sat, Mar 10, 2018 at 7:17 PM, Saad Mufti  wrote:
>
>> Hi,
>>
>> I am running a Spark job (Spark 2.2.1) on an EMR cluster in AWS. There is
>> no Hbase installed on the cluster, only HBase libs linked to my Spark app.
>> We are reading the snapshot info from a HBase folder in S3 using
>> TableSnapshotInputFormat class from HBase 1.4.0 to have the Spark job read
>> snapshot info directly from the S3 based filesystem instead of going
>> through any region server.
>>
>> I have observed a few behaviors while debugging performance that are
>> concerning, some we could mitigate and other I am looking for clarity on:
>>
>> 1)  the TableSnapshotInputFormatImpl code is trying to get locality
>> information for the region splits, for a snapshots with a large number of
>> files (over 35 in our case) this causing single threaded scan of all
>> the file listings in a single thread in the driver. And it was useless
>> because there is really no useful locality information to glean since all
>> the files are in S3 and not HDFS. So I was forced to make a copy of
>> TableSnapshotInputFormatImpl.java in our code and control this with a
>> config setting I made up. That got rid of the hours long scan, so I am good
>> with this part for now.
>>
>> 2) I have set a single column family in the Scan that I set on the hbase
>> configuration via
>>
>> scan.addFamily(str.getBytes()))
>>
>> hBaseConf.set(TableInputFormat.SCAN, convertScanToString(scan))
>>
>>
>> But when this code is executing under Spark and I observe the threads and
>> logs on Spark executors, I it is reading from S3 files for a column family
>> that was not included in the scan. This column family was intentionally
>> excluded because it is much larger than the others and so we wanted to
>> avoid the cost.
>>
>> Any advice on what I am doing wrong would be appreciated.
>>
>> 3) We also explicitly set caching of blocks to false on the scan,
>> although I see that in TableSnapshotInputFormatImpl.java it is again set
>> to false internally also. But when running the Spark job, some executors
>> were taking much longer than others, and when I observe their threads, I
>> see periodic messages about a few hundred megs of RAM used by the block
>> cache, and the thread is sitting there reading data from S3, and is
>> occasionally blocked a couple of other threads that have the
>> "hfile-prefetcher" name in them. Going back to 2) above, they seem to be
>> reading the wrong column family, but in this item I am more concerned about
>> why they appear to be prefetching blocks and caching them, when the Scan
>> object has a setting to not cache blocks at all?
>>
>
> I think I figured out item 3, the column family descriptor for the table
> in question has prefetch on open set in its schema. Now for the Spark job,
> I don't think this serves any useful purpose does it? But I can't see any
> way to override it. If these is, I'd appreciate some advice.
>

> Thanks.
>
>
>>
>> Thanks in advance for any insights anyone can provide.
>>
>> 
>> Saad
>>
>>
>
>


Re: TableSnapshotInputFormat Behavior In HBase 1.4.0

2018-03-10 Thread Saad Mufti
See below more I found on item 3.

Cheers.


Saad

On Sat, Mar 10, 2018 at 7:17 PM, Saad Mufti  wrote:

> Hi,
>
> I am running a Spark job (Spark 2.2.1) on an EMR cluster in AWS. There is
> no Hbase installed on the cluster, only HBase libs linked to my Spark app.
> We are reading the snapshot info from a HBase folder in S3 using
> TableSnapshotInputFormat class from HBase 1.4.0 to have the Spark job read
> snapshot info directly from the S3 based filesystem instead of going
> through any region server.
>
> I have observed a few behaviors while debugging performance that are
> concerning, some we could mitigate and other I am looking for clarity on:
>
> 1)  the TableSnapshotInputFormatImpl code is trying to get locality
> information for the region splits, for a snapshots with a large number of
> files (over 35 in our case) this causing single threaded scan of all
> the file listings in a single thread in the driver. And it was useless
> because there is really no useful locality information to glean since all
> the files are in S3 and not HDFS. So I was forced to make a copy of
> TableSnapshotInputFormatImpl.java in our code and control this with a
> config setting I made up. That got rid of the hours long scan, so I am good
> with this part for now.
>
> 2) I have set a single column family in the Scan that I set on the hbase
> configuration via
>
> scan.addFamily(str.getBytes()))
>
> hBaseConf.set(TableInputFormat.SCAN, convertScanToString(scan))
>
>
> But when this code is executing under Spark and I observe the threads and
> logs on Spark executors, I it is reading from S3 files for a column family
> that was not included in the scan. This column family was intentionally
> excluded because it is much larger than the others and so we wanted to
> avoid the cost.
>
> Any advice on what I am doing wrong would be appreciated.
>
> 3) We also explicitly set caching of blocks to false on the scan, although
> I see that in TableSnapshotInputFormatImpl.java it is again set to false
> internally also. But when running the Spark job, some executors were taking
> much longer than others, and when I observe their threads, I see periodic
> messages about a few hundred megs of RAM used by the block cache, and the
> thread is sitting there reading data from S3, and is occasionally blocked a
> couple of other threads that have the "hfile-prefetcher" name in them.
> Going back to 2) above, they seem to be reading the wrong column family,
> but in this item I am more concerned about why they appear to be
> prefetching blocks and caching them, when the Scan object has a setting to
> not cache blocks at all?
>

I think I figured out item 3, the column family descriptor for the table in
question has prefetch on open set in its schema. Now for the Spark job, I
don't think this serves any useful purpose does it? But I can't see any way
to override it. If these is, I'd appreciate some advice.

Thanks.


>
> Thanks in advance for any insights anyone can provide.
>
> 
> Saad
>
>


Re: How Long Will HBase Hold A Row Write Lock?

2018-03-10 Thread Saad Mufti
Although now that I think about this a bit more, all the failures we saw
were failure to obtain a row lock, and in the thread stack traces we always
saw it somewhere inside getRowLockInternal and similar. Never saw any
contention on bucket cache lock that I could see.

Cheers.


Saad


On Sat, Mar 10, 2018 at 8:04 PM, Saad Mufti  wrote:

> Also, for now we have mitigated this problem by using the new setting in
> HBase 1.4.0 that prevents one slow region server from blocking all client
> requests. Of course it causes some timeouts but our overall ecosystem
> contains Kafka queues for retries, so we can live with that. From what I
> can see, it looks like this setting also has the good effect of preventing
> clients from hammering a region server that is slow because its IPC queues
> are backed up, allowing it to recover faster.
>
> Does that make sense?
>
> Cheers.
>
> 
> Saad
>
>
> On Sat, Mar 10, 2018 at 7:04 PM, Saad Mufti  wrote:
>
>> So if I understand correctly, we would mitigate the problem by not
>> evicting blocks for archived files immediately? Wouldn't this potentially
>> lead to problems later if the LRU algo chooses to evict blocks for active
>> files and leave blocks for archived files in there?
>>
>> I would definitely love to test this!!! Unfortunately we are running on
>> EMR and the details of how to patch HBase under EMR are not clear to me :-(
>>
>> What we would really love would be a setting for actually immediately
>> caching blocks for a new compacted file. I have seen in the code that even
>> is we have the cache on write setting set to true, it will refuse to cache
>> blocks for a file that is a newly compacted one. In our case we have sized
>> the bucket cache to be big enough to hold all our data, and really want to
>> avoid having to go to S3 until the last possible moment. A config setting
>> to test this would be great.
>>
>> But thanks everyone for your feedback. Any more would also be welcome on
>> the idea to let a user cache all newly compacted files.
>>
>> 
>> Saad
>>
>>
>> On Wed, Mar 7, 2018 at 12:00 AM, Anoop John 
>> wrote:
>>
>>> >>a) it was indeed one of the regions that was being compacted, major
>>> compaction in one case, minor compaction in another, the issue started
>>> just
>>> after compaction completed blowing away bucket cached blocks for the
>>> older
>>> HFile's
>>>
>>> About this part.Ya after the compaction, there is a step where the
>>> compacted away HFile's blocks getting removed from cache. This op takes a
>>> write lock for this region (In Bucket Cache layer)..  Every read op which
>>> is part of checkAndPut will try read from BC and that in turn need a read
>>> lock for this region.  So there is chances that the read locks starve
>>> because of so many frequent write locks .  Each block evict will attain
>>> the
>>> write lock one after other.  Will it be possible for you to patch this
>>> evict and test once? We can avoid the immediate evict from BC after
>>> compaction. I can help you with a patch if you wish
>>>
>>> Anoop
>>>
>>>
>>>
>>> On Mon, Mar 5, 2018 at 11:07 AM, ramkrishna vasudevan <
>>> ramkrishna.s.vasude...@gmail.com> wrote:
>>> > Hi Saad
>>> >
>>> > Your argument here
>>> >>> The
>>> >>>theory is that since prefetch is an async operation, a lot of the
>>> reads
>>> in
>>> >>>the checkAndPut for the region in question start reading from S3
>>> which is
>>> >>>slow. So the write lock obtained for the checkAndPut is held for a
>>> longer
>>> >>>duration than normal. This has cascading upstream effects. Does that
>>> sound
>>> >>>plausible?
>>> >
>>> > Seems very much plausible. So before even the prefetch happens say for
>>> > 'block 1' - and you have already issues N checkAndPut calls for the
>>> rows
>>> in
>>> > that 'block 1' -  all those checkAndPut will have to read that block
>>> from
>>> > S3 to perform the get() and then apply the mutation.
>>> >
>>> > This may happen for multiple threads at the same time because we are
>>> not
>>> > sure when the prefetch would have actually been completed. I don know
>>> what
>>> > are the general read characteristics when a read happens from S3 but
>>> you
>>> > could try to see how things work when a read happens from S3 and after
>>> the
>>> > prefetch completes ensure the same checkandPut() is done (from cache
>>> this
>>> > time) to really know the difference what S3 does there.
>>> >
>>> > Regards
>>> > Ram
>>> >
>>> > On Fri, Mar 2, 2018 at 2:57 AM, Saad Mufti 
>>> wrote:
>>> >
>>> >> So after much investigation I can confirm:
>>> >>
>>> >> a) it was indeed one of the regions that was being compacted, major
>>> >> compaction in one case, minor compaction in another, the issue started
>>> just
>>> >> after compaction completed blowing away bucket cached blocks for the
>>> older
>>> >> HFile's
>>> >> b) in another case there was no compaction just a newly opened region
>>> in
>>> a
>>> >> region server that hadn't finished perfetching its pages from S3
>>> >>
>>> >> We

Re: HBase failed on local exception and failed servers list.

2018-03-10 Thread Saad Mufti
Are you using AuthUtil class to reauthenticate? This class is in Hbase, and
uses the Hadoop class UserGroupInformation to do the actual login and
re-login. But, if your UserGroupInformation class is from Hadoop 2.5.1 or
earlier, it has a bug if you are using Java 8, as most of us are. The
relogin code uses a test to decide whether the login is kerberos/keytab
based, and that test used to pass on Java 7 but fails in Java 8 because the
test tests for some specific class being in some underlying list of
kerberos objects assigned to your principal, which has disappeared in the
Java 8 implementation. We fixed this by upgrading our Hadoop dependency
explicitly to a newer version, in our case 2.6.1 and they have fixed this
problem in that newer version.

If this is the condition affecting your application, it is an easy enough
fix.

Hope this helps.

Cheers.


Saad



On Tue, Feb 27, 2018 at 1:16 PM, apratim sharma 
wrote:

> Hi Guys,
>
> I am using hbase 1.2.0 on a kerberos secured cloudera CDH 5.8 cluster.
> I have a persistant application that authenticates using keytab and creates
> hbase connection. Our code also takes care of reauthentication and
> recreating broken connectiion.
> The code worked fine in previous versions of hbase. However what we see
> with Hbase 1.2 is that after 24 hours the hbase connection does not work
> giving following error
>
> org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after
> attempts=2, exceptions:
> Tue Feb 13 12:57:51 PST 2018,
> RpcRetryingCaller{globalStartTime=1518555467140, pause=100, retries=2},
> org.apache.hadoop.hbase.exceptions.ConnectionClosingException: Call to
> pdmcdh01.xyz.com/192.168.145.62:60020 failed on local exception:
> org.apache.hadoop.hbase.exceptions.ConnectionClosingException: Connection
> to pdmcdh01.xyz.com/192.168.145.62:60020 is closing. Call id=137,
> waitTime=11
> Tue Feb 13 12:58:01 PST 2018,
> RpcRetryingCaller{globalStartTime=1518555467140, pause=100, retries=2},
> org.apache.hadoop.hbase.exceptions.ConnectionClosingException: Call to
> pdmcdh01.xyz.com/192.168.145.62:60020 failed on local exception:
> org.apache.hadoop.hbase.exceptions.ConnectionClosingException: Connection
> to pdmcdh01.xyz.com/192.168.145.62:60020 is closing. Call id=139,
> waitTime=13
>
> at
> org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(
> RpcRetryingCaller.java:147)
> at org.apache.hadoop.hbase.client.HTable.get(HTable.java:935)
> at org.apache.hadoop.hbase.client.HTable.get(HTable.java:901)
> Our code reauthnticates and creates connection again but it still keeps
> failing
> org.apache.hadoop.hbase.client.RetriesExhaustedException: Failed after
> attempts=2, exceptions:
> Wed Feb 21 14:30:31 PST 2018,
> RpcRetryingCaller{globalStartTime=1519252219159, pause=100, retries=2},
> java.io.IOException: Couldn't setup connection for p...@hadoop.xyz.com to
> hbase/pdmcdh01.xyz@hadoop.xyz.com
> Wed Feb 21 14:30:31 PST 2018,
> RpcRetryingCaller{globalStartTime=1519252219159, pause=100, retries=2},
> org.apache.hadoop.hbase.ipc.FailedServerException: This server is in the
> failed servers list: pdmcdh01.xyz.com/192.168.145.62:60020
>
> at
> org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(
> RpcRetryingCaller.java:147)
> at org.apache.hadoop.hbase.client.HTable.get(HTable.java:935)
> at org.apache.hadoop.hbase.client.HTable.get(HTable.java:901)
> I know that client keeps server in the failed list for few seconds in order
> to reduce too many connection attempts. So I waited and tried after some
> time but still same error.
> Once we restart our application everything starts working fine again for
> next 24 hours.
>
> This 24 hours gap indicates that it could be something related to Kerberos
> ticket expiry time, however there is no log to indicate Kerberos
> authentication issue.
> Moreover we are handling the exception and trying to authenticate and
> create connection again but nothing works until we restart JVM. this is
> very strange.
>
> I would really appreciate any help or pointers on this issue.
>
> Thanks a lot
> Apratim
>


Re: How Long Will HBase Hold A Row Write Lock?

2018-03-10 Thread Saad Mufti
Also, for now we have mitigated this problem by using the new setting in
HBase 1.4.0 that prevents one slow region server from blocking all client
requests. Of course it causes some timeouts but our overall ecosystem
contains Kafka queues for retries, so we can live with that. From what I
can see, it looks like this setting also has the good effect of preventing
clients from hammering a region server that is slow because its IPC queues
are backed up, allowing it to recover faster.

Does that make sense?

Cheers.


Saad


On Sat, Mar 10, 2018 at 7:04 PM, Saad Mufti  wrote:

> So if I understand correctly, we would mitigate the problem by not
> evicting blocks for archived files immediately? Wouldn't this potentially
> lead to problems later if the LRU algo chooses to evict blocks for active
> files and leave blocks for archived files in there?
>
> I would definitely love to test this!!! Unfortunately we are running on
> EMR and the details of how to patch HBase under EMR are not clear to me :-(
>
> What we would really love would be a setting for actually immediately
> caching blocks for a new compacted file. I have seen in the code that even
> is we have the cache on write setting set to true, it will refuse to cache
> blocks for a file that is a newly compacted one. In our case we have sized
> the bucket cache to be big enough to hold all our data, and really want to
> avoid having to go to S3 until the last possible moment. A config setting
> to test this would be great.
>
> But thanks everyone for your feedback. Any more would also be welcome on
> the idea to let a user cache all newly compacted files.
>
> 
> Saad
>
>
> On Wed, Mar 7, 2018 at 12:00 AM, Anoop John  wrote:
>
>> >>a) it was indeed one of the regions that was being compacted, major
>> compaction in one case, minor compaction in another, the issue started
>> just
>> after compaction completed blowing away bucket cached blocks for the older
>> HFile's
>>
>> About this part.Ya after the compaction, there is a step where the
>> compacted away HFile's blocks getting removed from cache. This op takes a
>> write lock for this region (In Bucket Cache layer)..  Every read op which
>> is part of checkAndPut will try read from BC and that in turn need a read
>> lock for this region.  So there is chances that the read locks starve
>> because of so many frequent write locks .  Each block evict will attain
>> the
>> write lock one after other.  Will it be possible for you to patch this
>> evict and test once? We can avoid the immediate evict from BC after
>> compaction. I can help you with a patch if you wish
>>
>> Anoop
>>
>>
>>
>> On Mon, Mar 5, 2018 at 11:07 AM, ramkrishna vasudevan <
>> ramkrishna.s.vasude...@gmail.com> wrote:
>> > Hi Saad
>> >
>> > Your argument here
>> >>> The
>> >>>theory is that since prefetch is an async operation, a lot of the reads
>> in
>> >>>the checkAndPut for the region in question start reading from S3 which
>> is
>> >>>slow. So the write lock obtained for the checkAndPut is held for a
>> longer
>> >>>duration than normal. This has cascading upstream effects. Does that
>> sound
>> >>>plausible?
>> >
>> > Seems very much plausible. So before even the prefetch happens say for
>> > 'block 1' - and you have already issues N checkAndPut calls for the rows
>> in
>> > that 'block 1' -  all those checkAndPut will have to read that block
>> from
>> > S3 to perform the get() and then apply the mutation.
>> >
>> > This may happen for multiple threads at the same time because we are not
>> > sure when the prefetch would have actually been completed. I don know
>> what
>> > are the general read characteristics when a read happens from S3 but you
>> > could try to see how things work when a read happens from S3 and after
>> the
>> > prefetch completes ensure the same checkandPut() is done (from cache
>> this
>> > time) to really know the difference what S3 does there.
>> >
>> > Regards
>> > Ram
>> >
>> > On Fri, Mar 2, 2018 at 2:57 AM, Saad Mufti 
>> wrote:
>> >
>> >> So after much investigation I can confirm:
>> >>
>> >> a) it was indeed one of the regions that was being compacted, major
>> >> compaction in one case, minor compaction in another, the issue started
>> just
>> >> after compaction completed blowing away bucket cached blocks for the
>> older
>> >> HFile's
>> >> b) in another case there was no compaction just a newly opened region
>> in
>> a
>> >> region server that hadn't finished perfetching its pages from S3
>> >>
>> >> We have prefetch on open set to true. Our load is heavy on checkAndPut
>> .The
>> >> theory is that since prefetch is an async operation, a lot of the reads
>> in
>> >> the checkAndPut for the region in question start reading from S3 which
>> is
>> >> slow. So the write lock obtained for the checkAndPut is held for a
>> longer
>> >> duration than normal. This has cascading upstream effects. Does that
>> sound
>> >> plausible?
>> >>
>> >> The part I don't understand still is all the locks held 

TableSnapshotInputFormat Behavior In HBase 1.4.0

2018-03-10 Thread Saad Mufti
Hi,

I am running a Spark job (Spark 2.2.1) on an EMR cluster in AWS. There is
no Hbase installed on the cluster, only HBase libs linked to my Spark app.
We are reading the snapshot info from a HBase folder in S3 using
TableSnapshotInputFormat class from HBase 1.4.0 to have the Spark job read
snapshot info directly from the S3 based filesystem instead of going
through any region server.

I have observed a few behaviors while debugging performance that are
concerning, some we could mitigate and other I am looking for clarity on:

1)  the TableSnapshotInputFormatImpl code is trying to get locality
information for the region splits, for a snapshots with a large number of
files (over 35 in our case) this causing single threaded scan of all
the file listings in a single thread in the driver. And it was useless
because there is really no useful locality information to glean since all
the files are in S3 and not HDFS. So I was forced to make a copy of
TableSnapshotInputFormatImpl.java in our code and control this with a
config setting I made up. That got rid of the hours long scan, so I am good
with this part for now.

2) I have set a single column family in the Scan that I set on the hbase
configuration via

scan.addFamily(str.getBytes()))

hBaseConf.set(TableInputFormat.SCAN, convertScanToString(scan))


But when this code is executing under Spark and I observe the threads and
logs on Spark executors, I it is reading from S3 files for a column family
that was not included in the scan. This column family was intentionally
excluded because it is much larger than the others and so we wanted to
avoid the cost.

Any advice on what I am doing wrong would be appreciated.

3) We also explicitly set caching of blocks to false on the scan, although
I see that in TableSnapshotInputFormatImpl.java it is again set to false
internally also. But when running the Spark job, some executors were taking
much longer than others, and when I observe their threads, I see periodic
messages about a few hundred megs of RAM used by the block cache, and the
thread is sitting there reading data from S3, and is occasionally blocked a
couple of other threads that have the "hfile-prefetcher" name in them.
Going back to 2) above, they seem to be reading the wrong column family,
but in this item I am more concerned about why they appear to be
prefetching blocks and caching them, when the Scan object has a setting to
not cache blocks at all?

Thanks in advance for any insights anyone can provide.


Saad


Re: How Long Will HBase Hold A Row Write Lock?

2018-03-10 Thread Saad Mufti
So if I understand correctly, we would mitigate the problem by not evicting
blocks for archived files immediately? Wouldn't this potentially lead to
problems later if the LRU algo chooses to evict blocks for active files and
leave blocks for archived files in there?

I would definitely love to test this!!! Unfortunately we are running on EMR
and the details of how to patch HBase under EMR are not clear to me :-(

What we would really love would be a setting for actually immediately
caching blocks for a new compacted file. I have seen in the code that even
is we have the cache on write setting set to true, it will refuse to cache
blocks for a file that is a newly compacted one. In our case we have sized
the bucket cache to be big enough to hold all our data, and really want to
avoid having to go to S3 until the last possible moment. A config setting
to test this would be great.

But thanks everyone for your feedback. Any more would also be welcome on
the idea to let a user cache all newly compacted files.


Saad


On Wed, Mar 7, 2018 at 12:00 AM, Anoop John  wrote:

> >>a) it was indeed one of the regions that was being compacted, major
> compaction in one case, minor compaction in another, the issue started just
> after compaction completed blowing away bucket cached blocks for the older
> HFile's
>
> About this part.Ya after the compaction, there is a step where the
> compacted away HFile's blocks getting removed from cache. This op takes a
> write lock for this region (In Bucket Cache layer)..  Every read op which
> is part of checkAndPut will try read from BC and that in turn need a read
> lock for this region.  So there is chances that the read locks starve
> because of so many frequent write locks .  Each block evict will attain the
> write lock one after other.  Will it be possible for you to patch this
> evict and test once? We can avoid the immediate evict from BC after
> compaction. I can help you with a patch if you wish
>
> Anoop
>
>
>
> On Mon, Mar 5, 2018 at 11:07 AM, ramkrishna vasudevan <
> ramkrishna.s.vasude...@gmail.com> wrote:
> > Hi Saad
> >
> > Your argument here
> >>> The
> >>>theory is that since prefetch is an async operation, a lot of the reads
> in
> >>>the checkAndPut for the region in question start reading from S3 which
> is
> >>>slow. So the write lock obtained for the checkAndPut is held for a
> longer
> >>>duration than normal. This has cascading upstream effects. Does that
> sound
> >>>plausible?
> >
> > Seems very much plausible. So before even the prefetch happens say for
> > 'block 1' - and you have already issues N checkAndPut calls for the rows
> in
> > that 'block 1' -  all those checkAndPut will have to read that block from
> > S3 to perform the get() and then apply the mutation.
> >
> > This may happen for multiple threads at the same time because we are not
> > sure when the prefetch would have actually been completed. I don know
> what
> > are the general read characteristics when a read happens from S3 but you
> > could try to see how things work when a read happens from S3 and after
> the
> > prefetch completes ensure the same checkandPut() is done (from cache this
> > time) to really know the difference what S3 does there.
> >
> > Regards
> > Ram
> >
> > On Fri, Mar 2, 2018 at 2:57 AM, Saad Mufti  wrote:
> >
> >> So after much investigation I can confirm:
> >>
> >> a) it was indeed one of the regions that was being compacted, major
> >> compaction in one case, minor compaction in another, the issue started
> just
> >> after compaction completed blowing away bucket cached blocks for the
> older
> >> HFile's
> >> b) in another case there was no compaction just a newly opened region in
> a
> >> region server that hadn't finished perfetching its pages from S3
> >>
> >> We have prefetch on open set to true. Our load is heavy on checkAndPut
> .The
> >> theory is that since prefetch is an async operation, a lot of the reads
> in
> >> the checkAndPut for the region in question start reading from S3 which
> is
> >> slow. So the write lock obtained for the checkAndPut is held for a
> longer
> >> duration than normal. This has cascading upstream effects. Does that
> sound
> >> plausible?
> >>
> >> The part I don't understand still is all the locks held are for the same
> >> region but are all for different rows. So once the prefetch is
> completed,
> >> shouldn't the problem clear up quickly? Or does the slow region slow
> down
> >> anyone trying to do checkAndPut on any row in the same region even after
> >> the prefetch has completed. That is, do the long held row locks prevent
> >> others from getting a row lock on a different row in the same region?
> >>
> >> In any case, we trying to use
> >> https://issues.apache.org/jira/browse/HBASE-16388 support in HBase
> 1.4.0
> >> to
> >> both insulate the app a bit from this situation and hoping that it will
> >> reduce pressure on the region server in question, allowing it to recover
> >> faster. I haven't quite test