View Javadoc

1   /**
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.client;
21  
22  import java.io.Closeable;
23  import java.io.IOException;
24  import java.io.InterruptedIOException;
25  import java.lang.reflect.UndeclaredThrowableException;
26  import java.net.SocketTimeoutException;
27  import java.util.Arrays;
28  import java.util.LinkedList;
29  import java.util.List;
30  import java.util.concurrent.atomic.AtomicInteger;
31  import java.util.regex.Pattern;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.Abortable;
37  import org.apache.hadoop.hbase.ClusterStatus;
38  import org.apache.hadoop.hbase.HBaseConfiguration;
39  import org.apache.hadoop.hbase.HColumnDescriptor;
40  import org.apache.hadoop.hbase.HConstants;
41  import org.apache.hadoop.hbase.HRegionInfo;
42  import org.apache.hadoop.hbase.HRegionLocation;
43  import org.apache.hadoop.hbase.HTableDescriptor;
44  import org.apache.hadoop.hbase.MasterNotRunningException;
45  import org.apache.hadoop.hbase.NotServingRegionException;
46  import org.apache.hadoop.hbase.RegionException;
47  import org.apache.hadoop.hbase.RemoteExceptionHandler;
48  import org.apache.hadoop.hbase.ServerName;
49  import org.apache.hadoop.hbase.TableExistsException;
50  import org.apache.hadoop.hbase.TableNotFoundException;
51  import org.apache.hadoop.hbase.UnknownRegionException;
52  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
53  import org.apache.hadoop.hbase.catalog.CatalogTracker;
54  import org.apache.hadoop.hbase.catalog.MetaReader;
55  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
56  import org.apache.hadoop.hbase.ipc.HMasterInterface;
57  import org.apache.hadoop.hbase.ipc.HRegionInterface;
58  import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
59  import org.apache.hadoop.hbase.util.Addressing;
60  import org.apache.hadoop.hbase.util.Bytes;
61  import org.apache.hadoop.hbase.util.Pair;
62  import org.apache.hadoop.hbase.util.Writables;
63  import org.apache.hadoop.ipc.RemoteException;
64  import org.apache.hadoop.util.StringUtils;
65  
66  /**
67   * Provides an interface to manage HBase database table metadata + general
68   * administrative functions.  Use HBaseAdmin to create, drop, list, enable and
69   * disable tables. Use it also to add and drop table column families.
70   *
71   * <p>See {@link HTable} to add, update, and delete data from an individual table.
72   * <p>Currently HBaseAdmin instances are not expected to be long-lived.  For
73   * example, an HBaseAdmin instance will not ride over a Master restart.
74   */
75  public class HBaseAdmin implements Abortable, Closeable {
76    private final Log LOG = LogFactory.getLog(this.getClass().getName());
77  //  private final HConnection connection;
78    private HConnection connection;
79    private volatile Configuration conf;
80    private final long pause;
81    private final int numRetries;
82    // Some operations can take a long time such as disable of big table.
83    // numRetries is for 'normal' stuff... Mutliply by this factor when
84    // want to wait a long time.
85    private final int retryLongerMultiplier;
86    private boolean aborted;
87    
88    /**
89     * Constructor
90     *
91     * @param c Configuration object
92     * @throws MasterNotRunningException if the master is not running
93     * @throws ZooKeeperConnectionException if unable to connect to zookeeper
94     */
95    public HBaseAdmin(Configuration c)
96    throws MasterNotRunningException, ZooKeeperConnectionException {
97      this.conf = HBaseConfiguration.create(c);
98        this.connection = HConnectionManager.getConnection(this.conf);
99      this.pause = this.conf.getLong("hbase.client.pause", 1000);
100     this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
101     this.retryLongerMultiplier = this.conf.getInt(
102         "hbase.client.retries.longer.multiplier", 10);
103     int tries = 0;
104     for (; tries < numRetries; ++tries) {
105       try {
106         this.connection.getMaster();
107         break;
108       } catch (MasterNotRunningException mnre) {
109         HConnectionManager.deleteStaleConnection(this.connection);
110         this.connection = HConnectionManager.getConnection(this.conf);
111       } catch (UndeclaredThrowableException ute) {
112         HConnectionManager.deleteStaleConnection(this.connection);
113         this.connection = HConnectionManager.getConnection(this.conf);
114       }
115       try { // Sleep
116         Thread.sleep(getPauseTime(tries));
117       } catch (InterruptedException e) {
118         Thread.currentThread().interrupt();
119         // we should delete connection between client and zookeeper
120         HConnectionManager.deleteStaleConnection(this.connection);
121         throw new MasterNotRunningException("Interrupted");
122       }
123     }
124     if (tries >= numRetries) {
125       // we should delete connection between client and zookeeper
126       HConnectionManager.deleteStaleConnection(this.connection);
127       throw new MasterNotRunningException("Retried " + numRetries + " times");
128     }
129   }
130 
131   /**
132    * @return A new CatalogTracker instance; call {@link #cleanupCatalogTracker(CatalogTracker)}
133    * to cleanup the returned catalog tracker.
134    * @throws ZooKeeperConnectionException
135    * @throws IOException
136    * @see #cleanupCatalogTracker(CatalogTracker)
137    */
138   private synchronized CatalogTracker getCatalogTracker()
139   throws ZooKeeperConnectionException, IOException {
140     CatalogTracker ct = null;
141     try {
142       ct = new CatalogTracker(this.conf);
143       ct.start();
144     } catch (InterruptedException e) {
145       // Let it out as an IOE for now until we redo all so tolerate IEs
146       Thread.currentThread().interrupt();
147       throw new IOException("Interrupted", e);
148     }
149     return ct;
150   }
151 
152   private void cleanupCatalogTracker(final CatalogTracker ct) {
153     ct.stop();
154   }
155 
156   @Override
157   public void abort(String why, Throwable e) {
158     // Currently does nothing but throw the passed message and exception
159     this.aborted = true;
160     throw new RuntimeException(why, e);
161   }
162   
163   @Override
164   public boolean isAborted(){
165     return this.aborted;
166   }
167 
168   /** @return HConnection used by this object. */
169   public HConnection getConnection() {
170     return connection;
171   }
172 
173   /**
174    * Get a connection to the currently set master.
175    * @return proxy connection to master server for this instance
176    * @throws MasterNotRunningException if the master is not running
177    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
178    */
179   public HMasterInterface getMaster()
180   throws MasterNotRunningException, ZooKeeperConnectionException {
181     return this.connection.getMaster();
182   }
183 
184   /** @return - true if the master server is running
185    * @throws ZooKeeperConnectionException
186    * @throws MasterNotRunningException */
187   public boolean isMasterRunning()
188   throws MasterNotRunningException, ZooKeeperConnectionException {
189     return this.connection.isMasterRunning();
190   }
191 
192   /**
193    * @param tableName Table to check.
194    * @return True if table exists already.
195    * @throws IOException
196    */
197   public boolean tableExists(final String tableName)
198   throws IOException {
199     boolean b = false;
200     CatalogTracker ct = getCatalogTracker();
201     try {
202       b = MetaReader.tableExists(ct, tableName);
203     } finally {
204       cleanupCatalogTracker(ct);
205     }
206     return b;
207   }
208 
209   /**
210    * @param tableName Table to check.
211    * @return True if table exists already.
212    * @throws IOException
213    */
214   public boolean tableExists(final byte [] tableName)
215   throws IOException {
216     return tableExists(Bytes.toString(tableName));
217   }
218 
219   /**
220    * List all the userspace tables.  In other words, scan the META table.
221    *
222    * If we wanted this to be really fast, we could implement a special
223    * catalog table that just contains table names and their descriptors.
224    * Right now, it only exists as part of the META table's region info.
225    *
226    * @return - returns an array of HTableDescriptors
227    * @throws IOException if a remote or network exception occurs
228    */
229   public HTableDescriptor[] listTables() throws IOException {
230     return this.connection.listTables();
231   }
232 
233   /**
234    * List all the userspace tables matching the given pattern.
235    *
236    * @param pattern The compiled regular expression to match against
237    * @return - returns an array of HTableDescriptors
238    * @throws IOException if a remote or network exception occurs
239    * @see #listTables()
240    */
241   public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
242     List<HTableDescriptor> matched = new LinkedList<HTableDescriptor>();
243     HTableDescriptor[] tables = listTables();
244     for (HTableDescriptor table : tables) {
245       if (pattern.matcher(table.getNameAsString()).matches()) {
246         matched.add(table);
247       }
248     }
249     return matched.toArray(new HTableDescriptor[matched.size()]);
250   }
251 
252   /**
253    * List all the userspace tables matching the given regular expression.
254    *
255    * @param regex The regular expression to match against
256    * @return - returns an array of HTableDescriptors
257    * @throws IOException if a remote or network exception occurs
258    * @see #listTables(java.util.regex.Pattern)
259    */
260   public HTableDescriptor[] listTables(String regex) throws IOException {
261     return listTables(Pattern.compile(regex));
262   }
263 
264 
265   /**
266    * Method for getting the tableDescriptor
267    * @param tableName as a byte []
268    * @return the tableDescriptor
269    * @throws TableNotFoundException
270    * @throws IOException if a remote or network exception occurs
271    */
272   public HTableDescriptor getTableDescriptor(final byte [] tableName)
273   throws TableNotFoundException, IOException {
274     return this.connection.getHTableDescriptor(tableName);
275   }
276 
277   private long getPauseTime(int tries) {
278     int triesCount = tries;
279     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
280       triesCount = HConstants.RETRY_BACKOFF.length - 1;
281     }
282     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
283   }
284 
285   /**
286    * Creates a new table.
287    * Synchronous operation.
288    *
289    * @param desc table descriptor for table
290    *
291    * @throws IllegalArgumentException if the table name is reserved
292    * @throws MasterNotRunningException if master is not running
293    * @throws TableExistsException if table already exists (If concurrent
294    * threads, the table may have been created between test-for-existence
295    * and attempt-at-creation).
296    * @throws IOException if a remote or network exception occurs
297    */
298   public void createTable(HTableDescriptor desc)
299   throws IOException {
300     createTable(desc, null);
301   }
302 
303   /**
304    * Creates a new table with the specified number of regions.  The start key
305    * specified will become the end key of the first region of the table, and
306    * the end key specified will become the start key of the last region of the
307    * table (the first region has a null start key and the last region has a
308    * null end key).
309    *
310    * BigInteger math will be used to divide the key range specified into
311    * enough segments to make the required number of total regions.
312    *
313    * Synchronous operation.
314    *
315    * @param desc table descriptor for table
316    * @param startKey beginning of key range
317    * @param endKey end of key range
318    * @param numRegions the total number of regions to create
319    *
320    * @throws IllegalArgumentException if the table name is reserved
321    * @throws MasterNotRunningException if master is not running
322    * @throws TableExistsException if table already exists (If concurrent
323    * threads, the table may have been created between test-for-existence
324    * and attempt-at-creation).
325    * @throws IOException
326    */
327   public void createTable(HTableDescriptor desc, byte [] startKey,
328       byte [] endKey, int numRegions)
329   throws IOException {
330     HTableDescriptor.isLegalTableName(desc.getName());
331     if(numRegions < 3) {
332       throw new IllegalArgumentException("Must create at least three regions");
333     } else if(Bytes.compareTo(startKey, endKey) >= 0) {
334       throw new IllegalArgumentException("Start key must be smaller than end key");
335     }
336     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
337     if(splitKeys == null || splitKeys.length != numRegions - 1) {
338       throw new IllegalArgumentException("Unable to split key range into enough regions");
339     }
340     createTable(desc, splitKeys);
341   }
342 
343   /**
344    * Creates a new table with an initial set of empty regions defined by the
345    * specified split keys.  The total number of regions created will be the
346    * number of split keys plus one. Synchronous operation.
347    *
348    * @param desc table descriptor for table
349    * @param splitKeys array of split keys for the initial regions of the table
350    *
351    * @throws IllegalArgumentException if the table name is reserved
352    * @throws MasterNotRunningException if master is not running
353    * @throws TableExistsException if table already exists (If concurrent
354    * threads, the table may have been created between test-for-existence
355    * and attempt-at-creation).
356    * @throws IOException
357    */
358   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
359   throws IOException {
360     HTableDescriptor.isLegalTableName(desc.getName());
361     try {
362       createTableAsync(desc, splitKeys);
363     } catch (SocketTimeoutException ste) {
364       LOG.warn("Creating " + desc.getNameAsString() + " took too long", ste);
365     }
366     int numRegs = splitKeys == null ? 1 : splitKeys.length + 1;
367     int prevRegCount = 0;
368     for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
369       ++tries) {
370       // Wait for new table to come on-line
371       final AtomicInteger actualRegCount = new AtomicInteger(0);
372       MetaScannerVisitor visitor = new MetaScannerVisitor() {
373         @Override
374         public boolean processRow(Result rowResult) throws IOException {
375           HRegionInfo info = Writables.getHRegionInfoOrNull(
376               rowResult.getValue(HConstants.CATALOG_FAMILY,
377                   HConstants.REGIONINFO_QUALIFIER));
378           //If regioninfo is null, skip this row
379           if (null == info) {
380             return true;
381           }
382           if (!(Bytes.equals(info.getTableName(), desc.getName()))) {
383             return false;
384           }
385           String hostAndPort = null;
386           byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
387               HConstants.SERVER_QUALIFIER);
388           // Make sure that regions are assigned to server
389           if (value != null && value.length > 0) {
390             hostAndPort = Bytes.toString(value);
391           }
392           if (!(info.isOffline() || info.isSplit()) && hostAndPort != null) {
393             actualRegCount.incrementAndGet();
394           }
395           return true;
396         }
397       };
398       MetaScanner.metaScan(conf, visitor, desc.getName());
399       if (actualRegCount.get() != numRegs) {
400         if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
401           throw new RegionOfflineException("Only " + actualRegCount.get() +
402             " of " + numRegs + " regions are online; retries exhausted.");
403         }
404         try { // Sleep
405           Thread.sleep(getPauseTime(tries));
406         } catch (InterruptedException e) {
407           throw new InterruptedIOException("Interrupted when opening" +
408               " regions; " + actualRegCount.get() + " of " + numRegs + 
409               " regions processed so far");
410         }
411         if (actualRegCount.get() > prevRegCount) { // Making progress
412           prevRegCount = actualRegCount.get();
413           tries = -1;
414         }
415       } else {
416         return;
417       }
418     }
419   }
420 
421   /**
422    * Creates a new table but does not block and wait for it to come online.
423    * Asynchronous operation.
424    *
425    * @param desc table descriptor for table
426    *
427    * @throws IllegalArgumentException Bad table name.
428    * @throws MasterNotRunningException if master is not running
429    * @throws TableExistsException if table already exists (If concurrent
430    * threads, the table may have been created between test-for-existence
431    * and attempt-at-creation).
432    * @throws IOException
433    */
434   public void createTableAsync(HTableDescriptor desc, byte [][] splitKeys)
435   throws IOException {
436     HTableDescriptor.isLegalTableName(desc.getName());
437     if(splitKeys != null && splitKeys.length > 1) {
438       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
439       // Verify there are no duplicate split keys
440       byte [] lastKey = null;
441       for(byte [] splitKey : splitKeys) {
442         if(lastKey != null && Bytes.equals(splitKey, lastKey)) {
443           throw new IllegalArgumentException("All split keys must be unique, " +
444             "found duplicate: " + Bytes.toStringBinary(splitKey) +
445             ", " + Bytes.toStringBinary(lastKey));
446         }
447         lastKey = splitKey;
448       }
449     }
450     try {
451       getMaster().createTable(desc, splitKeys);
452     } catch (RemoteException e) {
453       throw e.unwrapRemoteException();
454     }
455   }
456 
457   /**
458    * Deletes a table.
459    * Synchronous operation.
460    *
461    * @param tableName name of table to delete
462    * @throws IOException if a remote or network exception occurs
463    */
464   public void deleteTable(final String tableName) throws IOException {
465     deleteTable(Bytes.toBytes(tableName));
466   }
467 
468   /**
469    * Deletes a table.
470    * Synchronous operation.
471    *
472    * @param tableName name of table to delete
473    * @throws IOException if a remote or network exception occurs
474    */
475   public void deleteTable(final byte [] tableName) throws IOException {
476     isMasterRunning();
477     HTableDescriptor.isLegalTableName(tableName);
478     HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
479     try {
480       getMaster().deleteTable(tableName);
481     } catch (RemoteException e) {
482       throw RemoteExceptionHandler.decodeRemoteException(e);
483     }
484     // Wait until all regions deleted
485     HRegionInterface server =
486       connection.getHRegionConnection(firstMetaServer.getHostname(), firstMetaServer.getPort());
487     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
488       long scannerId = -1L;
489       try {
490 
491         Scan scan = MetaReader.getScanForTableName(tableName);
492         scan.addColumn(HConstants.CATALOG_FAMILY,
493             HConstants.REGIONINFO_QUALIFIER);
494         scannerId = server.openScanner(
495           firstMetaServer.getRegionInfo().getRegionName(), scan);
496         // Get a batch at a time.
497         Result values = server.next(scannerId);
498 
499         // let us wait until .META. table is updated and
500         // HMaster removes the table from its HTableDescriptors
501         if (values == null) {
502           boolean tableExists = false;
503           HTableDescriptor[] htds = getMaster().getHTableDescriptors();
504           if (htds != null && htds.length > 0) {
505             for (HTableDescriptor htd: htds) {
506               if (Bytes.equals(tableName, htd.getName())) {
507                 tableExists = true;
508                 break;
509               }
510             }
511           }
512           if (!tableExists) {
513             break;
514           }
515         }
516       } catch (IOException ex) {
517         if(tries == numRetries - 1) {           // no more tries left
518           if (ex instanceof RemoteException) {
519             ex = RemoteExceptionHandler.decodeRemoteException((RemoteException) ex);
520           }
521           throw ex;
522         }
523       } finally {
524         if (scannerId != -1L) {
525           try {
526             server.close(scannerId);
527           } catch (Exception ex) {
528             LOG.warn(ex);
529           }
530         }
531       }
532       try {
533         Thread.sleep(getPauseTime(tries));
534       } catch (InterruptedException e) {
535         // continue
536       }
537     }
538     // Delete cached information to prevent clients from using old locations
539     this.connection.clearRegionCache(tableName);
540     LOG.info("Deleted " + Bytes.toString(tableName));
541   }
542 
543   /**
544    * Deletes tables matching the passed in pattern and wait on completion.
545    *
546    * Warning: Use this method carefully, there is no prompting and the effect is
547    * immediate. Consider using {@link #listTables(java.lang.String)} and
548    * {@link #deleteTable(byte[])}
549    *
550    * @param regex The regular expression to match table names against
551    * @return Table descriptors for tables that couldn't be deleted
552    * @throws IOException
553    * @see #deleteTables(java.util.regex.Pattern)
554    * @see #deleteTable(java.lang.String)
555    */
556   public HTableDescriptor[] deleteTables(String regex) throws IOException {
557     return deleteTables(Pattern.compile(regex));
558   }
559 
560   /**
561    * Delete tables matching the passed in pattern and wait on completion.
562    *
563    * Warning: Use this method carefully, there is no prompting and the effect is
564    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
565    * {@link #deleteTable(byte[])}
566    *
567    * @param pattern The pattern to match table names against
568    * @return Table descriptors for tables that couldn't be deleted
569    * @throws IOException
570    */
571   public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
572     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
573     for (HTableDescriptor table : listTables(pattern)) {
574       try {
575         deleteTable(table.getName());
576       } catch (IOException ex) {
577         LOG.info("Failed to delete table " + table.getNameAsString(), ex);
578         failed.add(table);
579       }
580     }
581     return failed.toArray(new HTableDescriptor[failed.size()]);
582   }
583 
584 
585   public void enableTable(final String tableName)
586   throws IOException {
587     enableTable(Bytes.toBytes(tableName));
588   }
589 
590   /**
591    * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
592    * and {@link #isTableEnabled(byte[])} instead.
593    * The table has to be in disabled state for it to be enabled.
594    * @param tableName name of the table
595    * @throws IOException if a remote or network exception occurs
596    * There could be couple types of IOException
597    * TableNotFoundException means the table doesn't exist.
598    * TableNotDisabledException means the table isn't in disabled state.
599    * @see #isTableEnabled(byte[])
600    * @see #disableTable(byte[])
601    * @see #enableTableAsync(byte[])
602    */
603   public void enableTable(final byte [] tableName)
604   throws IOException {
605     enableTableAsync(tableName);
606 
607     // Wait until all regions are enabled
608     boolean enabled = false;
609     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
610       enabled = isTableEnabled(tableName);
611       if (enabled) {
612         break;
613       }
614       long sleep = getPauseTime(tries);
615       if (LOG.isDebugEnabled()) {
616         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
617           "enabled in " + Bytes.toString(tableName));
618       }
619       try {
620         Thread.sleep(sleep);
621       } catch (InterruptedException e) {
622         Thread.currentThread().interrupt();
623         // Do this conversion rather than let it out because do not want to
624         // change the method signature.
625         throw new IOException("Interrupted", e);
626       }
627     }
628     if (!enabled) {
629       throw new IOException("Unable to enable table " +
630         Bytes.toString(tableName));
631     }
632     LOG.info("Enabled table " + Bytes.toString(tableName));
633   }
634 
635   public void enableTableAsync(final String tableName)
636   throws IOException {
637     enableTableAsync(Bytes.toBytes(tableName));
638   }
639 
640   /**
641    * Brings a table on-line (enables it).  Method returns immediately though
642    * enable of table may take some time to complete, especially if the table
643    * is large (All regions are opened as part of enabling process).  Check
644    * {@link #isTableEnabled(byte[])} to learn when table is fully online.  If
645    * table is taking too long to online, check server logs.
646    * @param tableName
647    * @throws IOException
648    * @since 0.90.0
649    */
650   public void enableTableAsync(final byte [] tableName)
651   throws IOException {
652     isMasterRunning();
653     try {
654       getMaster().enableTable(tableName);
655     } catch (RemoteException e) {
656       throw e.unwrapRemoteException();
657     }
658     LOG.info("Started enable of " + Bytes.toString(tableName));
659   }
660 
661   /**
662    * Enable tables matching the passed in pattern and wait on completion.
663    *
664    * Warning: Use this method carefully, there is no prompting and the effect is
665    * immediate. Consider using {@link #listTables(java.lang.String)} and
666    * {@link #enableTable(byte[])}
667    *
668    * @param regex The regular expression to match table names against
669    * @throws IOException
670    * @see #enableTables(java.util.regex.Pattern)
671    * @see #enableTable(java.lang.String)
672    */
673   public HTableDescriptor[] enableTables(String regex) throws IOException {
674     return enableTables(Pattern.compile(regex));
675   }
676 
677   /**
678    * Enable tables matching the passed in pattern and wait on completion.
679    *
680    * Warning: Use this method carefully, there is no prompting and the effect is
681    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
682    * {@link #enableTable(byte[])}
683    *
684    * @param pattern The pattern to match table names against
685    * @throws IOException
686    */
687   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
688     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
689     for (HTableDescriptor table : listTables(pattern)) {
690       if (isTableDisabled(table.getName())) {
691         try {
692           enableTable(table.getName());
693         } catch (IOException ex) {
694           LOG.info("Failed to enable table " + table.getNameAsString(), ex);
695           failed.add(table);
696         }
697       }
698     }
699     return failed.toArray(new HTableDescriptor[failed.size()]);
700   }
701 
702   public void disableTableAsync(final String tableName) throws IOException {
703     disableTableAsync(Bytes.toBytes(tableName));
704   }
705 
706   /**
707    * Starts the disable of a table.  If it is being served, the master
708    * will tell the servers to stop serving it.  This method returns immediately.
709    * The disable of a table can take some time if the table is large (all
710    * regions are closed as part of table disable operation).
711    * Call {@link #isTableDisabled(byte[])} to check for when disable completes.
712    * If table is taking too long to online, check server logs.
713    * @param tableName name of table
714    * @throws IOException if a remote or network exception occurs
715    * @see #isTableDisabled(byte[])
716    * @see #isTableEnabled(byte[])
717    * @since 0.90.0
718    */
719   public void disableTableAsync(final byte [] tableName) throws IOException {
720     isMasterRunning();
721     try {
722       getMaster().disableTable(tableName);
723     } catch (RemoteException e) {
724       throw e.unwrapRemoteException();
725     }
726     LOG.info("Started disable of " + Bytes.toString(tableName));
727   }
728 
729   public void disableTable(final String tableName)
730   throws IOException {
731     disableTable(Bytes.toBytes(tableName));
732   }
733 
734   /**
735    * Disable table and wait on completion.  May timeout eventually.  Use
736    * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
737    * instead.
738    * The table has to be in enabled state for it to be disabled.
739    * @param tableName
740    * @throws IOException
741    * There could be couple types of IOException
742    * TableNotFoundException means the table doesn't exist.
743    * TableNotEnabledException means the table isn't in enabled state.
744    */
745   public void disableTable(final byte [] tableName)
746   throws IOException {
747     disableTableAsync(tableName);
748     // Wait until table is disabled
749     boolean disabled = false;
750     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
751       disabled = isTableDisabled(tableName);
752       if (disabled) {
753         break;
754       }
755       long sleep = getPauseTime(tries);
756       if (LOG.isDebugEnabled()) {
757         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
758           "disabled in " + Bytes.toString(tableName));
759       }
760       try {
761         Thread.sleep(sleep);
762       } catch (InterruptedException e) {
763         // Do this conversion rather than let it out because do not want to
764         // change the method signature.
765         Thread.currentThread().interrupt();
766         throw new IOException("Interrupted", e);
767       }
768     }
769     if (!disabled) {
770       throw new RegionException("Retries exhausted, it took too long to wait"+
771         " for the table " + Bytes.toString(tableName) + " to be disabled.");
772     }
773     LOG.info("Disabled " + Bytes.toString(tableName));
774   }
775 
776   /**
777    * Disable tables matching the passed in pattern and wait on completion.
778    *
779    * Warning: Use this method carefully, there is no prompting and the effect is
780    * immediate. Consider using {@link #listTables(java.lang.String)} and
781    * {@link #disableTable(byte[])}
782    *
783    * @param regex The regular expression to match table names against
784    * @return Table descriptors for tables that couldn't be disabled
785    * @throws IOException
786    * @see #disableTables(java.util.regex.Pattern)
787    * @see #disableTable(java.lang.String)
788    */
789   public HTableDescriptor[] disableTables(String regex) throws IOException {
790     return disableTables(Pattern.compile(regex));
791   }
792 
793   /**
794    * Disable tables matching the passed in pattern and wait on completion.
795    *
796    * Warning: Use this method carefully, there is no prompting and the effect is
797    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
798    * {@link #disableTable(byte[])}
799    *
800    * @param pattern The pattern to match table names against
801    * @return Table descriptors for tables that couldn't be disabled
802    * @throws IOException
803    */
804   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
805     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
806     for (HTableDescriptor table : listTables(pattern)) {
807       if (isTableEnabled(table.getName())) {
808         try {
809           disableTable(table.getName());
810         } catch (IOException ex) {
811           LOG.info("Failed to disable table " + table.getNameAsString(), ex);
812           failed.add(table);
813         }
814       }
815     }
816     return failed.toArray(new HTableDescriptor[failed.size()]);
817   }
818 
819   /**
820    * @param tableName name of table to check
821    * @return true if table is on-line
822    * @throws IOException if a remote or network exception occurs
823    */
824   public boolean isTableEnabled(String tableName) throws IOException {
825     return isTableEnabled(Bytes.toBytes(tableName));
826   }
827   /**
828    * @param tableName name of table to check
829    * @return true if table is on-line
830    * @throws IOException if a remote or network exception occurs
831    */
832   public boolean isTableEnabled(byte[] tableName) throws IOException {
833     HTableDescriptor.isLegalTableName(tableName);
834     return connection.isTableEnabled(tableName);
835   }
836 
837   /**
838    * @param tableName name of table to check
839    * @return true if table is off-line
840    * @throws IOException if a remote or network exception occurs
841    */
842   public boolean isTableDisabled(final String tableName) throws IOException {
843     return isTableDisabled(Bytes.toBytes(tableName));
844   }
845 
846   /**
847    * @param tableName name of table to check
848    * @return true if table is off-line
849    * @throws IOException if a remote or network exception occurs
850    */
851   public boolean isTableDisabled(byte[] tableName) throws IOException {
852     HTableDescriptor.isLegalTableName(tableName);
853     return connection.isTableDisabled(tableName);
854   }
855 
856   /**
857    * @param tableName name of table to check
858    * @return true if all regions of the table are available
859    * @throws IOException if a remote or network exception occurs
860    */
861   public boolean isTableAvailable(byte[] tableName) throws IOException {
862     return connection.isTableAvailable(tableName);
863   }
864 
865   /**
866    * @param tableName name of table to check
867    * @return true if all regions of the table are available
868    * @throws IOException if a remote or network exception occurs
869    */
870   public boolean isTableAvailable(String tableName) throws IOException {
871     return connection.isTableAvailable(Bytes.toBytes(tableName));
872   }
873 
874   /**
875    * Get the status of alter command - indicates how many regions have received
876    * the updated schema Asynchronous operation.
877    *
878    * @param tableName
879    *          name of the table to get the status of
880    * @return Pair indicating the number of regions updated Pair.getFirst() is the
881    *         regions that are yet to be updated Pair.getSecond() is the total number
882    *         of regions of the table
883    * @throws IOException
884    *           if a remote or network exception occurs
885    */
886   public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
887   throws IOException {
888     HTableDescriptor.isLegalTableName(tableName);
889     try {
890       return getMaster().getAlterStatus(tableName);
891     } catch (RemoteException e) {
892       throw RemoteExceptionHandler.decodeRemoteException(e);
893     }
894   }
895 
896   /**
897    * Add a column to an existing table.
898    * Asynchronous operation.
899    *
900    * @param tableName name of the table to add column to
901    * @param column column descriptor of column to be added
902    * @throws IOException if a remote or network exception occurs
903    */
904   public void addColumn(final String tableName, HColumnDescriptor column)
905   throws IOException {
906     addColumn(Bytes.toBytes(tableName), column);
907   }
908 
909   /**
910    * Add a column to an existing table.
911    * Asynchronous operation.
912    *
913    * @param tableName name of the table to add column to
914    * @param column column descriptor of column to be added
915    * @throws IOException if a remote or network exception occurs
916    */
917   public void addColumn(final byte [] tableName, HColumnDescriptor column)
918   throws IOException {
919     HTableDescriptor.isLegalTableName(tableName);
920     try {
921       getMaster().addColumn(tableName, column);
922     } catch (RemoteException e) {
923       throw RemoteExceptionHandler.decodeRemoteException(e);
924     }
925   }
926 
927   /**
928    * Delete a column from a table.
929    * Asynchronous operation.
930    *
931    * @param tableName name of table
932    * @param columnName name of column to be deleted
933    * @throws IOException if a remote or network exception occurs
934    */
935   public void deleteColumn(final String tableName, final String columnName)
936   throws IOException {
937     deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName));
938   }
939 
940   /**
941    * Delete a column from a table.
942    * Asynchronous operation.
943    *
944    * @param tableName name of table
945    * @param columnName name of column to be deleted
946    * @throws IOException if a remote or network exception occurs
947    */
948   public void deleteColumn(final byte [] tableName, final byte [] columnName)
949   throws IOException {
950     try {
951       getMaster().deleteColumn(tableName, columnName);
952     } catch (RemoteException e) {
953       throw RemoteExceptionHandler.decodeRemoteException(e);
954     }
955   }
956 
957   /**
958    * Modify an existing column family on a table.
959    * Asynchronous operation.
960    *
961    * @param tableName name of table
962    * @param descriptor new column descriptor to use
963    * @throws IOException if a remote or network exception occurs
964    */
965   public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
966   throws IOException {
967     modifyColumn(Bytes.toBytes(tableName), descriptor);
968   }
969 
970   /**
971    * Modify an existing column family on a table.
972    * Asynchronous operation.
973    *
974    * @param tableName name of table
975    * @param descriptor new column descriptor to use
976    * @throws IOException if a remote or network exception occurs
977    */
978   public void modifyColumn(final byte [] tableName, HColumnDescriptor descriptor)
979   throws IOException {
980     try {
981       getMaster().modifyColumn(tableName, descriptor);
982     } catch (RemoteException re) {
983       // Convert RE exceptions in here; client shouldn't have to deal with them,
984       // at least w/ the type of exceptions that come out of this method:
985       // TableNotFoundException, etc.
986       throw RemoteExceptionHandler.decodeRemoteException(re);
987     }
988   }
989 
990   /**
991    * Close a region. For expert-admins.  Runs close on the regionserver.  The
992    * master will not be informed of the close.
993    * @param regionname region name to close
994    * @param serverName If supplied, we'll use this location rather than
995    * the one currently in <code>.META.</code>
996    * @throws IOException if a remote or network exception occurs
997    */
998   public void closeRegion(final String regionname, final String serverName)
999   throws IOException {
1000     closeRegion(Bytes.toBytes(regionname), serverName);
1001   }
1002 
1003   /**
1004    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1005    * master will not be informed of the close.
1006    * @param regionname region name to close
1007    * @param serverName The servername of the regionserver.  If passed null we
1008    * will use servername found in the .META. table. A server name
1009    * is made of host, port and startcode.  Here is an example:
1010    * <code> host187.example.com,60020,1289493121758</code>
1011    * @throws IOException if a remote or network exception occurs
1012    */
1013   public void closeRegion(final byte [] regionname, final String serverName)
1014   throws IOException {
1015     CatalogTracker ct = getCatalogTracker();
1016     try {
1017       if (serverName != null) {
1018         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1019         if (pair == null || pair.getFirst() == null) {
1020           LOG.info("No region in .META. for " +
1021             Bytes.toStringBinary(regionname) + "; pair=" + pair);
1022         } else {
1023           closeRegion(new ServerName(serverName), pair.getFirst());
1024         }
1025       } else {
1026         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1027         if (pair == null || pair.getSecond() == null) {
1028           LOG.info("No server in .META. for " +
1029             Bytes.toStringBinary(regionname) + "; pair=" + pair);
1030         } else {
1031           closeRegion(pair.getSecond(), pair.getFirst());
1032         }
1033       }
1034     } finally {
1035       cleanupCatalogTracker(ct);
1036     }
1037   }
1038 
1039   /**
1040    * For expert-admins. Runs close on the regionserver. Closes a region based on
1041    * the encoded region name. The region server name is mandatory. If the
1042    * servername is provided then based on the online regions in the specified
1043    * regionserver the specified region will be closed. The master will not be
1044    * informed of the close. Note that the regionname is the encoded regionname.
1045    * 
1046    * @param encodedRegionName
1047    *          The encoded region name; i.e. the hash that makes up the region
1048    *          name suffix: e.g. if regionname is
1049    *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
1050    *          , then the encoded region name is:
1051    *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
1052    * @param serverName
1053    *          The servername of the regionserver. A server name is made of host,
1054    *          port and startcode. This is mandatory. Here is an example:
1055    *          <code> host187.example.com,60020,1289493121758</code>
1056    * @return true if the region was closed, false if not.
1057    * @throws IOException
1058    *           if a remote or network exception occurs
1059    */
1060   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1061       final String serverName) throws IOException {
1062     byte[] encodedRegionNameInBytes = Bytes.toBytes(encodedRegionName);
1063     if (null == serverName || ("").equals(serverName.trim())) {
1064       throw new IllegalArgumentException(
1065           "The servername cannot be null or empty.");
1066     }
1067     ServerName sn = new ServerName(serverName);
1068     HRegionInterface rs = this.connection.getHRegionConnection(
1069         sn.getHostname(), sn.getPort());
1070     // Close the region without updating zk state.
1071     boolean isRegionClosed = rs.closeRegion(encodedRegionNameInBytes, false);
1072     if (false == isRegionClosed) {
1073       LOG.error("Not able to close the region " + encodedRegionName + ".");
1074     }
1075     return isRegionClosed;
1076   }
1077 
1078   /**
1079    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1080    * master will not be informed of the close.
1081    * @param sn
1082    * @param hri
1083    * @throws IOException
1084    */
1085   public void closeRegion(final ServerName sn, final HRegionInfo hri)
1086   throws IOException {
1087     HRegionInterface rs =
1088       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1089     // Close the region without updating zk state.
1090     rs.closeRegion(hri, false);
1091   }
1092 
1093   /**
1094    * Flush a table or an individual region.
1095    * Asynchronous operation.
1096    *
1097    * @param tableNameOrRegionName table or region to flush
1098    * @throws IOException if a remote or network exception occurs
1099    * @throws InterruptedException
1100    */
1101   public void flush(final String tableNameOrRegionName)
1102   throws IOException, InterruptedException {
1103     flush(Bytes.toBytes(tableNameOrRegionName));
1104   }
1105 
1106   /**
1107    * Flush a table or an individual region.
1108    * Asynchronous operation.
1109    *
1110    * @param tableNameOrRegionName table or region to flush
1111    * @throws IOException if a remote or network exception occurs
1112    * @throws InterruptedException
1113    */
1114   public void flush(final byte [] tableNameOrRegionName)
1115   throws IOException, InterruptedException {
1116     boolean isRegionName = isRegionName(tableNameOrRegionName);
1117     CatalogTracker ct = getCatalogTracker();
1118     try {
1119       if (isRegionName) {
1120         Pair<HRegionInfo, ServerName> pair =
1121           MetaReader.getRegion(ct, tableNameOrRegionName);
1122         if (pair == null || pair.getSecond() == null) {
1123           LOG.info("No server in .META. for " +
1124             Bytes.toStringBinary(tableNameOrRegionName) + "; pair=" + pair);
1125         } else {
1126           flush(pair.getSecond(), pair.getFirst());
1127         }
1128       } else {
1129         List<Pair<HRegionInfo, ServerName>> pairs =
1130           MetaReader.getTableRegionsAndLocations(ct,
1131               Bytes.toString(tableNameOrRegionName));
1132         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1133           if (pair.getFirst().isOffline()) continue;
1134           if (pair.getSecond() == null) continue;
1135           try {
1136             flush(pair.getSecond(), pair.getFirst());
1137           } catch (NotServingRegionException e) {
1138             if (LOG.isDebugEnabled()) {
1139               LOG.debug("Trying to flush " + pair.getFirst() + ": " +
1140                 StringUtils.stringifyException(e));
1141             }
1142           }
1143         }
1144       }
1145     } finally {
1146       cleanupCatalogTracker(ct);
1147     }
1148   }
1149 
1150   private void flush(final ServerName sn, final HRegionInfo hri)
1151   throws IOException {
1152     HRegionInterface rs =
1153       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1154     rs.flushRegion(hri);
1155   }
1156 
1157   /**
1158    * Compact a table or an individual region.
1159    * Asynchronous operation.
1160    *
1161    * @param tableNameOrRegionName table or region to compact
1162    * @throws IOException if a remote or network exception occurs
1163    * @throws InterruptedException
1164    */
1165   public void compact(final String tableNameOrRegionName)
1166   throws IOException, InterruptedException {
1167     compact(Bytes.toBytes(tableNameOrRegionName));
1168   }
1169 
1170   /**
1171    * Compact a table or an individual region.
1172    * Asynchronous operation.
1173    *
1174    * @param tableNameOrRegionName table or region to compact
1175    * @throws IOException if a remote or network exception occurs
1176    * @throws InterruptedException
1177    */
1178   public void compact(final byte [] tableNameOrRegionName)
1179   throws IOException, InterruptedException {
1180     compact(tableNameOrRegionName, false);
1181   }
1182 
1183   /**
1184    * Major compact a table or an individual region.
1185    * Asynchronous operation.
1186    *
1187    * @param tableNameOrRegionName table or region to major compact
1188    * @throws IOException if a remote or network exception occurs
1189    * @throws InterruptedException
1190    */
1191   public void majorCompact(final String tableNameOrRegionName)
1192   throws IOException, InterruptedException {
1193     majorCompact(Bytes.toBytes(tableNameOrRegionName));
1194   }
1195 
1196   /**
1197    * Major compact a table or an individual region.
1198    * Asynchronous operation.
1199    *
1200    * @param tableNameOrRegionName table or region to major compact
1201    * @throws IOException if a remote or network exception occurs
1202    * @throws InterruptedException
1203    */
1204   public void majorCompact(final byte [] tableNameOrRegionName)
1205   throws IOException, InterruptedException {
1206     compact(tableNameOrRegionName, true);
1207   }
1208 
1209   /**
1210    * Compact a table or an individual region.
1211    * Asynchronous operation.
1212    *
1213    * @param tableNameOrRegionName table or region to compact
1214    * @param major True if we are to do a major compaction.
1215    * @throws IOException if a remote or network exception occurs
1216    * @throws InterruptedException
1217    */
1218   private void compact(final byte [] tableNameOrRegionName, final boolean major)
1219   throws IOException, InterruptedException {
1220     CatalogTracker ct = getCatalogTracker();
1221     try {
1222       if (isRegionName(tableNameOrRegionName)) {
1223         Pair<HRegionInfo, ServerName> pair =
1224           MetaReader.getRegion(ct, tableNameOrRegionName);
1225         if (pair == null || pair.getSecond() == null) {
1226           LOG.info("No server in .META. for " +
1227             Bytes.toStringBinary(tableNameOrRegionName) + "; pair=" + pair);
1228         } else {
1229           compact(pair.getSecond(), pair.getFirst(), major);
1230         }
1231       } else {
1232         List<Pair<HRegionInfo, ServerName>> pairs =
1233           MetaReader.getTableRegionsAndLocations(ct,
1234               Bytes.toString(tableNameOrRegionName));
1235         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1236           if (pair.getFirst().isOffline()) continue;
1237           if (pair.getSecond() == null) continue;
1238           try {
1239             compact(pair.getSecond(), pair.getFirst(), major);
1240           } catch (NotServingRegionException e) {
1241             if (LOG.isDebugEnabled()) {
1242               LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
1243                 pair.getFirst() + ": " +
1244                 StringUtils.stringifyException(e));
1245             }
1246           }
1247         }
1248       }
1249     } finally {
1250       cleanupCatalogTracker(ct);
1251     }
1252   }
1253 
1254   private void compact(final ServerName sn, final HRegionInfo hri,
1255       final boolean major)
1256   throws IOException {
1257     HRegionInterface rs =
1258       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1259     rs.compactRegion(hri, major);
1260   }
1261 
1262   /**
1263    * Move the region <code>r</code> to <code>dest</code>.
1264    * @param encodedRegionName The encoded region name; i.e. the hash that makes
1265    * up the region name suffix: e.g. if regionname is
1266    * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
1267    * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
1268    * @param destServerName The servername of the destination regionserver.  If
1269    * passed the empty byte array we'll assign to a random server.  A server name
1270    * is made of host, port and startcode.  Here is an example:
1271    * <code> host187.example.com,60020,1289493121758</code>
1272    * @throws UnknownRegionException Thrown if we can't find a region named
1273    * <code>encodedRegionName</code>
1274    * @throws ZooKeeperConnectionException
1275    * @throws MasterNotRunningException
1276    */
1277   public void move(final byte [] encodedRegionName, final byte [] destServerName)
1278   throws UnknownRegionException, MasterNotRunningException, ZooKeeperConnectionException {
1279     getMaster().move(encodedRegionName, destServerName);
1280   }
1281 
1282   /**
1283    * @param regionName
1284    *          Region name to assign.
1285    * @throws MasterNotRunningException
1286    * @throws ZooKeeperConnectionException
1287    * @throws IOException
1288    */
1289   public void assign(final byte[] regionName) throws MasterNotRunningException,
1290       ZooKeeperConnectionException, IOException {
1291     getMaster().assign(regionName);
1292   }
1293 
1294   /**
1295    * Unassign a region from current hosting regionserver.  Region will then be
1296    * assigned to a regionserver chosen at random.  Region could be reassigned
1297    * back to the same server.  Use {@link #move(byte[], byte[])} if you want
1298    * to control the region movement.
1299    * @param regionName Region to unassign. Will clear any existing RegionPlan
1300    * if one found.
1301    * @param force If true, force unassign (Will remove region from
1302    * regions-in-transition too if present. If results in double assignment
1303    * use hbck -fix to resolve. To be used by experts).
1304    * @throws MasterNotRunningException
1305    * @throws ZooKeeperConnectionException
1306    * @throws IOException
1307    */
1308   public void unassign(final byte [] regionName, final boolean force)
1309   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
1310     getMaster().unassign(regionName, force);
1311   }
1312 
1313   /**
1314    * Turn the load balancer on or off.
1315    * @param b If true, enable balancer. If false, disable balancer.
1316    * @return Previous balancer value
1317    */
1318   public boolean balanceSwitch(final boolean b)
1319   throws MasterNotRunningException, ZooKeeperConnectionException {
1320     return getMaster().balanceSwitch(b);
1321   }
1322 
1323   /**
1324    * Invoke the balancer.  Will run the balancer and if regions to move, it will
1325    * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
1326    * logs.
1327    * @return True if balancer ran, false otherwise.
1328    */
1329   public boolean balancer()
1330   throws MasterNotRunningException, ZooKeeperConnectionException {
1331     return getMaster().balance();
1332   }
1333 
1334   /**
1335    * Split a table or an individual region.
1336    * Asynchronous operation.
1337    *
1338    * @param tableNameOrRegionName table or region to split
1339    * @throws IOException if a remote or network exception occurs
1340    * @throws InterruptedException
1341    */
1342   public void split(final String tableNameOrRegionName)
1343   throws IOException, InterruptedException {
1344     split(Bytes.toBytes(tableNameOrRegionName));
1345   }
1346 
1347   /**
1348    * Split a table or an individual region.  Implicitly finds an optimal split
1349    * point.  Asynchronous operation.
1350    *
1351    * @param tableNameOrRegionName table to region to split
1352    * @throws IOException if a remote or network exception occurs
1353    * @throws InterruptedException
1354    */
1355   public void split(final byte [] tableNameOrRegionName)
1356   throws IOException, InterruptedException {
1357     split(tableNameOrRegionName, null);
1358   }
1359 
1360   public void split(final String tableNameOrRegionName,
1361     final String splitPoint) throws IOException, InterruptedException {
1362     split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
1363   }
1364 
1365   /**
1366    * Split a table or an individual region.
1367    * Asynchronous operation.
1368    *
1369    * @param tableNameOrRegionName table to region to split
1370    * @param splitPoint the explicit position to split on
1371    * @throws IOException if a remote or network exception occurs
1372    * @throws InterruptedException interrupt exception occurred
1373    */
1374   public void split(final byte [] tableNameOrRegionName,
1375       final byte [] splitPoint) throws IOException, InterruptedException {
1376     CatalogTracker ct = getCatalogTracker();
1377     try {
1378       if (isRegionName(tableNameOrRegionName)) {
1379         // Its a possible region name.
1380         Pair<HRegionInfo, ServerName> pair =
1381           MetaReader.getRegion(ct, tableNameOrRegionName);
1382         if (pair == null || pair.getSecond() == null) {
1383           LOG.info("No server in .META. for " +
1384             Bytes.toStringBinary(tableNameOrRegionName) + "; pair=" + pair);
1385         } else {
1386           split(pair.getSecond(), pair.getFirst(), splitPoint);
1387         }
1388       } else {
1389         List<Pair<HRegionInfo, ServerName>> pairs =
1390           MetaReader.getTableRegionsAndLocations(ct,
1391               Bytes.toString(tableNameOrRegionName));
1392         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1393           // May not be a server for a particular row
1394           if (pair.getSecond() == null) continue;
1395           HRegionInfo r = pair.getFirst();
1396           // check for parents
1397           if (r.isSplitParent()) continue;
1398           // if a split point given, only split that particular region
1399           if (splitPoint != null && !r.containsRow(splitPoint)) continue;
1400           // call out to region server to do split now
1401           split(pair.getSecond(), pair.getFirst(), splitPoint);
1402         }
1403       }
1404     } finally {
1405       cleanupCatalogTracker(ct);
1406     }
1407   }
1408 
1409   private void split(final ServerName sn, final HRegionInfo hri,
1410       byte[] splitPoint) throws IOException {
1411     HRegionInterface rs =
1412       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1413     rs.splitRegion(hri, splitPoint);
1414   }
1415 
1416   /**
1417    * Modify an existing table, more IRB friendly version.
1418    * Asynchronous operation.  This means that it may be a while before your
1419    * schema change is updated across all of the table.
1420    *
1421    * @param tableName name of table.
1422    * @param htd modified description of the table
1423    * @throws IOException if a remote or network exception occurs
1424    */
1425   public void modifyTable(final byte [] tableName, HTableDescriptor htd)
1426   throws IOException {
1427     try {
1428       getMaster().modifyTable(tableName, htd);
1429     } catch (RemoteException re) {
1430       // Convert RE exceptions in here; client shouldn't have to deal with them,
1431       // at least w/ the type of exceptions that come out of this method:
1432       // TableNotFoundException, etc.
1433       throw RemoteExceptionHandler.decodeRemoteException(re);
1434     }
1435   }
1436 
1437   /**
1438    * @param tableNameOrRegionName Name of a table or name of a region.
1439    * @return True if <code>tableNameOrRegionName</code> is *possibly* a region
1440    * name else false if a verified tablename (we call {@link #tableExists(byte[])};
1441    * else we throw an exception.
1442    * @throws IOException
1443    */
1444   private boolean isRegionName(final byte [] tableNameOrRegionName)
1445   throws IOException {
1446     if (tableNameOrRegionName == null) {
1447       throw new IllegalArgumentException("Pass a table name or region name");
1448     }
1449     return !tableExists(tableNameOrRegionName);
1450   }
1451 
1452   /**
1453    * Shuts down the HBase cluster
1454    * @throws IOException if a remote or network exception occurs
1455    */
1456   public synchronized void shutdown() throws IOException {
1457     isMasterRunning();
1458     try {
1459       getMaster().shutdown();
1460     } catch (RemoteException e) {
1461       throw RemoteExceptionHandler.decodeRemoteException(e);
1462     }
1463   }
1464 
1465   /**
1466    * Shuts down the current HBase master only.
1467    * Does not shutdown the cluster.
1468    * @see #shutdown()
1469    * @throws IOException if a remote or network exception occurs
1470    */
1471   public synchronized void stopMaster() throws IOException {
1472     isMasterRunning();
1473     try {
1474       getMaster().stopMaster();
1475     } catch (RemoteException e) {
1476       throw RemoteExceptionHandler.decodeRemoteException(e);
1477     }
1478   }
1479 
1480   /**
1481    * Stop the designated regionserver
1482    * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
1483    * <code>example.org:1234</code>
1484    * @throws IOException if a remote or network exception occurs
1485    */
1486   public synchronized void stopRegionServer(final String hostnamePort)
1487   throws IOException {
1488     String hostname = Addressing.parseHostname(hostnamePort);
1489     int port = Addressing.parsePort(hostnamePort);
1490     HRegionInterface rs =
1491       this.connection.getHRegionConnection(hostname, port);
1492     rs.stop("Called by admin client " + this.connection.toString());
1493   }
1494 
1495   /**
1496    * @return cluster status
1497    * @throws IOException if a remote or network exception occurs
1498    */
1499   public ClusterStatus getClusterStatus() throws IOException {
1500     return getMaster().getClusterStatus();
1501   }
1502 
1503   private HRegionLocation getFirstMetaServerForTable(final byte [] tableName)
1504   throws IOException {
1505     return connection.locateRegion(HConstants.META_TABLE_NAME,
1506       HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
1507   }
1508 
1509   /**
1510    * @return Configuration used by the instance.
1511    */
1512   public Configuration getConfiguration() {
1513     return this.conf;
1514   }
1515 
1516   /**
1517    * Check to see if HBase is running. Throw an exception if not.
1518    *
1519    * @param conf system configuration
1520    * @throws MasterNotRunningException if the master is not running
1521    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
1522    */
1523   public static void checkHBaseAvailable(Configuration conf)
1524   throws MasterNotRunningException, ZooKeeperConnectionException {
1525     Configuration copyOfConf = HBaseConfiguration.create(conf);
1526     copyOfConf.setInt("hbase.client.retries.number", 1);
1527     HBaseAdmin admin = new HBaseAdmin(copyOfConf);
1528     try {
1529       admin.close();
1530     } catch (IOException ioe) {
1531       admin.LOG.info("Failed to close connection", ioe);
1532     }
1533   }
1534 
1535   /**
1536    * get the regions of a given table.
1537    *
1538    * @param tableName the name of the table
1539    * @return Ordered list of {@link HRegionInfo}.
1540    * @throws IOException
1541    */  
1542   public List<HRegionInfo> getTableRegions(final byte[] tableName)
1543   throws IOException {
1544     CatalogTracker ct = getCatalogTracker();
1545     List<HRegionInfo> Regions = null;
1546     try {
1547       Regions = MetaReader.getTableRegions(ct, tableName, true);
1548     } finally {
1549       cleanupCatalogTracker(ct);
1550     }
1551     return Regions;
1552   }
1553   
1554   public void close() throws IOException {
1555     if (this.connection != null) {
1556       this.connection.close();
1557     }
1558   }
1559 
1560  /**
1561  * Get tableDescriptors
1562  * @param tableNames List of table names
1563  * @return HTD[] the tableDescriptor
1564  * @throws IOException if a remote or network exception occurs
1565  */
1566   public HTableDescriptor[] getTableDescriptors(List<String> tableNames)
1567   throws IOException {
1568     return this.connection.getHTableDescriptors(tableNames);
1569   }
1570 
1571   /**
1572    * Roll the log writer. That is, start writing log messages to a new file.
1573    * 
1574    * @param serverName
1575    *          The servername of the regionserver. A server name is made of host,
1576    *          port and startcode. This is mandatory. Here is an example:
1577    *          <code> host187.example.com,60020,1289493121758</code>
1578    * @return If lots of logs, flush the returned regions so next time through
1579    * we can clean logs. Returns null if nothing to flush.  Names are actual
1580    * region names as returned by {@link HRegionInfo#getEncodedName()}  
1581    * @throws IOException if a remote or network exception occurs
1582    * @throws FailedLogCloseException
1583    */
1584  public synchronized  byte[][] rollHLogWriter(String serverName)
1585       throws IOException, FailedLogCloseException {
1586     ServerName sn = new ServerName(serverName);
1587     HRegionInterface rs = this.connection.getHRegionConnection(
1588         sn.getHostname(), sn.getPort());
1589     return rs.rollHLogWriter();
1590   }
1591 
1592   public String[] getMasterCoprocessors() {
1593     try {
1594       return getClusterStatus().getMasterCoprocessors();
1595     } catch (IOException e) {
1596       LOG.error("Could not getClusterStatus()",e);
1597       return null;
1598     }
1599   }
1600 }