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.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.shouldSplitHlog) {
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          am.regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
93          verifyAndAssignMetaWithRetries();
94        } else if (!this.services.getCatalogTracker().isMetaLocationAvailable()) {
95          // the meta location as per master is null. This could happen in case when meta assignment
96          // in previous run failed, while meta znode has been updated to null. We should try to
97          // assign the meta again.
98          verifyAndAssignMetaWithRetries();
99        } else {
100         LOG.info("META has been assigned to otherwhere, skip assigning.");
101       }
102 
103       try {
104         if (this.shouldSplitHlog && distributedLogReplay) {
105           if (!am.waitOnRegionToClearRegionsInTransition(HRegionInfo.FIRST_META_REGIONINFO,
106             regionAssignmentWaitTimeout)) {
107             // Wait here is to avoid log replay hits current dead server and incur a RPC timeout
108             // when replay happens before region assignment completes.
109             LOG.warn("Region " + HRegionInfo.FIRST_META_REGIONINFO.getEncodedName()
110                 + " didn't complete assignment in time");
111           }
112           this.services.getMasterFileSystem().splitMetaLog(serverName);
113         }
114       } catch (Exception ex) {
115         if (ex instanceof IOException) {
116           this.services.getExecutorService().submit(this);
117           this.deadServers.add(serverName);
118           throw new IOException("failed log splitting for " + serverName + ", will retry", ex);
119         } else {
120           throw new IOException(ex);
121         }
122       }
123 
124       gotException = false;
125     } finally {
126       if (gotException){
127         // If we had an exception, this.deadServers.finish will be skipped in super.process()
128         this.deadServers.finish(serverName);
129       }     
130     }
131 
132     super.process();
133     // Clear this counter on successful handling.
134     this.eventExceptionCount.set(0);
135   }
136 
137   @Override
138   boolean isCarryingMeta() {
139     return true;
140   }
141 
142   /**
143    * Before assign the hbase:meta region, ensure it haven't
144    *  been assigned by other place
145    * <p>
146    * Under some scenarios, the hbase:meta region can be opened twice, so it seemed online
147    * in two regionserver at the same time.
148    * If the hbase:meta region has been assigned, so the operation can be canceled.
149    * @throws InterruptedException
150    * @throws IOException
151    * @throws KeeperException
152    */
153   private void verifyAndAssignMeta()
154       throws InterruptedException, IOException, KeeperException {
155     long timeout = this.server.getConfiguration().
156         getLong("hbase.catalog.verification.timeout", 1000);
157     if (!this.server.getCatalogTracker().verifyMetaRegionLocation(timeout)) {
158       this.services.getAssignmentManager().assignMeta();
159     } else if (serverName.equals(server.getCatalogTracker().getMetaLocation())) {
160       throw new IOException("hbase:meta is onlined on the dead server "
161           + serverName);
162     } else {
163       LOG.info("Skip assigning hbase:meta, because it is online on the "
164           + server.getCatalogTracker().getMetaLocation());
165     }
166   }
167 
168   /**
169    * Failed many times, shutdown processing
170    * @throws IOException
171    */
172   private void verifyAndAssignMetaWithRetries() throws IOException {
173     int iTimes = this.server.getConfiguration().getInt(
174         "hbase.catalog.verification.retries", 10);
175 
176     long waitTime = this.server.getConfiguration().getLong(
177         "hbase.catalog.verification.timeout", 1000);
178 
179     int iFlag = 0;
180     while (true) {
181       try {
182         verifyAndAssignMeta();
183         break;
184       } catch (KeeperException e) {
185         this.server.abort("In server shutdown processing, assigning meta", e);
186         throw new IOException("Aborting", e);
187       } catch (Exception e) {
188         if (iFlag >= iTimes) {
189           this.server.abort("verifyAndAssignMeta failed after" + iTimes
190               + " times retries, aborting", e);
191           throw new IOException("Aborting", e);
192         }
193         try {
194           Thread.sleep(waitTime);
195         } catch (InterruptedException e1) {
196           LOG.warn("Interrupted when is the thread sleep", e1);
197           Thread.currentThread().interrupt();
198           throw (InterruptedIOException)new InterruptedIOException().initCause(e1);
199         }
200         iFlag++;
201       }
202     }
203   }
204 
205   @Override
206   protected void handleException(Throwable t) {
207     int count = eventExceptionCount.getAndIncrement();
208     if (count < 0) count = eventExceptionCount.getAndSet(0);
209     if (count > SHOW_STRACKTRACE_FREQUENCY) { // Too frequent, let's slow reporting
210       Threads.sleep(1000);
211     }
212     if (count % SHOW_STRACKTRACE_FREQUENCY == 0) {
213       LOG.error("Caught " + eventType + ", count=" + this.eventExceptionCount, t); 
214     } else {
215       LOG.error("Caught " + eventType + ", count=" + this.eventExceptionCount +
216         "; " + t.getMessage() + "; stack trace shows every " + SHOW_STRACKTRACE_FREQUENCY +
217         "th time.");
218     }
219   }
220 }