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.List;
23  import java.util.concurrent.ExecutorService;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.classification.InterfaceAudience;
28  import org.apache.hadoop.hbase.TableName;
29  import org.apache.hadoop.hbase.HRegionInfo;
30  import org.apache.hadoop.hbase.Server;
31  import org.apache.hadoop.hbase.TableNotEnabledException;
32  import org.apache.hadoop.hbase.TableNotFoundException;
33  import org.apache.hadoop.hbase.catalog.CatalogTracker;
34  import org.apache.hadoop.hbase.catalog.MetaReader;
35  import org.apache.hadoop.hbase.constraint.ConstraintException;
36  import org.apache.hadoop.hbase.executor.EventHandler;
37  import org.apache.hadoop.hbase.executor.EventType;
38  import org.apache.hadoop.hbase.master.AssignmentManager;
39  import org.apache.hadoop.hbase.master.BulkAssigner;
40  import org.apache.hadoop.hbase.master.HMaster;
41  import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
42  import org.apache.hadoop.hbase.master.RegionStates;
43  import org.apache.hadoop.hbase.master.TableLockManager;
44  import org.apache.hadoop.hbase.master.RegionState.State;
45  import org.apache.hadoop.hbase.master.TableLockManager.TableLock;
46  import org.apache.zookeeper.KeeperException;
47  import org.cloudera.htrace.Trace;
48  
49  /**
50   * Handler to run disable of a table.
51   */
52  @InterfaceAudience.Private
53  public class DisableTableHandler extends EventHandler {
54    private static final Log LOG = LogFactory.getLog(DisableTableHandler.class);
55    private final TableName tableName;
56    private final AssignmentManager assignmentManager;
57    private final TableLockManager tableLockManager;
58    private final CatalogTracker catalogTracker;
59    private final boolean skipTableStateCheck;
60    private TableLock tableLock;
61  
62    public DisableTableHandler(Server server, TableName tableName,
63        CatalogTracker catalogTracker, AssignmentManager assignmentManager,
64        TableLockManager tableLockManager, boolean skipTableStateCheck) {
65      super(server, EventType.C_M_DISABLE_TABLE);
66      this.tableName = tableName;
67      this.assignmentManager = assignmentManager;
68      this.catalogTracker = catalogTracker;
69      this.tableLockManager = tableLockManager;
70      this.skipTableStateCheck = skipTableStateCheck;
71    }
72  
73    public DisableTableHandler prepare()
74        throws TableNotFoundException, TableNotEnabledException, IOException {
75      if(tableName.equals(TableName.META_TABLE_NAME)) {
76        throw new ConstraintException("Cannot disable catalog table");
77      }
78      //acquire the table write lock, blocking
79      this.tableLock = this.tableLockManager.writeLock(tableName,
80          EventType.C_M_DISABLE_TABLE.toString());
81      this.tableLock.acquire();
82  
83      boolean success = false;
84      try {
85        // Check if table exists
86        if (!MetaReader.tableExists(catalogTracker, tableName)) {
87          throw new TableNotFoundException(tableName);
88        }
89  
90        // There could be multiple client requests trying to disable or enable
91        // the table at the same time. Ensure only the first request is honored
92        // After that, no other requests can be accepted until the table reaches
93        // DISABLED or ENABLED.
94        //TODO: reevaluate this since we have table locks now
95        if (!skipTableStateCheck) {
96          try {
97            if (!this.assignmentManager.getZKTable().checkEnabledAndSetDisablingTable
98              (this.tableName)) {
99              LOG.info("Table " + tableName + " isn't enabled; skipping disable");
100             throw new TableNotEnabledException(this.tableName);
101           }
102         } catch (KeeperException e) {
103           throw new IOException("Unable to ensure that the table will be" +
104             " disabling because of a ZooKeeper issue", e);
105         }
106       }
107       success = true;
108     } finally {
109       if (!success) {
110         releaseTableLock();
111       }
112     }
113 
114     return this;
115   }
116 
117   @Override
118   public String toString() {
119     String name = "UnknownServerName";
120     if(server != null && server.getServerName() != null) {
121       name = server.getServerName().toString();
122     }
123     return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
124         tableName;
125   }
126 
127   @Override
128   public void process() {
129     try {
130       LOG.info("Attempting to disable table " + this.tableName);
131       MasterCoprocessorHost cpHost = ((HMaster) this.server)
132           .getCoprocessorHost();
133       if (cpHost != null) {
134         cpHost.preDisableTableHandler(this.tableName);
135       }
136       handleDisableTable();
137       if (cpHost != null) {
138         cpHost.postDisableTableHandler(this.tableName);
139       }
140     } catch (IOException e) {
141       LOG.error("Error trying to disable table " + this.tableName, e);
142     } catch (KeeperException e) {
143       LOG.error("Error trying to disable table " + this.tableName, e);
144     } finally {
145       releaseTableLock();
146     }
147   }
148 
149   private void releaseTableLock() {
150     if (this.tableLock != null) {
151       try {
152         this.tableLock.release();
153       } catch (IOException ex) {
154         LOG.warn("Could not release the table lock", ex);
155       }
156     }
157   }
158 
159   private void handleDisableTable() throws IOException, KeeperException {
160     // Set table disabling flag up in zk.
161     this.assignmentManager.getZKTable().setDisablingTable(this.tableName);
162     boolean done = false;
163     while (true) {
164       // Get list of online regions that are of this table.  Regions that are
165       // already closed will not be included in this list; i.e. the returned
166       // list is not ALL regions in a table, its all online regions according
167       // to the in-memory state on this master.
168       final List<HRegionInfo> regions = this.assignmentManager
169         .getRegionStates().getRegionsOfTable(tableName);
170       if (regions.size() == 0) {
171         done = true;
172         break;
173       }
174       LOG.info("Offlining " + regions.size() + " regions.");
175       BulkDisabler bd = new BulkDisabler(this.server, regions);
176       try {
177         if (bd.bulkAssign()) {
178           done = true;
179           break;
180         }
181       } catch (InterruptedException e) {
182         LOG.warn("Disable was interrupted");
183         // Preserve the interrupt.
184         Thread.currentThread().interrupt();
185         break;
186       }
187     }
188     // Flip the table to disabled if success.
189     if (done) this.assignmentManager.getZKTable().setDisabledTable(this.tableName);
190     LOG.info("Disabled table is done=" + done);
191   }
192 
193   /**
194    * Run bulk disable.
195    */
196   class BulkDisabler extends BulkAssigner {
197     private final List<HRegionInfo> regions;
198 
199     BulkDisabler(final Server server, final List<HRegionInfo> regions) {
200       super(server);
201       this.regions = regions;
202     }
203 
204     @Override
205     protected void populatePool(ExecutorService pool) {
206       RegionStates regionStates = assignmentManager.getRegionStates();
207       for (HRegionInfo region: regions) {
208         if (regionStates.isRegionInTransition(region)
209             && !regionStates.isRegionInState(region, State.FAILED_CLOSE)) {
210           continue;
211         }
212         final HRegionInfo hri = region;
213         pool.execute(Trace.wrap("DisableTableHandler.BulkDisabler",new Runnable() {
214           public void run() {
215             assignmentManager.unassign(hri, true);
216           }
217         }));
218       }
219     }
220 
221     @Override
222     protected boolean waitUntilDone(long timeout)
223     throws InterruptedException {
224       long startTime = System.currentTimeMillis();
225       long remaining = timeout;
226       List<HRegionInfo> regions = null;
227       while (!server.isStopped() && remaining > 0) {
228         Thread.sleep(waitingTimeForEvents);
229         regions = assignmentManager.getRegionStates().getRegionsOfTable(tableName);
230         LOG.debug("Disable waiting until done; " + remaining + " ms remaining; " + regions);
231         if (regions.isEmpty()) break;
232         remaining = timeout - (System.currentTimeMillis() - startTime);
233       }
234       return regions != null && regions.isEmpty();
235     }
236   }
237 }