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