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