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