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