View Javadoc

1   /**
2    * Copyright The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one or more
5    * contributor license agreements. See the NOTICE file distributed with this
6    * work for additional information regarding copyright ownership. The ASF
7    * licenses this file to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance with the License.
9    * 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, WITHOUT
15   * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
16   * License for the specific language governing permissions and limitations
17   * under the License.
18   */
19  package org.apache.hadoop.hbase.regionserver;
20  
21  import java.io.IOException;
22  
23  import org.apache.commons.logging.Log;
24  import org.apache.commons.logging.LogFactory;
25  import org.apache.hadoop.hbase.classification.InterfaceAudience;
26  import org.apache.hadoop.hbase.DroppedSnapshotException;
27  import org.apache.hadoop.hbase.RemoteExceptionHandler;
28  import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
29  import org.apache.hadoop.hbase.security.User;
30  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
31  import org.apache.hadoop.util.StringUtils;
32  
33  import com.google.common.base.Preconditions;
34  
35  /**
36   * Handles processing region merges. Put in a queue, owned by HRegionServer.
37   */
38  @InterfaceAudience.Private
39  class RegionMergeRequest implements Runnable {
40    static final Log LOG = LogFactory.getLog(RegionMergeRequest.class);
41    private final HRegion region_a;
42    private final HRegion region_b;
43    private final HRegionServer server;
44    private final boolean forcible;
45    private TableLock tableLock;
46    private final long masterSystemTime;
47    private final User user;
48  
49    RegionMergeRequest(Region a, Region b, HRegionServer hrs, boolean forcible,
50      long masterSystemTime, User user) {
51      Preconditions.checkNotNull(hrs);
52      this.region_a = (HRegion)a;
53      this.region_b = (HRegion)b;
54      this.server = hrs;
55      this.forcible = forcible;
56      this.masterSystemTime = masterSystemTime;
57      this.user = user;
58    }
59  
60    @Override
61    public String toString() {
62      return "MergeRequest,regions:" + region_a + ", " + region_b + ", forcible="
63          + forcible;
64    }
65  
66    @Override
67    public void run() {
68      if (this.server.isStopping() || this.server.isStopped()) {
69        LOG.debug("Skipping merge because server is stopping="
70            + this.server.isStopping() + " or stopped=" + this.server.isStopped());
71        return;
72      }
73      try {
74        final long startTime = EnvironmentEdgeManager.currentTime();
75        RegionMergeTransactionImpl mt = new RegionMergeTransactionImpl(region_a,
76            region_b, forcible, masterSystemTime);
77  
78        //acquire a shared read lock on the table, so that table schema modifications
79        //do not happen concurrently
80        tableLock = server.getTableLockManager().readLock(region_a.getTableDesc().getTableName()
81            , "MERGE_REGIONS:" + region_a.getRegionInfo().getRegionNameAsString() + ", " +
82                region_b.getRegionInfo().getRegionNameAsString());
83        try {
84          tableLock.acquire();
85        } catch (IOException ex) {
86          tableLock = null;
87          throw ex;
88        }
89  
90        // If prepare does not return true, for some reason -- logged inside in
91        // the prepare call -- we are not ready to merge just now. Just return.
92        if (!mt.prepare(this.server)) return;
93        try {
94          mt.execute(this.server, this.server, this.user);
95        } catch (Exception e) {
96          if (this.server.isStopping() || this.server.isStopped()) {
97            LOG.info(
98                "Skip rollback/cleanup of failed merge of " + region_a + " and "
99                    + region_b + " because server is"
100                   + (this.server.isStopping() ? " stopping" : " stopped"), e);
101           return;
102         }
103         if (e instanceof DroppedSnapshotException) {
104           server.abort("Replay of WAL required. Forcing server shutdown", e);
105           return;
106         }
107         try {
108           LOG.warn("Running rollback/cleanup of failed merge of "
109                   + region_a +" and "+ region_b + "; " + e.getMessage(), e);
110           if (mt.rollback(this.server, this.server)) {
111             LOG.info("Successful rollback of failed merge of "
112                 + region_a +" and "+ region_b);
113           } else {
114             this.server.abort("Abort; we got an error after point-of-no-return"
115                 + "when merging " + region_a + " and " + region_b);
116           }
117         } catch (RuntimeException ee) {
118           String msg = "Failed rollback of failed merge of "
119               + region_a +" and "+ region_b + " -- aborting server";
120           // If failed rollback, kill this server to avoid having a hole in
121           // table.
122           LOG.info(msg, ee);
123           this.server.abort(msg);
124         }
125         return;
126       }
127       LOG.info("Regions merged, hbase:meta updated, and report to master. region_a="
128           + region_a + ", region_b=" + region_b + ",merged region="
129           + mt.getMergedRegionInfo().getRegionNameAsString()
130           + ". Region merge took "
131           + StringUtils.formatTimeDiff(EnvironmentEdgeManager.currentTime(), startTime));
132     } catch (IOException ex) {
133       LOG.error("Merge failed " + this,
134           RemoteExceptionHandler.checkIOException(ex));
135       server.checkFileSystem();
136     } finally {
137       releaseTableLock();
138     }
139   }
140 
141   protected void releaseTableLock() {
142     if (this.tableLock != null) {
143       try {
144         this.tableLock.release();
145       } catch (IOException ex) {
146         LOG.error("Could not release the table lock (something is really wrong). "
147            + "Aborting this server to avoid holding the lock forever.");
148         this.server.abort("Abort; we got an error when releasing the table lock "
149                          + "on " + region_a.getRegionInfo().getRegionNameAsString());
150       }
151     }
152   }
153 }