1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.zookeeper;
20
21 import org.apache.commons.logging.Log;
22 import org.apache.commons.logging.LogFactory;
23 import org.apache.hadoop.classification.InterfaceAudience;
24 import org.apache.hadoop.classification.InterfaceStability;
25 import org.apache.hadoop.hbase.Abortable;
26 import org.apache.zookeeper.KeeperException;
27
28
29
30
31
32
33
34
35
36
37 @InterfaceAudience.Public
38 @InterfaceStability.Evolving
39 public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
40
41 static final Log LOG = LogFactory.getLog(ZooKeeperNodeTracker.class);
42
43 protected final String node;
44
45
46 private byte [] data;
47
48
49 protected final Abortable abortable;
50
51 private boolean stopped = false;
52
53
54
55
56
57
58
59
60
61
62 public ZooKeeperNodeTracker(ZooKeeperWatcher watcher, String node,
63 Abortable abortable) {
64 super(watcher);
65 this.node = node;
66 this.abortable = abortable;
67 this.data = null;
68 }
69
70
71
72
73
74
75
76 public synchronized void start() {
77 this.watcher.registerListener(this);
78 try {
79 if(ZKUtil.watchAndCheckExists(watcher, node)) {
80 byte [] data = ZKUtil.getDataAndWatch(watcher, node);
81 if(data != null) {
82 this.data = data;
83 } else {
84
85 LOG.debug("Try starting again because there is no data from " + node);
86 start();
87 }
88 }
89 } catch (KeeperException e) {
90 abortable.abort("Unexpected exception during initialization, aborting", e);
91 }
92 }
93
94 public synchronized void stop() {
95 this.stopped = true;
96 notifyAll();
97 }
98
99
100
101
102
103
104
105 public synchronized byte [] blockUntilAvailable()
106 throws InterruptedException {
107 return blockUntilAvailable(0, false);
108 }
109
110
111
112
113
114
115
116
117
118
119 public synchronized byte [] blockUntilAvailable(long timeout, boolean refresh)
120 throws InterruptedException {
121 if (timeout < 0) throw new IllegalArgumentException();
122 boolean notimeout = timeout == 0;
123 long startTime = System.currentTimeMillis();
124 long remaining = timeout;
125 if (refresh) {
126 try {
127
128 this.data = ZKUtil.getDataAndWatch(watcher, node);
129 } catch(KeeperException e) {
130
131
132 LOG.warn("Unexpected exception handling blockUntilAvailable", e);
133 abortable.abort("Unexpected exception handling blockUntilAvailable", e);
134 }
135 }
136 boolean nodeExistsChecked = (!refresh ||data!=null);
137 while (!this.stopped && (notimeout || remaining > 0) && this.data == null) {
138 if (!nodeExistsChecked) {
139 try {
140 nodeExistsChecked = (ZKUtil.checkExists(watcher, node) != -1);
141 } catch (KeeperException e) {
142 LOG.warn(
143 "Got exception while trying to check existence in ZooKeeper" +
144 " of the node: "+node+", retrying if timeout not reached",e );
145 }
146
147
148 if (nodeExistsChecked){
149 LOG.info("Node "+node+" now exists, resetting a watcher");
150 try {
151
152 this.data = ZKUtil.getDataAndWatch(watcher, node);
153 } catch (KeeperException e) {
154 LOG.warn("Unexpected exception handling blockUntilAvailable", e);
155 abortable.abort("Unexpected exception handling blockUntilAvailable", e);
156 }
157 }
158 }
159
160
161 wait(100);
162 remaining = timeout - (System.currentTimeMillis() - startTime);
163 }
164 return this.data;
165 }
166
167
168
169
170
171
172
173
174
175
176 public synchronized byte [] getData(boolean refresh) {
177 if (refresh) {
178 try {
179 this.data = ZKUtil.getDataAndWatch(watcher, node);
180 } catch(KeeperException e) {
181 abortable.abort("Unexpected exception handling getData", e);
182 }
183 }
184 return this.data;
185 }
186
187 public String getNode() {
188 return this.node;
189 }
190
191 @Override
192 public synchronized void nodeCreated(String path) {
193 if (!path.equals(node)) return;
194 try {
195 byte [] data = ZKUtil.getDataAndWatch(watcher, node);
196 if (data != null) {
197 this.data = data;
198 notifyAll();
199 } else {
200 nodeDeleted(path);
201 }
202 } catch(KeeperException e) {
203 abortable.abort("Unexpected exception handling nodeCreated event", e);
204 }
205 }
206
207 @Override
208 public synchronized void nodeDeleted(String path) {
209 if(path.equals(node)) {
210 try {
211 if(ZKUtil.watchAndCheckExists(watcher, node)) {
212 nodeCreated(path);
213 } else {
214 this.data = null;
215 }
216 } catch(KeeperException e) {
217 abortable.abort("Unexpected exception handling nodeDeleted event", e);
218 }
219 }
220 }
221
222 @Override
223 public synchronized void nodeDataChanged(String path) {
224 if(path.equals(node)) {
225 nodeCreated(path);
226 }
227 }
228
229
230
231
232
233
234
235 public boolean checkIfBaseNodeAvailable() {
236 try {
237 if (ZKUtil.checkExists(watcher, watcher.baseZNode) == -1) {
238 return false;
239 }
240 } catch (KeeperException e) {
241 abortable
242 .abort(
243 "Exception while checking if basenode ("+watcher.baseZNode+
244 ") exists in ZooKeeper.",
245 e);
246 }
247 return true;
248 }
249 }