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.util.HashMap;
23  import java.util.HashSet;
24  import java.util.List;
25  import java.util.Map;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.hbase.CoordinatedStateException;
30  import org.apache.hadoop.hbase.HRegionInfo;
31  import org.apache.hadoop.hbase.MetaTableAccessor;
32  import org.apache.hadoop.hbase.Server;
33  import org.apache.hadoop.hbase.ServerName;
34  import org.apache.hadoop.hbase.TableName;
35  import org.apache.hadoop.hbase.TableNotDisabledException;
36  import org.apache.hadoop.hbase.TableNotFoundException;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.executor.EventHandler;
39  import org.apache.hadoop.hbase.executor.EventType;
40  import org.apache.hadoop.hbase.master.AssignmentManager;
41  import org.apache.hadoop.hbase.master.BulkAssigner;
42  import org.apache.hadoop.hbase.master.GeneralBulkAssigner;
43  import org.apache.hadoop.hbase.master.HMaster;
44  import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
45  import org.apache.hadoop.hbase.master.MasterServices;
46  import org.apache.hadoop.hbase.master.RegionStates;
47  import org.apache.hadoop.hbase.master.ServerManager;
48  import org.apache.hadoop.hbase.master.TableLockManager;
49  import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
50  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
51  import org.apache.hadoop.hbase.util.Pair;
52  import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
53  
54  /**
55   * Handler to run enable of a table.
56   */
57  @InterfaceAudience.Private
58  public class EnableTableHandler extends EventHandler {
59    private static final Log LOG = LogFactory.getLog(EnableTableHandler.class);
60    private final TableName tableName;
61    private final AssignmentManager assignmentManager;
62    private final TableLockManager tableLockManager;
63    private boolean skipTableStateCheck = false;
64    private TableLock tableLock;
65    private MasterServices services;
66  
67    public EnableTableHandler(Server server, TableName tableName,
68        AssignmentManager assignmentManager, TableLockManager tableLockManager,
69        boolean skipTableStateCheck) {
70      super(server, EventType.C_M_ENABLE_TABLE);
71      this.tableName = tableName;
72      this.assignmentManager = assignmentManager;
73      this.tableLockManager = tableLockManager;
74      this.skipTableStateCheck = skipTableStateCheck;
75    }
76  
77    public EnableTableHandler(MasterServices services, TableName tableName,
78        AssignmentManager assignmentManager,
79        TableLockManager tableLockManager, boolean skipTableStateCheck) {
80      this((Server)services, tableName, assignmentManager, tableLockManager,
81          skipTableStateCheck);
82      this.services = services;
83    }
84  
85    public EnableTableHandler prepare()
86        throws TableNotFoundException, TableNotDisabledException, IOException {
87      //acquire the table write lock, blocking
88      this.tableLock = this.tableLockManager.writeLock(tableName,
89          EventType.C_M_ENABLE_TABLE.toString());
90      this.tableLock.acquire();
91  
92      boolean success = false;
93      try {
94        // Check if table exists
95        if (!MetaTableAccessor.tableExists(this.server.getConnection(), tableName)) {
96          // retainAssignment is true only during recovery.  In normal case it is false
97          if (!this.skipTableStateCheck) {
98            throw new TableNotFoundException(tableName);
99          }
100         try {
101           this.assignmentManager.getTableStateManager().checkAndRemoveTableState(tableName,
102             ZooKeeperProtos.Table.State.ENABLING, true);
103           throw new TableNotFoundException(tableName);
104         } catch (CoordinatedStateException e) {
105           // TODO : Use HBCK to clear such nodes
106           LOG.warn("Failed to delete the ENABLING node for the table " + tableName
107               + ".  The table will remain unusable. Run HBCK to manually fix the problem.");
108         }
109       }
110 
111       // There could be multiple client requests trying to disable or enable
112       // the table at the same time. Ensure only the first request is honored
113       // After that, no other requests can be accepted until the table reaches
114       // DISABLED or ENABLED.
115       if (!skipTableStateCheck) {
116         try {
117           if (!this.assignmentManager.getTableStateManager().setTableStateIfInStates(
118               this.tableName, ZooKeeperProtos.Table.State.ENABLING,
119               ZooKeeperProtos.Table.State.DISABLED)) {
120             LOG.info("Table " + tableName + " isn't disabled; skipping enable");
121             throw new TableNotDisabledException(this.tableName);
122           }
123         } catch (CoordinatedStateException e) {
124           throw new IOException("Unable to ensure that the table will be" +
125             " enabling because of a coordination engine issue", e);
126         }
127       }
128       success = true;
129     } finally {
130       if (!success) {
131         releaseTableLock();
132       }
133     }
134     return this;
135   }
136 
137   @Override
138   public String toString() {
139     String name = "UnknownServerName";
140     if(server != null && server.getServerName() != null) {
141       name = server.getServerName().toString();
142     }
143     return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
144         tableName;
145   }
146 
147   @Override
148   public void process() {
149     try {
150       LOG.info("Attempting to enable the table " + this.tableName);
151       MasterCoprocessorHost cpHost = ((HMaster) this.server)
152           .getMasterCoprocessorHost();
153       if (cpHost != null) {
154         cpHost.preEnableTableHandler(this.tableName);
155       }
156       handleEnableTable();
157       if (cpHost != null) {
158         cpHost.postEnableTableHandler(this.tableName);
159       }
160     } catch (IOException e) {
161       LOG.error("Error trying to enable the table " + this.tableName, e);
162     } catch (CoordinatedStateException e) {
163       LOG.error("Error trying to enable the table " + this.tableName, e);
164     } catch (InterruptedException e) {
165       LOG.error("Error trying to enable the table " + this.tableName, e);
166     } finally {
167       releaseTableLock();
168     }
169   }
170 
171   private void releaseTableLock() {
172     if (this.tableLock != null) {
173       try {
174         this.tableLock.release();
175       } catch (IOException ex) {
176         LOG.warn("Could not release the table lock", ex);
177       }
178     }
179   }
180 
181   private void handleEnableTable() throws IOException, CoordinatedStateException,
182       InterruptedException {
183     // I could check table is disabling and if so, not enable but require
184     // that user first finish disabling but that might be obnoxious.
185 
186     // Set table enabling flag up in zk.
187     this.assignmentManager.getTableStateManager().setTableState(this.tableName,
188       ZooKeeperProtos.Table.State.ENABLING);
189     boolean done = false;
190     ServerManager serverManager = ((HMaster)this.server).getServerManager();
191     // Get the regions of this table. We're done when all listed
192     // tables are onlined.
193     List<Pair<HRegionInfo, ServerName>> tableRegionsAndLocations;
194     if (TableName.META_TABLE_NAME.equals(tableName)) {
195       tableRegionsAndLocations = new MetaTableLocator().getMetaRegionsAndLocations(
196         server.getZooKeeper());
197     } else {
198       tableRegionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(
199         server.getZooKeeper(), server.getConnection(), tableName, true);
200     }
201 
202     int countOfRegionsInTable = tableRegionsAndLocations.size();
203     Map<HRegionInfo, ServerName> regionsToAssign =
204         regionsToAssignWithServerName(tableRegionsAndLocations);
205     if (services != null) {
206       // need to potentially create some regions for the replicas
207       List<HRegionInfo> unrecordedReplicas = AssignmentManager.replicaRegionsNotRecordedInMeta(
208           new HashSet<HRegionInfo>(regionsToAssign.keySet()), services);
209       Map<ServerName, List<HRegionInfo>> srvToUnassignedRegs =
210             this.assignmentManager.getBalancer().roundRobinAssignment(unrecordedReplicas,
211                 serverManager.getOnlineServersList());
212       if (srvToUnassignedRegs != null) {
213         for (Map.Entry<ServerName, List<HRegionInfo>> entry : srvToUnassignedRegs.entrySet()) {
214           for (HRegionInfo h : entry.getValue()) {
215             regionsToAssign.put(h, entry.getKey());
216           }
217         }
218       }
219     }
220     int regionsCount = regionsToAssign.size();
221     if (regionsCount == 0) {
222       done = true;
223     }
224     LOG.info("Table '" + this.tableName + "' has " + countOfRegionsInTable
225       + " regions, of which " + regionsCount + " are offline.");
226     List<ServerName> onlineServers = serverManager.createDestinationServersList();
227     Map<ServerName, List<HRegionInfo>> bulkPlan =
228         this.assignmentManager.getBalancer().retainAssignment(regionsToAssign, onlineServers);
229     if (bulkPlan != null) {
230       LOG.info("Bulk assigning " + regionsCount + " region(s) across " + bulkPlan.size()
231           + " server(s), retainAssignment=true");
232 
233       BulkAssigner ba =
234           new GeneralBulkAssigner(this.server, bulkPlan, this.assignmentManager, true);
235       try {
236         if (ba.bulkAssign()) {
237           done = true;
238         }
239       } catch (InterruptedException e) {
240         LOG.warn("Enable operation was interrupted when enabling table '"
241             + this.tableName + "'");
242         // Preserve the interrupt.
243         Thread.currentThread().interrupt();
244       }
245     } else {
246       done = true;
247       LOG.info("Balancer was unable to find suitable servers for table " + tableName
248           + ", leaving unassigned");
249     }
250     if (done) {
251       // Flip the table to enabled.
252       this.assignmentManager.getTableStateManager().setTableState(
253         this.tableName, ZooKeeperProtos.Table.State.ENABLED);
254       LOG.info("Table '" + this.tableName
255       + "' was successfully enabled. Status: done=" + done);
256     } else {
257       LOG.warn("Table '" + this.tableName
258       + "' wasn't successfully enabled. Status: done=" + done);
259     }
260   }
261 
262   /**
263    * @param regionsInMeta
264    * @return List of regions neither in transition nor assigned.
265    * @throws IOException
266    */
267   private Map<HRegionInfo, ServerName> regionsToAssignWithServerName(
268       final List<Pair<HRegionInfo, ServerName>> regionsInMeta) throws IOException {
269     Map<HRegionInfo, ServerName> regionsToAssign =
270         new HashMap<HRegionInfo, ServerName>(regionsInMeta.size());
271     RegionStates regionStates = this.assignmentManager.getRegionStates();
272     for (Pair<HRegionInfo, ServerName> regionLocation : regionsInMeta) {
273       HRegionInfo hri = regionLocation.getFirst();
274       ServerName sn = regionLocation.getSecond();
275       if (regionStates.isRegionOffline(hri)) {
276         regionsToAssign.put(hri, sn);
277       } else {
278         if (LOG.isDebugEnabled()) {
279           LOG.debug("Skipping assign for the region " + hri + " during enable table "
280               + hri.getTable() + " because its already in tranition or assigned.");
281         }
282       }
283     }
284     return regionsToAssign;
285   }
286 }