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