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