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