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
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.classification.InterfaceAudience;
26 import org.apache.hadoop.hbase.HRegionInfo;
27 import org.apache.hadoop.hbase.Server;
28 import org.apache.hadoop.hbase.ServerName;
29 import org.apache.hadoop.hbase.executor.EventType;
30 import org.apache.hadoop.hbase.master.DeadServer;
31 import org.apache.hadoop.hbase.master.MasterServices;
32 import org.apache.zookeeper.KeeperException;
33
34
35
36
37 @InterfaceAudience.Private
38 public class MetaServerShutdownHandler extends ServerShutdownHandler {
39 private static final Log LOG = LogFactory.getLog(MetaServerShutdownHandler.class);
40 public MetaServerShutdownHandler(final Server server,
41 final MasterServices services,
42 final DeadServer deadServers, final ServerName serverName) {
43 super(server, services, deadServers, serverName,
44 EventType.M_META_SERVER_SHUTDOWN, true);
45 }
46
47 @Override
48 public void process() throws IOException {
49 boolean gotException = true;
50 try{
51 try {
52 LOG.info("Splitting META logs for " + serverName);
53 if (this.shouldSplitHlog) {
54 this.services.getMasterFileSystem().splitMetaLog(serverName);
55 }
56 } catch (IOException ioe) {
57 this.services.getExecutorService().submit(this);
58 this.deadServers.add(serverName);
59 throw new IOException("failed log splitting for " +
60 serverName + ", will retry", ioe);
61 }
62
63
64
65
66 if (this.services.getAssignmentManager().isCarryingMeta(serverName)) {
67 LOG.info("Server " + serverName + " was carrying META. Trying to assign.");
68 this.services.getAssignmentManager().regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
69 verifyAndAssignMetaWithRetries();
70 } else {
71 LOG.info("META has been assigned to otherwhere, skip assigning.");
72 }
73
74 gotException = false;
75 } finally {
76 if (gotException){
77
78 this.deadServers.finish(serverName);
79 }
80 }
81 super.process();
82 }
83
84
85
86
87
88
89
90
91
92
93
94
95 private void verifyAndAssignMeta()
96 throws InterruptedException, IOException, KeeperException {
97 long timeout = this.server.getConfiguration().
98 getLong("hbase.catalog.verification.timeout", 1000);
99 if (!this.server.getCatalogTracker().verifyMetaRegionLocation(timeout)) {
100 this.services.getAssignmentManager().assignMeta();
101 } else if (serverName.equals(server.getCatalogTracker().getMetaLocation())) {
102 throw new IOException(".META. is onlined on the dead server "
103 + serverName);
104 } else {
105 LOG.info("Skip assigning .META., because it is online on the "
106 + server.getCatalogTracker().getMetaLocation());
107 }
108 }
109
110
111
112
113
114 private void verifyAndAssignMetaWithRetries() throws IOException {
115 int iTimes = this.server.getConfiguration().getInt(
116 "hbase.catalog.verification.retries", 10);
117
118 long waitTime = this.server.getConfiguration().getLong(
119 "hbase.catalog.verification.timeout", 1000);
120
121 int iFlag = 0;
122 while (true) {
123 try {
124 verifyAndAssignMeta();
125 break;
126 } catch (KeeperException e) {
127 this.server.abort("In server shutdown processing, assigning meta", e);
128 throw new IOException("Aborting", e);
129 } catch (Exception e) {
130 if (iFlag >= iTimes) {
131 this.server.abort("verifyAndAssignMeta failed after" + iTimes
132 + " times retries, aborting", e);
133 throw new IOException("Aborting", e);
134 }
135 try {
136 Thread.sleep(waitTime);
137 } catch (InterruptedException e1) {
138 LOG.warn("Interrupted when is the thread sleep", e1);
139 Thread.currentThread().interrupt();
140 throw new IOException("Interrupted", e1);
141 }
142 iFlag++;
143 }
144 }
145 }
146
147 @Override
148 public String toString() {
149 String name = "UnknownServerName";
150 if(server != null && server.getServerName() != null) {
151 name = server.getServerName().toString();
152 }
153 return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
154 }
155 }