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