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 com.google.protobuf.InvalidProtocolBufferException;
23
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.TableName;
26 import org.apache.hadoop.hbase.exceptions.DeserializationException;
27 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
28 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
29 import org.apache.zookeeper.KeeperException;
30
31 import java.util.HashSet;
32 import java.util.List;
33 import java.util.Set;
34
35
36
37
38
39
40
41
42
43 @InterfaceAudience.Private
44 public class ZKTableStateClientSideReader {
45
46 private ZKTableStateClientSideReader() {}
47
48
49
50
51
52
53
54
55
56
57 public static boolean isDisabledTable(final ZooKeeperWatcher zkw,
58 final TableName tableName)
59 throws KeeperException, InterruptedException {
60 ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
61 return isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
62 }
63
64
65
66
67
68
69
70
71
72
73 public static boolean isEnabledTable(final ZooKeeperWatcher zkw,
74 final TableName tableName)
75 throws KeeperException, InterruptedException {
76 return getTableState(zkw, tableName) == ZooKeeperProtos.Table.State.ENABLED;
77 }
78
79
80
81
82
83
84
85
86
87
88
89 public static boolean isDisablingOrDisabledTable(final ZooKeeperWatcher zkw,
90 final TableName tableName)
91 throws KeeperException, InterruptedException {
92 ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
93 return isTableState(ZooKeeperProtos.Table.State.DISABLING, state) ||
94 isTableState(ZooKeeperProtos.Table.State.DISABLED, state);
95 }
96
97
98
99
100
101
102 public static Set<TableName> getDisabledTables(ZooKeeperWatcher zkw)
103 throws KeeperException, InterruptedException {
104 Set<TableName> disabledTables = new HashSet<TableName>();
105 List<String> children =
106 ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
107 for (String child: children) {
108 TableName tableName =
109 TableName.valueOf(child);
110 ZooKeeperProtos.Table.State state = getTableState(zkw, tableName);
111 if (state == ZooKeeperProtos.Table.State.DISABLED) disabledTables.add(tableName);
112 }
113 return disabledTables;
114 }
115
116
117
118
119
120
121 public static Set<TableName> getDisabledOrDisablingTables(ZooKeeperWatcher zkw)
122 throws KeeperException, InterruptedException {
123 return
124 getTablesInStates(
125 zkw,
126 ZooKeeperProtos.Table.State.DISABLED,
127 ZooKeeperProtos.Table.State.DISABLING);
128 }
129
130
131
132
133
134
135
136
137 public static Set<TableName> getEnablingTables(ZooKeeperWatcher zkw)
138 throws KeeperException, InterruptedException {
139 return getTablesInStates(zkw, ZooKeeperProtos.Table.State.ENABLING);
140 }
141
142
143
144
145
146
147
148
149
150 private static Set<TableName> getTablesInStates(
151 ZooKeeperWatcher zkw,
152 ZooKeeperProtos.Table.State... states)
153 throws KeeperException, InterruptedException {
154 Set<TableName> tableNameSet = new HashSet<TableName>();
155 List<String> children = ZKUtil.listChildrenNoWatch(zkw, zkw.tableZNode);
156 TableName tableName;
157 ZooKeeperProtos.Table.State tableState;
158 for (String child: children) {
159 tableName = TableName.valueOf(child);
160 tableState = getTableState(zkw, tableName);
161 for (ZooKeeperProtos.Table.State state : states) {
162 if (tableState == state) {
163 tableNameSet.add(tableName);
164 break;
165 }
166 }
167 }
168 return tableNameSet;
169 }
170
171 static boolean isTableState(final ZooKeeperProtos.Table.State expectedState,
172 final ZooKeeperProtos.Table.State currentState) {
173 return currentState != null && currentState.equals(expectedState);
174 }
175
176
177
178
179
180
181
182 static ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,
183 final TableName tableName)
184 throws KeeperException, InterruptedException {
185 String znode = ZKUtil.joinZNode(zkw.tableZNode, tableName.getNameAsString());
186 byte [] data = ZKUtil.getData(zkw, znode);
187 if (data == null || data.length <= 0) return null;
188 try {
189 ProtobufUtil.expectPBMagicPrefix(data);
190 ZooKeeperProtos.Table.Builder builder = ZooKeeperProtos.Table.newBuilder();
191 int magicLen = ProtobufUtil.lengthOfPBMagic();
192 ZooKeeperProtos.Table t = builder.mergeFrom(data, magicLen, data.length - magicLen).build();
193 return t.getState();
194 } catch (InvalidProtocolBufferException e) {
195 KeeperException ke = new KeeperException.DataInconsistencyException();
196 ke.initCause(e);
197 throw ke;
198 } catch (DeserializationException e) {
199 throw ZKUtil.convert(e);
200 }
201 }
202 }