1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.master.handler;
20
21 import java.io.IOException;
22 import java.io.InterruptedIOException;
23 import java.util.ArrayList;
24 import java.util.HashSet;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.NavigableMap;
28 import java.util.Set;
29
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.classification.InterfaceAudience;
33 import org.apache.hadoop.hbase.HConstants;
34 import org.apache.hadoop.hbase.HRegionInfo;
35 import org.apache.hadoop.hbase.Server;
36 import org.apache.hadoop.hbase.ServerName;
37 import org.apache.hadoop.hbase.catalog.CatalogTracker;
38 import org.apache.hadoop.hbase.catalog.MetaReader;
39 import org.apache.hadoop.hbase.client.Result;
40 import org.apache.hadoop.hbase.executor.EventHandler;
41 import org.apache.hadoop.hbase.executor.EventType;
42 import org.apache.hadoop.hbase.master.AssignmentManager;
43 import org.apache.hadoop.hbase.master.DeadServer;
44 import org.apache.hadoop.hbase.master.MasterServices;
45 import org.apache.hadoop.hbase.master.RegionState;
46 import org.apache.hadoop.hbase.master.RegionStates;
47 import org.apache.hadoop.hbase.master.ServerManager;
48 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
49 import org.apache.zookeeper.KeeperException;
50
51
52
53
54
55
56 @InterfaceAudience.Private
57 public class ServerShutdownHandler extends EventHandler {
58 private static final Log LOG = LogFactory.getLog(ServerShutdownHandler.class);
59 protected final ServerName serverName;
60 protected final MasterServices services;
61 protected final DeadServer deadServers;
62 protected final boolean shouldSplitHlog;
63 protected final boolean distributedLogReplay;
64 protected final int regionAssignmentWaitTimeout;
65
66 public ServerShutdownHandler(final Server server, final MasterServices services,
67 final DeadServer deadServers, final ServerName serverName,
68 final boolean shouldSplitHlog) {
69 this(server, services, deadServers, serverName, EventType.M_SERVER_SHUTDOWN,
70 shouldSplitHlog);
71 }
72
73 ServerShutdownHandler(final Server server, final MasterServices services,
74 final DeadServer deadServers, final ServerName serverName, EventType type,
75 final boolean shouldSplitHlog) {
76 super(server, type);
77 this.serverName = serverName;
78 this.server = server;
79 this.services = services;
80 this.deadServers = deadServers;
81 if (!this.deadServers.isDeadServer(this.serverName)) {
82 LOG.warn(this.serverName + " is NOT in deadservers; it should be!");
83 }
84 this.shouldSplitHlog = shouldSplitHlog;
85 this.distributedLogReplay = server.getConfiguration().getBoolean(
86 HConstants.DISTRIBUTED_LOG_REPLAY_KEY,
87 HConstants.DEFAULT_DISTRIBUTED_LOG_REPLAY_CONFIG);
88 this.regionAssignmentWaitTimeout = server.getConfiguration().getInt(
89 HConstants.LOG_REPLAY_WAIT_REGION_TIMEOUT, 15000);
90 }
91
92 @Override
93 public String getInformativeName() {
94 if (serverName != null) {
95 return this.getClass().getSimpleName() + " for " + serverName;
96 } else {
97 return super.getInformativeName();
98 }
99 }
100
101
102
103
104 boolean isCarryingMeta() {
105 return false;
106 }
107
108 @Override
109 public String toString() {
110 String name = "UnknownServerName";
111 if(server != null && server.getServerName() != null) {
112 name = server.getServerName().toString();
113 }
114 return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
115 }
116
117 @Override
118 public void process() throws IOException {
119 boolean hasLogReplayWork = false;
120 final ServerName serverName = this.serverName;
121 try {
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144 if (isCarryingMeta()
145 || !services.getAssignmentManager().isFailoverCleanupDone()) {
146 this.services.getServerManager().processDeadServer(serverName, this.shouldSplitHlog);
147 return;
148 }
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165 NavigableMap<HRegionInfo, Result> hris = null;
166 while (!this.server.isStopped()) {
167 try {
168 this.server.getCatalogTracker().waitForMeta();
169 hris = MetaReader.getServerUserRegions(this.server.getCatalogTracker(),
170 this.serverName);
171 break;
172 } catch (InterruptedException e) {
173 Thread.currentThread().interrupt();
174 throw new IOException("Interrupted", e);
175 } catch (IOException ioe) {
176 LOG.info("Received exception accessing META during server shutdown of " +
177 serverName + ", retrying META read", ioe);
178 }
179 }
180 if (this.server.isStopped()) {
181 throw new IOException("Server is stopped");
182 }
183
184 try {
185 if (this.shouldSplitHlog) {
186 LOG.info("Splitting logs for " + serverName + " before assignment.");
187 if (this.distributedLogReplay) {
188 LOG.info("Mark regions in recovery before assignment.");
189 Set<ServerName> serverNames = new HashSet<ServerName>();
190 serverNames.add(serverName);
191 this.services.getMasterFileSystem().prepareLogReplay(serverNames);
192 } else {
193 this.services.getMasterFileSystem().splitLog(serverName);
194 }
195 } else {
196 LOG.info("Skipping log splitting for " + serverName);
197 }
198 } catch (IOException ioe) {
199 resubmit(serverName, ioe);
200 }
201
202
203
204
205
206 AssignmentManager am = services.getAssignmentManager();
207 List<HRegionInfo> regionsInTransition = am.processServerShutdown(serverName);
208 LOG.info("Reassigning " + ((hris == null)? 0: hris.size()) +
209 " region(s) that " + (serverName == null? "null": serverName) +
210 " was carrying (and " + regionsInTransition.size() +
211 " regions(s) that were opening on this server)");
212
213 List<HRegionInfo> toAssignRegions = new ArrayList<HRegionInfo>();
214 toAssignRegions.addAll(regionsInTransition);
215
216
217 if (hris != null) {
218 RegionStates regionStates = am.getRegionStates();
219 for (Map.Entry<HRegionInfo, Result> e: hris.entrySet()) {
220 HRegionInfo hri = e.getKey();
221 if (regionsInTransition.contains(hri)) {
222 continue;
223 }
224 RegionState rit = regionStates.getRegionTransitionState(hri);
225 if (processDeadRegion(hri, e.getValue(), am, server.getCatalogTracker())) {
226 ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
227 if (addressFromAM != null && !addressFromAM.equals(this.serverName)) {
228
229
230 LOG.info("Skip assigning region " + hri.getRegionNameAsString()
231 + " because it has been opened in " + addressFromAM.getServerName());
232 continue;
233 }
234 if (rit != null) {
235 if (!rit.isOnServer(serverName)
236 || rit.isClosed() || rit.isOpened()) {
237
238
239 LOG.info("Skip assigning region " + rit);
240 continue;
241 }
242 try{
243
244 LOG.info("Reassigning region with rs = " + rit + " and deleting zk node if exists");
245 ZKAssign.deleteNodeFailSilent(services.getZooKeeper(), hri);
246 } catch (KeeperException ke) {
247 this.server.abort("Unexpected ZK exception deleting unassigned node " + hri, ke);
248 return;
249 }
250 }
251 toAssignRegions.add(hri);
252 } else if (rit != null) {
253 if ((rit.isClosing() || rit.isPendingClose())
254 && am.getZKTable().isDisablingOrDisabledTable(hri.getTableName())) {
255
256
257
258
259
260 am.deleteClosingOrClosedNode(hri);
261 am.regionOffline(hri);
262 } else {
263 LOG.warn("THIS SHOULD NOT HAPPEN: unexpected region in transition "
264 + rit + " not to be assigned by SSH of server " + serverName);
265 }
266 }
267 }
268 }
269
270 try {
271 am.assign(toAssignRegions);
272 } catch (InterruptedException ie) {
273 LOG.error("Caught " + ie + " during round-robin assignment");
274 throw new IOException(ie);
275 }
276
277 if (this.shouldSplitHlog && this.distributedLogReplay) {
278
279 for (HRegionInfo hri : toAssignRegions) {
280 try {
281 if (!am.waitOnRegionToClearRegionsInTransition(hri, regionAssignmentWaitTimeout)) {
282
283
284 LOG.warn("Region " + hri.getEncodedName()
285 + " didn't complete assignment in time");
286 }
287 } catch (InterruptedException ie) {
288 throw new InterruptedIOException("Caught " + ie
289 + " during waitOnRegionToClearRegionsInTransition");
290 }
291 }
292
293 this.services.getExecutorService().submit(
294 new LogReplayHandler(this.server, this.services, this.deadServers, this.serverName));
295 hasLogReplayWork = true;
296 }
297 } finally {
298 this.deadServers.finish(serverName);
299 }
300
301 if (!hasLogReplayWork) {
302 LOG.info("Finished processing of shutdown of " + serverName);
303 }
304 }
305
306 private void resubmit(final ServerName serverName, IOException ex) throws IOException {
307
308
309 this.services.getExecutorService().submit((ServerShutdownHandler) this);
310 this.deadServers.add(serverName);
311 throw new IOException("failed log splitting for " + serverName + ", will retry", ex);
312 }
313
314
315
316
317
318
319
320
321
322
323
324 public static boolean processDeadRegion(HRegionInfo hri, Result result,
325 AssignmentManager assignmentManager, CatalogTracker catalogTracker)
326 throws IOException {
327 boolean tablePresent = assignmentManager.getZKTable().isTablePresent(
328 hri.getTableName());
329 if (!tablePresent) {
330 LOG.info("The table " + hri.getTableName()
331 + " was deleted. Hence not proceeding.");
332 return false;
333 }
334
335 boolean disabled = assignmentManager.getZKTable().isDisabledTable(
336 hri.getTableName());
337 if (disabled){
338 LOG.info("The table " + hri.getTableName()
339 + " was disabled. Hence not proceeding.");
340 return false;
341 }
342 if (hri.isOffline() && hri.isSplit()) {
343
344
345
346 return false;
347 }
348 boolean disabling = assignmentManager.getZKTable().isDisablingTable(
349 hri.getTableName());
350 if (disabling) {
351 LOG.info("The table " + hri.getTableName()
352 + " is disabled. Hence not assigning region" + hri.getEncodedName());
353 return false;
354 }
355 return true;
356 }
357 }