Uploaded image for project: 'HBase'
  1. HBase
  2. HBASE-23076

[HBOSS] ZKTreeLockManager shouldn't try to acquire a lock from the InterProcessMutex instance when checking if other processes hold it.

    XMLWordPrintableJSON

Details

    • Improvement
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • hbase-filesystem-1.0.0-alpha1
    • None
    • None

    Description

      While going through some internal tests, our elserj had faced major bottleneck problems when creating tables with reasonable large number of pre-split tables:

       create 'josh', 'f1', {SPLITS=> (1..500).map {|i| "user#{1000+i*(9999-1000)/500}"}}
      

      The above resulted in RSes taking long time to complete all assignments, leading to APs timeout failures from master point of view, which in turn submits further APs, in a cascade fashion, until RSes RPC queues got flood and started throwing CallQueueFullException, leaving Master with loads of procedures to complete and many RITs.

      Jstack analysis pointed to potential lock contentions inside ZKTreeLockManager.isLocked method. To quote elserj  report:

      Specifically, lots of threads that look like this:

      "RpcServer.priority.FPBQ.Fifo.handler=8,queue=0,port=16020" #100 daemon prio=5 os_prio=0 tid=0x00007f5d6dc3a000 nid=0x6b1 waiting for monitor entry [0x00007f5d3bafb000]
         java.lang.Thread.State: BLOCKED (on object monitor)
      	at org.apache.hadoop.hbase.oss.thirdparty.org.apache.curator.framework.recipes.locks.LockInternals.internalLockLoop(LockInternals.java:289)
      	- waiting to lock <0x000000074ddd0d10> (a org.apache.hadoop.hbase.oss.thirdparty.org.apache.curator.framework.recipes.locks.LockInternals)
      	at org.apache.hadoop.hbase.oss.thirdparty.org.apache.curator.framework.recipes.locks.LockInternals.attemptLock(LockInternals.java:219)
      	at org.apache.hadoop.hbase.oss.thirdparty.org.apache.curator.framework.recipes.locks.InterProcessMutex.internalLock(InterProcessMutex.java:237)
      	at org.apache.hadoop.hbase.oss.thirdparty.org.apache.curator.framework.recipes.locks.InterProcessMutex.acquire(InterProcessMutex.java:108)
      	at org.apache.hadoop.hbase.oss.sync.ZKTreeLockManager.isLocked(ZKTreeLockManager.java:310)
      	at org.apache.hadoop.hbase.oss.sync.ZKTreeLockManager.writeLockAbove(ZKTreeLockManager.java:183)
      	at org.apache.hadoop.hbase.oss.sync.TreeLockManager.treeReadLock(TreeLockManager.java:282)
      	at org.apache.hadoop.hbase.oss.sync.TreeLockManager.lock(TreeLockManager.java:449)
      	at org.apache.hadoop.hbase.oss.HBaseObjectStoreSemantics.open(HBaseObjectStoreSemantics.java:181)
      	at org.apache.hadoop.fs.FilterFileSystem.open(FilterFileSystem.java:166)
      	at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:911)
      	at org.apache.hadoop.hbase.util.FSTableDescriptors.readTableDescriptor(FSTableDescriptors.java:566)
      	at org.apache.hadoop.hbase.util.FSTableDescriptors.getTableDescriptorFromFs(FSTableDescriptors.java:559)
      	at org.apache.hadoop.hbase.util.FSTableDescriptors.getTableDescriptorFromFs(FSTableDescriptors.java:545)
      	at org.apache.hadoop.hbase.util.FSTableDescriptors.get(FSTableDescriptors.java:241)
      	at org.apache.hadoop.hbase.regionserver.RSRpcServices.executeOpenRegionProcedures(RSRpcServices.java:3626)
      	at org.apache.hadoop.hbase.regionserver.RSRpcServices.lambda$executeProcedures$2(RSRpcServices.java:3694)
      	at org.apache.hadoop.hbase.regionserver.RSRpcServices$$Lambda$107/1985163471.accept(Unknown Source)
      	at java.util.ArrayList.forEach(ArrayList.java:1257)
      	at java.util.Collections$UnmodifiableCollection.forEach(Collections.java:1080)
      	at org.apache.hadoop.hbase.regionserver.RSRpcServices.executeProcedures(RSRpcServices.java:3694)
      	at org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos$AdminService$2.callBlockingMethod(AdminProtos.java:29774)
      	at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:413)
      	at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:132)
      	at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:338)
      	at org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run(RpcExecutor.java:318)
      
         Locked ownable synchronizers:
      	- None
      

      This means that we can only open Regions in a table one at a time now (across all regionservers). That's pretty bad and would explain why that part was so slow.

      Two thoughts already:

      1) Having to grab the lock to determine if it's held is sub-optimal. That's what the top of this stacktrace is and I think we need to come up with some other approach because this doesn't scale.
      2) We're all blocked in reading the TableDescriptor. Maybe the Master can include the TableDescriptor in the OpenRegionRequest so the RS's don't have to read it back?

      From elserj suggestion above, #2 would require changes at hbase project side, but we still can try optmize hboss ZKTreeLockManager.isLocked method as mentioned in #1.

      Looking at curator's InterProcessMutex, we can use its getParticipantNodes() method for checking if there's any process locking the given node.

      Attachments

        Activity

          People

            wchevreuil Wellington Chevreuil
            wchevreuil Wellington Chevreuil
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: