1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.master;
19
20 import java.util.Date;
21 import java.util.concurrent.atomic.AtomicLong;
22
23 import org.apache.hadoop.classification.InterfaceAudience;
24 import org.apache.hadoop.classification.InterfaceStability;
25 import org.apache.hadoop.hbase.HRegionInfo;
26 import org.apache.hadoop.hbase.ServerName;
27 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos;
28
29
30
31
32
33
34 @InterfaceAudience.Private
35 public class RegionState {
36
37 @InterfaceAudience.Public
38 @InterfaceStability.Evolving
39 public enum State {
40 OFFLINE,
41 PENDING_OPEN,
42 OPENING,
43 OPEN,
44 PENDING_CLOSE,
45 CLOSING,
46 CLOSED,
47 SPLITTING,
48 SPLIT,
49 FAILED_OPEN,
50 FAILED_CLOSE,
51 MERGING,
52 MERGED,
53 SPLITTING_NEW,
54
55
56 MERGING_NEW
57
58
59 }
60
61
62 private final AtomicLong stamp;
63 private HRegionInfo hri;
64
65 private volatile ServerName serverName;
66 private volatile State state;
67
68 public RegionState() {
69 this.stamp = new AtomicLong(System.currentTimeMillis());
70 }
71
72 public RegionState(HRegionInfo region, State state) {
73 this(region, state, System.currentTimeMillis(), null);
74 }
75
76 public RegionState(HRegionInfo region,
77 State state, ServerName serverName) {
78 this(region, state, System.currentTimeMillis(), serverName);
79 }
80
81 public RegionState(HRegionInfo region,
82 State state, long stamp, ServerName serverName) {
83 this.hri = region;
84 this.state = state;
85 this.stamp = new AtomicLong(stamp);
86 this.serverName = serverName;
87 }
88
89 public void updateTimestampToNow() {
90 setTimestamp(System.currentTimeMillis());
91 }
92
93 public State getState() {
94 return state;
95 }
96
97 public long getStamp() {
98 return stamp.get();
99 }
100
101 public HRegionInfo getRegion() {
102 return hri;
103 }
104
105 public ServerName getServerName() {
106 return serverName;
107 }
108
109 public boolean isClosing() {
110 return state == State.CLOSING;
111 }
112
113 public boolean isClosed() {
114 return state == State.CLOSED;
115 }
116
117 public boolean isPendingClose() {
118 return state == State.PENDING_CLOSE;
119 }
120
121 public boolean isOpening() {
122 return state == State.OPENING;
123 }
124
125 public boolean isOpened() {
126 return state == State.OPEN;
127 }
128
129 public boolean isPendingOpen() {
130 return state == State.PENDING_OPEN;
131 }
132
133 public boolean isOffline() {
134 return state == State.OFFLINE;
135 }
136
137 public boolean isSplitting() {
138 return state == State.SPLITTING;
139 }
140
141 public boolean isSplit() {
142 return state == State.SPLIT;
143 }
144
145 public boolean isSplittingNew() {
146 return state == State.SPLITTING_NEW;
147 }
148
149 public boolean isFailedOpen() {
150 return state == State.FAILED_OPEN;
151 }
152
153 public boolean isFailedClose() {
154 return state == State.FAILED_CLOSE;
155 }
156
157 public boolean isMerging() {
158 return state == State.MERGING;
159 }
160
161 public boolean isMerged() {
162 return state == State.MERGED;
163 }
164
165 public boolean isMergingNew() {
166 return state == State.MERGING_NEW;
167 }
168
169 public boolean isOpenOrMergingOnServer(final ServerName sn) {
170 return isOnServer(sn) && (isOpened() || isMerging());
171 }
172
173 public boolean isOpenOrMergingNewOnServer(final ServerName sn) {
174 return isOnServer(sn) && (isOpened() || isMergingNew());
175 }
176
177 public boolean isOpenOrSplittingOnServer(final ServerName sn) {
178 return isOnServer(sn) && (isOpened() || isSplitting());
179 }
180
181 public boolean isOpenOrSplittingNewOnServer(final ServerName sn) {
182 return isOnServer(sn) && (isOpened() || isSplittingNew());
183 }
184
185 public boolean isPendingOpenOrOpeningOnServer(final ServerName sn) {
186 return isOnServer(sn) && isPendingOpenOrOpening();
187 }
188
189
190 public boolean isPendingOpenOrOpening() {
191 return isPendingOpen() || isOpening() || isFailedOpen();
192 }
193
194 public boolean isPendingCloseOrClosingOnServer(final ServerName sn) {
195 return isOnServer(sn) && isPendingCloseOrClosing();
196 }
197
198
199 public boolean isPendingCloseOrClosing() {
200 return isPendingClose() || isClosing() || isFailedClose();
201 }
202
203 public boolean isOnServer(final ServerName sn) {
204 return serverName != null && serverName.equals(sn);
205 }
206
207
208
209
210 public boolean isReadyToOffline() {
211 return isMerged() || isSplit() || isOffline()
212 || isSplittingNew() || isMergingNew();
213 }
214
215
216
217
218 public boolean isReadyToOnline() {
219 return isOpened() || isSplittingNew() || isMergingNew();
220 }
221
222
223
224
225
226 public boolean isUnassignable() {
227 return isUnassignable(state);
228 }
229
230
231
232
233
234 public static boolean isUnassignable(State state) {
235 return state == State.MERGED || state == State.SPLIT || state == State.OFFLINE
236 || state == State.SPLITTING_NEW || state == State.MERGING_NEW;
237 }
238
239 @Override
240 public String toString() {
241 return "{" + hri.getShortNameToLog()
242 + " state=" + state
243 + ", ts=" + stamp
244 + ", server=" + serverName + "}";
245 }
246
247
248
249
250 public String toDescriptiveString() {
251 long lstamp = stamp.get();
252 long relTime = System.currentTimeMillis() - lstamp;
253
254 return hri.getRegionNameAsString()
255 + " state=" + state
256 + ", ts=" + new Date(lstamp) + " (" + (relTime/1000) + "s ago)"
257 + ", server=" + serverName;
258 }
259
260
261
262
263
264
265 public ClusterStatusProtos.RegionState convert() {
266 ClusterStatusProtos.RegionState.Builder regionState = ClusterStatusProtos.RegionState.newBuilder();
267 ClusterStatusProtos.RegionState.State rs;
268 switch (this.state) {
269 case OFFLINE:
270 rs = ClusterStatusProtos.RegionState.State.OFFLINE;
271 break;
272 case PENDING_OPEN:
273 rs = ClusterStatusProtos.RegionState.State.PENDING_OPEN;
274 break;
275 case OPENING:
276 rs = ClusterStatusProtos.RegionState.State.OPENING;
277 break;
278 case OPEN:
279 rs = ClusterStatusProtos.RegionState.State.OPEN;
280 break;
281 case PENDING_CLOSE:
282 rs = ClusterStatusProtos.RegionState.State.PENDING_CLOSE;
283 break;
284 case CLOSING:
285 rs = ClusterStatusProtos.RegionState.State.CLOSING;
286 break;
287 case CLOSED:
288 rs = ClusterStatusProtos.RegionState.State.CLOSED;
289 break;
290 case SPLITTING:
291 rs = ClusterStatusProtos.RegionState.State.SPLITTING;
292 break;
293 case SPLIT:
294 rs = ClusterStatusProtos.RegionState.State.SPLIT;
295 break;
296 case FAILED_OPEN:
297 rs = ClusterStatusProtos.RegionState.State.FAILED_OPEN;
298 break;
299 case FAILED_CLOSE:
300 rs = ClusterStatusProtos.RegionState.State.FAILED_CLOSE;
301 break;
302 case MERGING:
303 rs = ClusterStatusProtos.RegionState.State.MERGING;
304 break;
305 case MERGED:
306 rs = ClusterStatusProtos.RegionState.State.MERGED;
307 break;
308 case SPLITTING_NEW:
309 rs = ClusterStatusProtos.RegionState.State.SPLITTING_NEW;
310 break;
311 case MERGING_NEW:
312 rs = ClusterStatusProtos.RegionState.State.MERGING_NEW;
313 break;
314 default:
315 throw new IllegalStateException("");
316 }
317 regionState.setRegionInfo(HRegionInfo.convert(hri));
318 regionState.setState(rs);
319 regionState.setStamp(getStamp());
320 return regionState.build();
321 }
322
323
324
325
326
327
328 public static RegionState convert(ClusterStatusProtos.RegionState proto) {
329 RegionState.State state;
330 switch (proto.getState()) {
331 case OFFLINE:
332 state = State.OFFLINE;
333 break;
334 case PENDING_OPEN:
335 state = State.PENDING_OPEN;
336 break;
337 case OPENING:
338 state = State.OPENING;
339 break;
340 case OPEN:
341 state = State.OPEN;
342 break;
343 case PENDING_CLOSE:
344 state = State.PENDING_CLOSE;
345 break;
346 case CLOSING:
347 state = State.CLOSING;
348 break;
349 case CLOSED:
350 state = State.CLOSED;
351 break;
352 case SPLITTING:
353 state = State.SPLITTING;
354 break;
355 case SPLIT:
356 state = State.SPLIT;
357 break;
358 case FAILED_OPEN:
359 state = State.FAILED_OPEN;
360 break;
361 case FAILED_CLOSE:
362 state = State.FAILED_CLOSE;
363 break;
364 case MERGING:
365 state = State.MERGING;
366 break;
367 case MERGED:
368 state = State.MERGED;
369 break;
370 case SPLITTING_NEW:
371 state = State.SPLITTING_NEW;
372 break;
373 case MERGING_NEW:
374 state = State.MERGING_NEW;
375 break;
376 default:
377 throw new IllegalStateException("");
378 }
379
380 return new RegionState(HRegionInfo.convert(proto.getRegionInfo()),state,proto.getStamp(),null);
381 }
382
383 protected void setTimestamp(final long timestamp) {
384 stamp.set(timestamp);
385 }
386
387
388
389
390 @Override
391 public boolean equals(Object obj) {
392 if (this == obj) return true;
393 if (obj == null || getClass() != obj.getClass()) {
394 return false;
395 }
396 RegionState tmp = (RegionState)obj;
397 return tmp.hri.equals(hri) && tmp.state == state
398 && ((serverName != null && serverName.equals(tmp.serverName))
399 || (tmp.serverName == null && serverName == null));
400 }
401
402
403
404
405 @Override
406 public int hashCode() {
407 return (serverName != null ? serverName.hashCode() * 11 : 0)
408 + hri.hashCode() + 5 * state.ordinal();
409 }
410 }