View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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.hbase.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.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
37  import org.apache.hadoop.hbase.util.Threads;
38  import org.apache.zookeeper.KeeperException;
39  
40  import com.google.common.annotations.VisibleForTesting;
41  
42  import java.util.concurrent.atomic.AtomicInteger;
43  
44  /**
45   * Shutdown handler for the server hosting <code>hbase:meta</code>
46   */
47  @InterfaceAudience.Private
48  public class MetaServerShutdownHandler extends ServerShutdownHandler {
49    private static final Log LOG = LogFactory.getLog(MetaServerShutdownHandler.class);
50    private AtomicInteger eventExceptionCount = new AtomicInteger(0);
51    @VisibleForTesting
52    static final int SHOW_STRACKTRACE_FREQUENCY = 100;
53  
54    public MetaServerShutdownHandler(final Server server,
55        final MasterServices services,
56        final DeadServer deadServers, final ServerName serverName) {
57      super(server, services, deadServers, serverName,
58        EventType.M_META_SERVER_SHUTDOWN, true);
59    }
60  
61    @Override
62    public void process() throws IOException {
63      boolean gotException = true; 
64      try {
65        AssignmentManager am = this.services.getAssignmentManager();
66        this.services.getMasterFileSystem().setLogRecoveryMode();
67        boolean distributedLogReplay = 
68          (this.services.getMasterFileSystem().getLogRecoveryMode() == RecoveryMode.LOG_REPLAY);
69        try {
70          if (this.shouldSplitWal) {
71            LOG.info("Splitting hbase:meta logs for " + serverName);
72            if (distributedLogReplay) {
73              Set<HRegionInfo> regions = new HashSet<HRegionInfo>();
74              regions.add(HRegionInfo.FIRST_META_REGIONINFO);
75              this.services.getMasterFileSystem().prepareLogReplay(serverName, regions);
76            } else {
77              this.services.getMasterFileSystem().splitMetaLog(serverName);
78            }
79            am.getRegionStates().logSplit(HRegionInfo.FIRST_META_REGIONINFO);
80          }
81        } catch (IOException ioe) {
82          this.services.getExecutorService().submit(this);
83          this.deadServers.add(serverName);
84          throw new IOException("failed log splitting for " + serverName + ", will retry", ioe);
85        }
86    
87        // Assign meta if we were carrying it.
88        // Check again: region may be assigned to other where because of RIT
89        // timeout
90        if (am.isCarryingMeta(serverName)) {
91          LOG.info("Server " + serverName + " was carrying META. Trying to assign.");
92          verifyAndAssignMetaWithRetries();
93        } else {
94          LOG.info("META has been assigned to otherwhere, skip assigning.");
95        }
96  
97        try {
98          if (this.shouldSplitWal && distributedLogReplay) {
99            if (!am.waitOnRegionToClearRegionsInTransition(HRegionInfo.FIRST_META_REGIONINFO,
100             regionAssignmentWaitTimeout)) {
101             // Wait here is to avoid log replay hits current dead server and incur a RPC timeout
102             // when replay happens before region assignment completes.
103             LOG.warn("Region " + HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()
104                 + " didn't complete assignment in time");
105           }
106           this.services.getMasterFileSystem().splitMetaLog(serverName);
107         }
108       } catch (Exception ex) {
109         if (ex instanceof IOException) {
110           this.services.getExecutorService().submit(this);
111           this.deadServers.add(serverName);
112           throw new IOException("failed log splitting for " + serverName + ", will retry", ex);
113         } else {
114           throw new IOException(ex);
115         }
116       }
117 
118       gotException = false;
119     } finally {
120       if (gotException){
121         // If we had an exception, this.deadServers.finish will be skipped in super.process()
122         this.deadServers.finish(serverName);
123       }     
124     }
125 
126     super.process();
127     // Clear this counter on successful handling.
128     this.eventExceptionCount.set(0);
129   }
130 
131   @Override
132   boolean isCarryingMeta() {
133     return true;
134   }
135 
136   /**
137    * Before assign the hbase:meta region, ensure it haven't
138    *  been assigned by other place
139    * <p>
140    * Under some scenarios, the hbase:meta region can be opened twice, so it seemed online
141    * in two regionserver at the same time.
142    * If the hbase:meta region has been assigned, so the operation can be canceled.
143    * @throws InterruptedException
144    * @throws IOException
145    * @throws KeeperException
146    */
147   private void verifyAndAssignMeta()
148       throws InterruptedException, IOException, KeeperException {
149     long timeout = this.server.getConfiguration().
150         getLong("hbase.catalog.verification.timeout", 1000);
151     if (!server.getMetaTableLocator().verifyMetaRegionLocation(server.getShortCircuitConnection(),
152       this.server.getZooKeeper(), timeout)) {
153       this.services.getAssignmentManager().assignMeta();
154     } else if (serverName.equals(server.getMetaTableLocator().getMetaRegionLocation(
155       this.server.getZooKeeper()))) {
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.getMetaTableLocator().getMetaRegionLocation(this.server.getZooKeeper()));
161     }
162   }
163 
164   /**
165    * Failed many times, shutdown processing
166    * @throws IOException
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   protected void handleException(Throwable t) {
203     int count = eventExceptionCount.getAndIncrement();
204     if (count < 0) count = eventExceptionCount.getAndSet(0);
205     if (count > SHOW_STRACKTRACE_FREQUENCY) { // Too frequent, let's slow reporting
206       Threads.sleep(1000);
207     }
208     if (count % SHOW_STRACKTRACE_FREQUENCY == 0) {
209       LOG.error("Caught " + eventType + ", count=" + this.eventExceptionCount, t); 
210     } else {
211       LOG.error("Caught " + eventType + ", count=" + this.eventExceptionCount +
212         "; " + t.getMessage() + "; stack trace shows every " + SHOW_STRACKTRACE_FREQUENCY +
213         "th time.");
214     }
215   }
216 }