[ 
https://issues.apache.org/jira/browse/HBASE-21428?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16689264#comment-16689264
 ] 

Yu Li commented on HBASE-21428:
-------------------------------

Thanks for drawing my attention [~reidchan].

As mentioned in HBASE-19260, allowing parallel accessing meta to locate region 
is dangerous and a behavior change comparing 0.98, so we regarded it as a 
regression issue.

While indeed we neglected the already existing 
{{hbase.hconnection.meta.lookup.threads.max/core}} configurations and maybe it 
could be a better idea to remove the lock and set the default settings of these 
two properties to 1, so advanced users could increase the settings (although 
not recommended) when they're sure their parallel access won't drive the 
PriorityHandlers mad.

And I don't think we need a group of locks when tuning the configuration is 
enough to control the parallelism.

> Performance issue due to userRegionLock in the ConnectionManager.
> -----------------------------------------------------------------
>
>                 Key: HBASE-21428
>                 URL: https://issues.apache.org/jira/browse/HBASE-21428
>             Project: HBase
>          Issue Type: Bug
>    Affects Versions: 1.2.7
>            Reporter: koo
>            Priority: Major
>
> My service is that execute a lot of puts using HTableMultiplexer.
> After the version change, most of the requests are rejected.
> It works fine in 1.2.6.1, but there is a problem in 1.2.7.
> This issue is related with the HBASE-19260.
> Most of my threads are using a lot of time as below.
>  
> |"Worker-972" #2479 daemon prio=5 os_prio=0 tid=0x00007f8cea86b000 nid=0x4c8c 
> waiting on condition [0x00007f8b78104000]
>  java.lang.Thread.State: WAITING (parking)
>  at sun.misc.Unsafe.park(Native Method)
>  - parking to wait for <0x00000005dd703b78> (a 
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
>  at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>  at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
>  at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:870)
>  at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1199)
>  at 
> java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:209)
>  at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:285)
>  at 
> org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegionInMeta(ConnectionManager.java:1274)
>  at 
> org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegion(ConnectionManager.java:1186)
>  at 
> org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegion(ConnectionManager.java:1170)
>  at 
> org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.locateRegion(ConnectionManager.java:1127)
>  at 
> org.apache.hadoop.hbase.client.ConnectionManager$HConnectionImplementation.getRegionLocation(ConnectionManager.java:962)
>  at 
> org.apache.hadoop.hbase.client.HTableMultiplexer.put(HTableMultiplexer.java:206)
>  at 
> org.apache.hadoop.hbase.client.HTableMultiplexer.put(HTableMultiplexer.java:150)|
>  
> When I looked at the issue(HBASE-19260), I recognized the dangerous of to 
> allow accessessing multiple threads.
> However, Already create many threads with the limitations
> I think it is very inefficient to allow only one thread access.
>  
> | this.metaLookupPool = getThreadPool(
>  conf.getInt("hbase.hconnection.meta.lookup.threads.max", 128),
>  conf.getInt("hbase.hconnection.meta.lookup.threads.core", 10),
>  "-metaLookup-shared-", new LinkedBlockingQueue<Runnable>());|
>  
> I want to suggest changing it that allow to have multiple locks.(but not the 
> entire thread)
> The following is pseudocode.
>  
> |int lockSize = conf.getInt("hbase.hconnection.meta.lookup.threads.max", 128) 
> / 2;
> BlockingQueue<ReentrantLock> userRegionLockQueue = new 
> LinkedBlockingQueue<ReentrantLock>();
>  for (int i=0; i <lockSize; i++) {
>  userRegionLockQueue.put(new ReentrantLock());
>  }|
>  
> thanks.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to