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