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.util;
20  
21  import java.io.IOException;
22  import java.util.Collection;
23  import java.util.List;
24  import java.util.Map;
25  import java.util.Random;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.Path;
31  import org.apache.hadoop.hbase.HRegionInfo;
32  import org.apache.hadoop.hbase.HTableDescriptor;
33  import org.apache.hadoop.hbase.MetaTableAccessor;
34  import org.apache.hadoop.hbase.ServerName;
35  import org.apache.hadoop.hbase.TableName;
36  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.client.Admin;
39  import org.apache.hadoop.hbase.client.ClusterConnection;
40  import org.apache.hadoop.hbase.client.Connection;
41  import org.apache.hadoop.hbase.client.ConnectionFactory;
42  import org.apache.hadoop.hbase.client.HConnection;
43  import org.apache.hadoop.hbase.client.Put;
44  import org.apache.hadoop.hbase.client.Table;
45  import org.apache.hadoop.hbase.master.RegionState;
46  import org.apache.hadoop.hbase.master.ServerManager;
47  import org.apache.hadoop.hbase.regionserver.HRegion;
48  import org.apache.zookeeper.KeeperException;
49  
50  import java.io.IOException;
51  import java.util.Collection;
52  import java.util.List;
53  import java.util.Map;
54  import java.util.Random;
55  
56  /**
57   * This class contains helper methods that repair parts of hbase's filesystem
58   * contents.
59   */
60  @InterfaceAudience.Private
61  public class HBaseFsckRepair {
62    private static final Log LOG = LogFactory.getLog(HBaseFsckRepair.class);
63  
64    /**
65     * Fix multiple assignment by doing silent closes on each RS hosting the region
66     * and then force ZK unassigned node to OFFLINE to trigger assignment by
67     * master.
68     *
69     * @param connection HBase connection to the cluster
70     * @param region Region to undeploy
71     * @param servers list of Servers to undeploy from
72     */
73    public static void fixMultiAssignment(HConnection connection, HRegionInfo region,
74        List<ServerName> servers)
75    throws IOException, KeeperException, InterruptedException {
76      HRegionInfo actualRegion = new HRegionInfo(region);
77  
78      // Close region on the servers silently
79      for(ServerName server : servers) {
80        closeRegionSilentlyAndWait(connection, server, actualRegion);
81      }
82  
83      // Force ZK node to OFFLINE so master assigns
84      forceOfflineInZK(connection.getAdmin(), actualRegion);
85    }
86  
87    /**
88     * Fix unassigned by creating/transition the unassigned ZK node for this
89     * region to OFFLINE state with a special flag to tell the master that this is
90     * a forced operation by HBCK.
91     *
92     * This assumes that info is in META.
93     *
94     * @param admin
95     * @param region
96     * @throws IOException
97     * @throws KeeperException
98     */
99    public static void fixUnassigned(Admin admin, HRegionInfo region)
100       throws IOException, KeeperException, InterruptedException {
101     HRegionInfo actualRegion = new HRegionInfo(region);
102 
103     // Force ZK node to OFFLINE so master assigns
104     forceOfflineInZK(admin, actualRegion);
105   }
106 
107   /**
108    * In 0.90, this forces an HRI offline by setting the RegionTransitionData
109    * in ZK to have HBCK_CODE_NAME as the server.  This is a special case in
110    * the AssignmentManager that attempts an assign call by the master.
111    *
112    * @see org.apache.hadoop.hbase.master.AssignementManager#handleHBCK
113    *
114    * This doesn't seem to work properly in the updated version of 0.92+'s hbck
115    * so we use assign to force the region into transition.  This has the
116    * side-effect of requiring a HRegionInfo that considers regionId (timestamp)
117    * in comparators that is addressed by HBASE-5563.
118    */
119   private static void forceOfflineInZK(Admin admin, final HRegionInfo region)
120   throws ZooKeeperConnectionException, KeeperException, IOException, InterruptedException {
121     admin.assign(region.getRegionName());
122   }
123 
124   /*
125    * Should we check all assignments or just not in RIT?
126    */
127   public static void waitUntilAssigned(Admin admin,
128       HRegionInfo region) throws IOException, InterruptedException {
129     long timeout = admin.getConfiguration().getLong("hbase.hbck.assign.timeout", 120000);
130     long expiration = timeout + EnvironmentEdgeManager.currentTime();
131     while (EnvironmentEdgeManager.currentTime() < expiration) {
132       try {
133         Map<String, RegionState> rits=
134             admin.getClusterStatus().getRegionsInTransition();
135 
136         if (rits.keySet() != null && !rits.keySet().contains(region.getEncodedName())) {
137           // yay! no longer RIT
138           return;
139         }
140         // still in rit
141         LOG.info("Region still in transition, waiting for "
142             + "it to become assigned: " + region);
143       } catch (IOException e) {
144         LOG.warn("Exception when waiting for region to become assigned,"
145             + " retrying", e);
146       }
147       Thread.sleep(1000);
148     }
149     throw new IOException("Region " + region + " failed to move out of " +
150         "transition within timeout " + timeout + "ms");
151   }
152 
153   /**
154    * Contacts a region server and waits up to hbase.hbck.close.timeout ms
155    * (default 120s) to close the region.  This bypasses the active hmaster.
156    */
157   @SuppressWarnings("deprecation")
158   public static void closeRegionSilentlyAndWait(HConnection connection,
159       ServerName server, HRegionInfo region) throws IOException, InterruptedException {
160     long timeout = connection.getConfiguration()
161       .getLong("hbase.hbck.close.timeout", 120000);
162     ServerManager.closeRegionSilentlyAndWait((ClusterConnection)connection, server,
163         region, timeout);
164   }
165 
166   /**
167    * Puts the specified HRegionInfo into META with replica related columns
168    */
169   public static void fixMetaHoleOnlineAndAddReplicas(Configuration conf,
170       HRegionInfo hri, Collection<ServerName> servers, int numReplicas) throws IOException {
171     Connection conn = ConnectionFactory.createConnection(conf);
172     Table meta = conn.getTable(TableName.META_TABLE_NAME);
173     Put put = MetaTableAccessor.makePutFromRegionInfo(hri);
174     if (numReplicas > 1) {
175       Random r = new Random();
176       ServerName[] serversArr = servers.toArray(new ServerName[servers.size()]);
177       for (int i = 1; i < numReplicas; i++) {
178         ServerName sn = serversArr[r.nextInt(serversArr.length)];
179         // the column added here is just to make sure the master is able to
180         // see the additional replicas when it is asked to assign. The
181         // final value of these columns will be different and will be updated
182         // by the actual regionservers that start hosting the respective replicas
183         MetaTableAccessor.addLocation(put, sn, sn.getStartcode(), -1, i);
184       }
185     }
186     meta.put(put);
187     meta.close();
188     conn.close();
189   }
190 
191   /**
192    * Creates, flushes, and closes a new region.
193    */
194   public static HRegion createHDFSRegionDir(Configuration conf,
195       HRegionInfo hri, HTableDescriptor htd) throws IOException {
196     // Create HRegion
197     Path root = FSUtils.getRootDir(conf);
198     HRegion region = HRegion.createHRegion(hri, root, conf, htd, null);
199 
200     // Close the new region to flush to disk. Close log file too.
201     HRegion.closeHRegion(region);
202     return region;
203   }
204 
205   /*
206    * Remove parent
207    */
208   public static void removeParentInMeta(Configuration conf, HRegionInfo hri) throws IOException {
209     Connection conn = ConnectionFactory.createConnection(conf);
210     MetaTableAccessor.deleteRegion(conn, hri);
211   }
212 }