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