1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.zookeeper.lock;
21
22 import java.io.IOException;
23 import java.util.List;
24 import java.util.TreeSet;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.classification.InterfaceAudience;
29 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
30 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
31 import org.apache.zookeeper.KeeperException;
32
33
34
35
36 @InterfaceAudience.Private
37 public class ZKInterProcessWriteLock extends ZKInterProcessLockBase {
38
39 private static final Log LOG = LogFactory.getLog(ZKInterProcessWriteLock.class);
40
41 public ZKInterProcessWriteLock(ZooKeeperWatcher zooKeeperWatcher,
42 String znode, byte[] metadata, MetadataHandler handler) {
43 super(zooKeeperWatcher, znode, metadata, handler, WRITE_LOCK_CHILD_NODE_PREFIX);
44 }
45
46
47
48
49 @Override
50 protected String getLockPath(String createdZNode, List<String> children)
51 throws IOException, InterruptedException {
52 TreeSet<String> sortedChildren =
53 new TreeSet<String>(ZNodeComparator.COMPARATOR);
54 sortedChildren.addAll(children);
55 String pathToWatch = sortedChildren.lower(createdZNode);
56 if (pathToWatch != null) {
57 String nodeHoldingLock = sortedChildren.first();
58 String znode = ZKUtil.joinZNode(parentLockNode, nodeHoldingLock);
59 try {
60 handleLockMetadata(znode);
61 } catch (IOException e) {
62 LOG.warn("Error processing lock metadata in " + nodeHoldingLock, e);
63 }
64 }
65 return pathToWatch;
66 }
67
68
69
70
71
72 public void reapAllLocks() throws IOException {
73 List<String> children;
74 try {
75 children = ZKUtil.listChildrenNoWatch(zkWatcher, parentLockNode);
76 } catch (KeeperException e) {
77 LOG.error("Unexpected ZooKeeper error when listing children", e);
78 throw new IOException("Unexpected ZooKeeper exception", e);
79 }
80
81 KeeperException deferred = null;
82 for (String child : children) {
83 if (isChildWriteLock(child)) {
84 String znode = ZKUtil.joinZNode(parentLockNode, child);
85 LOG.info("Reaping write lock for znode:" + znode);
86 try {
87 ZKUtil.deleteNodeFailSilent(zkWatcher, znode);
88 } catch (KeeperException ex) {
89 LOG.warn("Error reaping the znode for write lock :" + znode);
90 deferred = ex;
91 }
92 }
93 }
94 if (deferred != null) {
95 throw new IOException("ZK exception while reaping locks:", deferred);
96 }
97 }
98 }