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 com.google.protobuf.InvalidProtocolBufferException;
22 import org.apache.commons.logging.Log;
23 import org.apache.commons.logging.LogFactory;
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.CoordinatedStateException;
26 import org.apache.hadoop.hbase.TableName;
27 import org.apache.hadoop.hbase.TableStateManager;
28 import org.apache.hadoop.hbase.exceptions.DeserializationException;
29 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
30 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
31 import org.apache.zookeeper.KeeperException;
32
33 import java.io.InterruptedIOException;
34 import java.util.Arrays;
35 import java.util.HashMap;
36 import java.util.HashSet;
37 import java.util.List;
38 import java.util.Map;
39 import java.util.Set;
40
41
42
43
44
45
46
47
48
49
50 @InterfaceAudience.Private
51 public class ZKTableStateManager implements TableStateManager {
52
53
54
55
56
57 private static final Log LOG = LogFactory.getLog(ZKTableStateManager.class);
58 private final ZooKeeperWatcher watcher;
59
60
61
62
63
64
65 private final Map<TableName, ZooKeeperProtos.Table.State> cache =
66 new HashMap<TableName, ZooKeeperProtos.Table.State>();
67
68 public ZKTableStateManager(final ZooKeeperWatcher zkw) throws KeeperException,
69 InterruptedException {
70 super();
71 this.watcher = zkw;
72 populateTableStates();
73 }
74
75
76
77
78
79 private void populateTableStates() throws KeeperException, InterruptedException {
80 synchronized (this.cache) {
81 List<String> children = ZKUtil.listChildrenNoWatch(this.watcher, this.watcher.tableZNode);
82 if (children == null) return;
83 for (String child: children) {
84 TableName tableName = TableName.valueOf(child);
85 ZooKeeperProtos.Table.State state = getTableState(this.watcher, tableName);
86 if (state != null) this.cache.put(tableName, state);
87 }
88 }
89 }
90
91
92
93
94
95
96 @Override
97 public void setTableState(TableName tableName, ZooKeeperProtos.Table.State state)
98 throws CoordinatedStateException {
99 synchronized (this.cache) {
100 LOG.warn("Moving table " + tableName + " state from " + this.cache.get(tableName)
101 + " to " + state);
102 try {
103 setTableStateInZK(tableName, state);
104 } catch (KeeperException e) {
105 throw new CoordinatedStateException(e);
106 }
107 }
108 }
109
110
111
112
113
114 @Override
115 public boolean setTableStateIfInStates(TableName tableName,
116 ZooKeeperProtos.Table.State newState,
117 ZooKeeperProtos.Table.State... states)
118 throws CoordinatedStateException {
119 synchronized (this.cache) {
120
121
122 if (
123 (newState == ZooKeeperProtos.Table.State.DISABLING) &&
124 this.cache.get(tableName) != null && !isTableState(tableName, states) ||
125 (newState != ZooKeeperProtos.Table.State.DISABLING &&
126 !isTableState(tableName, states) )) {
127 return false;
128 }
129 try {
130 setTableStateInZK(tableName, newState);
131 } catch (KeeperException e) {
132 throw new CoordinatedStateException(e);
133 }
134 return true;
135 }
136 }
137
138
139
140
141
142 @Override
143 public boolean setTableStateIfNotInStates(TableName tableName,
144 ZooKeeperProtos.Table.State newState,
145 ZooKeeperProtos.Table.State... states)
146 throws CoordinatedStateException {
147 synchronized (this.cache) {
148 if (isTableState(tableName, states)) {
149
150
151
152
153
154
155 try {
156 ZooKeeperProtos.Table.State curstate = getTableState(watcher, tableName);
157
158 if (isTableInState(Arrays.asList(states), curstate)) {
159 return false;
160 }
161 } catch (KeeperException e) {
162 throw new CoordinatedStateException(e);
163 } catch (InterruptedException e) {
164 throw new CoordinatedStateException(e);
165 }
166 }
167 try {
168 setTableStateInZK(tableName, newState);
169 } catch (KeeperException e) {
170 throw new CoordinatedStateException(e);
171 }
172 return true;
173 }
174 }
175
176 private void setTableStateInZK(final TableName tableName,
177 final ZooKeeperProtos.Table.State state)
178 throws KeeperException {
179 String znode = ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString());
180 if (ZKUtil.checkExists(this.watcher, znode) == -1) {
181 ZKUtil.createAndFailSilent(this.watcher, znode);
182 }
183 synchronized (this.cache) {
184 ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
185 builder.setState(state);
186 byte [] data = ProtobufUtil.prependPBMagic(builder.build().toByteArray());
187 ZKUtil.setData(this.watcher, znode, data);
188 this.cache.put(tableName, state);
189 }
190 }
191
192
193
194
195
196
197 @Override
198 public boolean isTableState(final TableName tableName,
199 final ZooKeeperProtos.Table.State... states) {
200 synchronized (this.cache) {
201 ZooKeeperProtos.Table.State currentState = this.cache.get(tableName);
202 return isTableInState(Arrays.asList(states), currentState);
203 }
204 }
205
206
207
208
209
210
211
212 @Override
213 public void setDeletedTable(final TableName tableName)
214 throws CoordinatedStateException {
215 synchronized (this.cache) {
216 if (this.cache.remove(tableName) == null) {
217 LOG.warn("Moving table " + tableName + " state to deleted but was " +
218 "already deleted");
219 }
220 try {
221 ZKUtil.deleteNodeFailSilent(this.watcher,
222 ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
223 } catch (KeeperException e) {
224 throw new CoordinatedStateException(e);
225 }
226 }
227 }
228
229
230
231
232
233
234
235 @Override
236 public boolean isTablePresent(final TableName tableName) {
237 synchronized (this.cache) {
238 ZooKeeperProtos.Table.State state = this.cache.get(tableName);
239 return !(state == null);
240 }
241 }
242
243
244
245
246
247
248 @Override
249 public Set<TableName> getTablesInStates(ZooKeeperProtos.Table.State... states)
250 throws InterruptedIOException, CoordinatedStateException {
251 try {
252 return getAllTables(states);
253 } catch (KeeperException e) {
254 throw new CoordinatedStateException(e);
255 }
256 }
257
258
259
260
261 @Override
262 public void checkAndRemoveTableState(TableName tableName, ZooKeeperProtos.Table.State states,
263 boolean deletePermanentState)
264 throws CoordinatedStateException {
265 synchronized (this.cache) {
266 if (isTableState(tableName, states)) {
267 this.cache.remove(tableName);
268 if (deletePermanentState) {
269 try {
270 ZKUtil.deleteNodeFailSilent(this.watcher,
271 ZKUtil.joinZNode(this.watcher.tableZNode, tableName.getNameAsString()));
272 } catch (KeeperException e) {
273 throw new CoordinatedStateException(e);
274 }
275 }
276 }
277 }
278 }
279
280
281
282
283
284
285 Set<TableName> getAllTables(final ZooKeeperProtos.Table.State... states)
286 throws KeeperException, InterruptedIOException {
287
288 Set<TableName> allTables = new HashSet<TableName>();
289 List<String> children =
290 ZKUtil.listChildrenNoWatch(watcher, watcher.tableZNode);
291 if(children == null) return allTables;
292 for (String child: children) {
293 TableName tableName = TableName.valueOf(child);
294 ZooKeeperProtos.Table.State state;
295 try {
296 state = getTableState(watcher, tableName);
297 } catch (InterruptedException e) {
298 throw new InterruptedIOException();
299 }
300 for (ZooKeeperProtos.Table.State expectedState: states) {
301 if (state == expectedState) {
302 allTables.add(tableName);
303 break;
304 }
305 }
306 }
307 return allTables;
308 }
309
310
311
312
313
314
315
316
317 private ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
318 final TableName tableName)
319 throws KeeperException, InterruptedException {
320 String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
321 byte [] data = ZKUtil.getData(zkw, znode);
322 if (data == null || data.length <= 0) return null;
323 try {
324 ProtobufUtil.expectPBMagicPrefix(data);
325 ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
326 int magicLen = ProtobufUtil.lengthOfPBMagic();
327 ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
328 return t.getState();
329 } catch (InvalidProtocolBufferException e) {
330 KeeperException ke = new KeeperException.DataInconsistencyException();
331 ke.initCause(e);
332 throw ke;
333 } catch (DeserializationException e) {
334 throw ZKUtil.convert(e);
335 }
336 }
337
338
339
340
341
342 private boolean isTableInState(final List<ZooKeeperProtos.Table.State> expectedStates,
343 final ZooKeeperProtos.Table.State currentState) {
344 return currentState != null && expectedStates.contains(currentState);
345 }
346 }