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

Mike Drob commented on HBASE-18628:
-----------------------------------

No, I was not able to figure out the root cause. Executor _shouldn't_ be shut 
down because we can still see it's thread in the jstack, I thought.

The childrenChangedFuture does not need to be volatile because it is always 
read from the same thread - the ZK EventThread.

There _is_ a guarantee that there will be no more than one refreshNodes action, 
because they are processed by a single threaded executor (and all the ZK 
notifications also come in through a single thread). So there will be at most 
one action happening, and one new action coming in. If we always cancel the 
preceding action, then we can maintain this invariant.

The original reason for preemption was performance - either you or Ted claimed 
that updating permissions on a large cluster can take several minutes, and if 
multiple updates come in close proximity then there is no reason to process the 
stale data. I'm happy to see this code removed because I agree that it is 
complicated and brittle, but I did manage to run this against the one cluster 
I've seen the problem on and the issue went away (without other issues 
appearing). No promises that my new code is foolproof, but I feel good about it.

> ZKPermissionWatcher blocks all ZK notifications
> -----------------------------------------------
>
>                 Key: HBASE-18628
>                 URL: https://issues.apache.org/jira/browse/HBASE-18628
>             Project: HBase
>          Issue Type: Bug
>          Components: regionserver
>            Reporter: Mike Drob
>            Assignee: Mike Drob
>            Priority: Critical
>             Fix For: 3.0.0, 2.0.0-alpha-3
>
>         Attachments: HBASE-18628.branch-1.v5.patch, HBASE-18628.patch, 
> HBASE-18628.v2.patch, HBASE-18628.v3.patch, HBASE-18628.v4.patch, 
> HBASE-18628.v5.patch, jstack
>
>
> Buckle up folks, we're going for a ride here. I've seeing this on a branch-2 
> based build, but I think the problem will affect branch-1 as well. I'm not 
> able to easily reproduce the issue, but it will usually come up within an 
> hour on a given cluster that I have, at which point the problem persists 
> until an RS restart. I've been seeing the problem and paying attention for 
> maybe two months, but I suspect it's been happening much longer than that.
> h3. Problem
> When running in a secure cluster, sometimes the ZK EventThread will get stuck 
> on a permissions update and not be able to process new notifications. This 
> happens to also block flush and snapshot, which is how we found it.
> h3. Analysis
> The main smoking gun is seeing this in repeated jstacks:
> {noformat}
> "main-EventThread" #43 daemon prio=5 os_prio=0 tid=0x00007f0b92644000 
> nid=0x6e69 waiting on condition [0x00007f0b6730f000]
>    java.lang.Thread.State: TIMED_WAITING (sleeping)
>         at java.lang.Thread.sleep(Native Method)
>         at 
> org.apache.hadoop.hbase.security.access.ZKPermissionWatcher.nodeChildrenChanged(ZKPermissionWatcher.java:191)
>         at 
> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher.process(ZooKeeperWatcher.java:503)
>         at 
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:522)
>         at 
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
> {noformat}
> That sleep is a 20ms sleep in an {{AtomicReference.compareAndSet}} loop - but 
> it never gets past the condition.
> {code}
>         while (!nodes.compareAndSet(null, nodeList)) {
>           try {
>             Thread.sleep(20);
>           } catch (InterruptedException e) {
>             LOG.warn("Interrupted while setting node list", e);
>             Thread.currentThread().interrupt();
>           }
>         }
> {code}
> The warning never shows up in the logs, it just keeps looping and looping. 
> The last relevant line from the watcher in logs is:
> {noformat}
> 2017-08-17 21:25:12,379 DEBUG 
> org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher: 
> regionserver:22101-0x15df38884c80024, quorum=zk1:2181,zk2:2181,zk3:2181, 
> baseZNode=/hbase Received ZooKeeper Event, type=NodeChildrenChanged, 
> state=SyncConnected, path=/hbase/acl
> {noformat}
> Which makes sense, because the code snippet is from permission watcher's 
> {{nodeChildrenChanged}} handler.
> The separate thread introduced in HBASE-14370 is present, but not doing 
> anything. And this event hasn't gotten to the part where it splits off into a 
> thread:
> {noformat}
> "zk-permission-watcher4-thread-1" #160 daemon prio=5 os_prio=0 
> tid=0x0000000001750800 nid=0x6fd9 waiting on condition [0x00007f0b5dce5000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00000007436ecea0> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>         at 
> java.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)
>         at 
> java.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> {noformat}
> h3. Solutions
> There's a few approaches we can take to fix this, I think they are all 
> complimentary. It might be useful to file subtasks or new issues for some of 
> the solutions if they are longer term.
> # Move flush and snapshot to ProcedureV2. This makes my proximate problem go 
> away, but it's only relevant to branch-2 and master, and doesn't fix anything 
> on branch-1. Also, Permissions updates would still get stuck, preventing 
> future permissions updates. I think this is important long term for the 
> robustness of the system, but not a viable short term fix.
> # Add an Executor to ZookeeperWatcher and launch threads from there. Maybe 
> we'd want to pull the Executor out of ZKPW, but that's not strictly necessary 
> and can be optimized later -- if we're already threading, then adding another 
> layer isn't a huge cost.
> # Figure out the race condition or logic problem that causes {{nodes}} to be 
> non-null above. I've tried looking at this and visual inspection isn't 
> getting me anywhere.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to