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.Proxy;
26  import java.lang.reflect.UndeclaredThrowableException;
27  import java.net.SocketTimeoutException;
28  import java.util.ArrayList;
29  import java.util.Arrays;
30  import java.util.LinkedList;
31  import java.util.List;
32  import java.util.concurrent.atomic.AtomicInteger;
33  import java.util.concurrent.atomic.AtomicReference;
34  import java.util.regex.Pattern;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.conf.Configuration;
39  import org.apache.hadoop.hbase.Abortable;
40  import org.apache.hadoop.hbase.ClusterStatus;
41  import org.apache.hadoop.hbase.HBaseConfiguration;
42  import org.apache.hadoop.hbase.HColumnDescriptor;
43  import org.apache.hadoop.hbase.HConstants;
44  import org.apache.hadoop.hbase.HRegionInfo;
45  import org.apache.hadoop.hbase.HRegionLocation;
46  import org.apache.hadoop.hbase.HTableDescriptor;
47  import org.apache.hadoop.hbase.MasterNotRunningException;
48  import org.apache.hadoop.hbase.NotServingRegionException;
49  import org.apache.hadoop.hbase.RegionException;
50  import org.apache.hadoop.hbase.RemoteExceptionHandler;
51  import org.apache.hadoop.hbase.ServerName;
52  import org.apache.hadoop.hbase.TableExistsException;
53  import org.apache.hadoop.hbase.TableNotEnabledException;
54  import org.apache.hadoop.hbase.TableNotFoundException;
55  import org.apache.hadoop.hbase.UnknownRegionException;
56  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
57  import org.apache.hadoop.hbase.catalog.CatalogTracker;
58  import org.apache.hadoop.hbase.catalog.MetaReader;
59  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
60  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
61  import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
62  import org.apache.hadoop.hbase.ipc.HMasterInterface;
63  import org.apache.hadoop.hbase.ipc.HRegionInterface;
64  import org.apache.hadoop.hbase.ipc.MasterExecRPCInvoker;
65  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
66  import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest.CompactionState;
67  import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
68  import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
69  import org.apache.hadoop.hbase.snapshot.HSnapshotDescription;
70  import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
71  import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
72  import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
73  import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
74  import org.apache.hadoop.hbase.util.Addressing;
75  import org.apache.hadoop.hbase.util.Bytes;
76  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
77  import org.apache.hadoop.hbase.util.Pair;
78  import org.apache.hadoop.ipc.RemoteException;
79  import org.apache.hadoop.util.StringUtils;
80  
81  import com.google.protobuf.ServiceException;
82  
83  /**
84   * Provides an interface to manage HBase database table metadata + general
85   * administrative functions.  Use HBaseAdmin to create, drop, list, enable and
86   * disable tables. Use it also to add and drop table column families.
87   *
88   * <p>See {@link HTable} to add, update, and delete data from an individual table.
89   * <p>Currently HBaseAdmin instances are not expected to be long-lived.  For
90   * example, an HBaseAdmin instance will not ride over a Master restart.
91   */
92  public class HBaseAdmin implements Abortable, Closeable {
93    private final Log LOG = LogFactory.getLog(this.getClass().getName());
94  //  private final HConnection connection;
95    private HConnection connection;
96    private volatile Configuration conf;
97    private final long pause;
98    private final int numRetries;
99    // Some operations can take a long time such as disable of big table.
100   // numRetries is for 'normal' stuff... Mutliply by this factor when
101   // want to wait a long time.
102   private final int retryLongerMultiplier;
103   private boolean aborted;
104 
105   private static volatile boolean synchronousBalanceSwitchSupported = true;
106 
107   /**
108    * Constructor
109    *
110    * @param c Configuration object
111    * @throws MasterNotRunningException if the master is not running
112    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
113    */
114   public HBaseAdmin(Configuration c)
115   throws MasterNotRunningException, ZooKeeperConnectionException {
116     this.conf = HBaseConfiguration.create(c);
117     this.connection = HConnectionManager.getConnection(this.conf);
118     this.pause = this.conf.getLong("hbase.client.pause", 1000);
119     this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
120     this.retryLongerMultiplier = this.conf.getInt(
121         "hbase.client.retries.longer.multiplier", 10);
122 
123     int tries = 0;
124     while ( true ){
125       try {
126 
127         this.connection.getMaster();
128         return;
129 
130       } catch (MasterNotRunningException mnre) {
131         HConnectionManager.deleteStaleConnection(this.connection);
132         this.connection = HConnectionManager.getConnection(this.conf);
133       }
134 
135       tries++;
136       if (tries >= numRetries) {
137         // we should delete connection between client and zookeeper
138         HConnectionManager.deleteStaleConnection(this.connection);
139         throw new MasterNotRunningException("Retried " + numRetries + " times");
140       }
141 
142       try {
143         Thread.sleep(getPauseTime(tries));
144       } catch (InterruptedException e) {
145         Thread.currentThread().interrupt();
146         // we should delete connection between client and zookeeper
147         HConnectionManager.deleteStaleConnection(this.connection);
148         throw new MasterNotRunningException(
149           "Interrupted after "+tries+" tries");
150       }
151     }
152   }
153 
154  /**
155    * Constructor for externally managed HConnections.
156    * This constructor fails fast if the HMaster is not running.
157    * The HConnection can be re-used again in another attempt.
158    * This constructor fails fast.
159    *
160    * @param connection The HConnection instance to use
161    * @throws MasterNotRunningException if the master is not running
162    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
163    */
164   public HBaseAdmin(HConnection connection)
165       throws MasterNotRunningException, ZooKeeperConnectionException {
166     this.conf = connection.getConfiguration();
167     this.connection = connection;
168 
169     this.pause = this.conf.getLong("hbase.client.pause", 1000);
170     this.numRetries = this.conf.getInt("hbase.client.retries.number", 10);
171     this.retryLongerMultiplier = this.conf.getInt(
172         "hbase.client.retries.longer.multiplier", 10);
173 
174     this.connection.getMaster();
175   }
176 
177   /**
178    * @return A new CatalogTracker instance; call {@link #cleanupCatalogTracker(CatalogTracker)}
179    * to cleanup the returned catalog tracker.
180    * @throws ZooKeeperConnectionException
181    * @throws IOException
182    * @see #cleanupCatalogTracker(CatalogTracker)
183    */
184   private synchronized CatalogTracker getCatalogTracker()
185   throws ZooKeeperConnectionException, IOException {
186     CatalogTracker ct = null;
187     try {
188       ct = new CatalogTracker(this.conf);
189       ct.start();
190     } catch (InterruptedException e) {
191       // Let it out as an IOE for now until we redo all so tolerate IEs
192       Thread.currentThread().interrupt();
193       throw new IOException("Interrupted", e);
194     }
195     return ct;
196   }
197 
198   private void cleanupCatalogTracker(final CatalogTracker ct) {
199     ct.stop();
200   }
201 
202   @Override
203   public void abort(String why, Throwable e) {
204     // Currently does nothing but throw the passed message and exception
205     this.aborted = true;
206     throw new RuntimeException(why, e);
207   }
208 
209   @Override
210   public boolean isAborted(){
211     return this.aborted;
212   }
213 
214   /** @return HConnection used by this object. */
215   public HConnection getConnection() {
216     return connection;
217   }
218 
219   /**
220    * Get a connection to the currently set master.
221    * @return proxy connection to master server for this instance
222    * @throws MasterNotRunningException if the master is not running
223    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
224    * @deprecated  Master is an implementation detail for HBaseAdmin.
225    * Deprecated in HBase 0.94
226    */
227   @Deprecated
228   public HMasterInterface getMaster()
229   throws MasterNotRunningException, ZooKeeperConnectionException {
230     return this.connection.getMaster();
231   }
232 
233   /** @return - true if the master server is running
234    * @throws ZooKeeperConnectionException
235    * @throws MasterNotRunningException */
236   public boolean isMasterRunning()
237   throws MasterNotRunningException, ZooKeeperConnectionException {
238     return this.connection.isMasterRunning();
239   }
240 
241   /**
242    * @param tableName Table to check.
243    * @return True if table exists already.
244    * @throws IOException
245    */
246   public boolean tableExists(final String tableName)
247   throws IOException {
248     boolean b = false;
249     CatalogTracker ct = getCatalogTracker();
250     try {
251       b = MetaReader.tableExists(ct, tableName);
252     } finally {
253       cleanupCatalogTracker(ct);
254     }
255     return b;
256   }
257 
258   /**
259    * @param tableName Table to check.
260    * @return True if table exists already.
261    * @throws IOException
262    */
263   public boolean tableExists(final byte [] tableName)
264   throws IOException {
265     return tableExists(Bytes.toString(tableName));
266   }
267 
268   /**
269    * List all the userspace tables.  In other words, scan the META table.
270    *
271    * If we wanted this to be really fast, we could implement a special
272    * catalog table that just contains table names and their descriptors.
273    * Right now, it only exists as part of the META table's region info.
274    *
275    * @return - returns an array of HTableDescriptors
276    * @throws IOException if a remote or network exception occurs
277    */
278   public HTableDescriptor[] listTables() throws IOException {
279     return this.connection.listTables();
280   }
281 
282   /**
283    * List all the userspace tables matching the given pattern.
284    *
285    * @param pattern The compiled regular expression to match against
286    * @return - returns an array of HTableDescriptors
287    * @throws IOException if a remote or network exception occurs
288    * @see #listTables()
289    */
290   public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
291     List<HTableDescriptor> matched = new LinkedList<HTableDescriptor>();
292     HTableDescriptor[] tables = listTables();
293     for (HTableDescriptor table : tables) {
294       if (pattern.matcher(table.getNameAsString()).matches()) {
295         matched.add(table);
296       }
297     }
298     return matched.toArray(new HTableDescriptor[matched.size()]);
299   }
300 
301   /**
302    * List all the userspace tables matching the given regular expression.
303    *
304    * @param regex The regular expression to match against
305    * @return - returns an array of HTableDescriptors
306    * @throws IOException if a remote or network exception occurs
307    * @see #listTables(java.util.regex.Pattern)
308    */
309   public HTableDescriptor[] listTables(String regex) throws IOException {
310     return listTables(Pattern.compile(regex));
311   }
312 
313 
314   /**
315    * Method for getting the tableDescriptor
316    * @param tableName as a byte []
317    * @return the tableDescriptor
318    * @throws TableNotFoundException
319    * @throws IOException if a remote or network exception occurs
320    */
321   public HTableDescriptor getTableDescriptor(final byte [] tableName)
322   throws TableNotFoundException, IOException {
323     return this.connection.getHTableDescriptor(tableName);
324   }
325 
326   private long getPauseTime(int tries) {
327     int triesCount = tries;
328     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
329       triesCount = HConstants.RETRY_BACKOFF.length - 1;
330     }
331     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
332   }
333 
334   /**
335    * Creates a new table.
336    * Synchronous operation.
337    *
338    * @param desc table descriptor for table
339    *
340    * @throws IllegalArgumentException if the table name is reserved
341    * @throws MasterNotRunningException if master is not running
342    * @throws TableExistsException if table already exists (If concurrent
343    * threads, the table may have been created between test-for-existence
344    * and attempt-at-creation).
345    * @throws IOException if a remote or network exception occurs
346    */
347   public void createTable(HTableDescriptor desc)
348   throws IOException {
349     createTable(desc, null);
350   }
351 
352   /**
353    * Creates a new table with the specified number of regions.  The start key
354    * specified will become the end key of the first region of the table, and
355    * the end key specified will become the start key of the last region of the
356    * table (the first region has a null start key and the last region has a
357    * null end key).
358    *
359    * BigInteger math will be used to divide the key range specified into
360    * enough segments to make the required number of total regions.
361    *
362    * Synchronous operation.
363    *
364    * @param desc table descriptor for table
365    * @param startKey beginning of key range
366    * @param endKey end of key range
367    * @param numRegions the total number of regions to create
368    *
369    * @throws IllegalArgumentException if the table name is reserved
370    * @throws MasterNotRunningException if master is not running
371    * @throws TableExistsException if table already exists (If concurrent
372    * threads, the table may have been created between test-for-existence
373    * and attempt-at-creation).
374    * @throws IOException
375    */
376   public void createTable(HTableDescriptor desc, byte [] startKey,
377       byte [] endKey, int numRegions)
378   throws IOException {
379     HTableDescriptor.isLegalTableName(desc.getName());
380     if(numRegions < 3) {
381       throw new IllegalArgumentException("Must create at least three regions");
382     } else if(Bytes.compareTo(startKey, endKey) >= 0) {
383       throw new IllegalArgumentException("Start key must be smaller than end key");
384     }
385     if (numRegions == 3) {
386       createTable(desc, new byte[][] { startKey, endKey });
387       return;
388     }
389     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
390     if(splitKeys == null || splitKeys.length != numRegions - 1) {
391       throw new IllegalArgumentException("Unable to split key range into enough regions");
392     }
393     createTable(desc, splitKeys);
394   }
395 
396   /**
397    * Creates a new table with an initial set of empty regions defined by the
398    * specified split keys.  The total number of regions created will be the
399    * number of split keys plus one. Synchronous operation.
400    * Note : Avoid passing empty split key.
401    *
402    * @param desc table descriptor for table
403    * @param splitKeys array of split keys for the initial regions of the table
404    *
405    * @throws IllegalArgumentException if the table name is reserved, if the split keys
406    * are repeated and if the split key has empty byte array.
407    * @throws MasterNotRunningException if master is not running
408    * @throws TableExistsException if table already exists (If concurrent
409    * threads, the table may have been created between test-for-existence
410    * and attempt-at-creation).
411    * @throws IOException
412    */
413   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
414   throws IOException {
415     HTableDescriptor.isLegalTableName(desc.getName());
416     try {
417       createTableAsync(desc, splitKeys);
418     } catch (SocketTimeoutException ste) {
419       LOG.warn("Creating " + desc.getNameAsString() + " took too long", ste);
420     }
421     int numRegs = splitKeys == null ? 1 : splitKeys.length + 1;
422     int prevRegCount = 0;
423     boolean doneWithMetaScan = false;
424     for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
425       ++tries) {
426       if (!doneWithMetaScan) {
427         // Wait for new table to come on-line
428         final AtomicInteger actualRegCount = new AtomicInteger(0);
429         MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
430           @Override
431           public boolean processRow(Result rowResult) throws IOException {
432             if (rowResult == null || rowResult.size() <= 0) {
433               return true;
434             }
435             HRegionInfo info = MetaReader.parseHRegionInfoFromCatalogResult(
436               rowResult, HConstants.REGIONINFO_QUALIFIER);
437             if (info == null) {
438               LOG.warn("No serialized HRegionInfo in " + rowResult);
439               return true;
440             }
441             if (!(Bytes.equals(info.getTableName(), desc.getName()))) {
442               return false;
443             }
444             String hostAndPort = null;
445             byte [] value = rowResult.getValue(HConstants.CATALOG_FAMILY,
446               HConstants.SERVER_QUALIFIER);
447             // Make sure that regions are assigned to server
448             if (value != null && value.length > 0) {
449               hostAndPort = Bytes.toString(value);
450             }
451             if (!(info.isOffline() || info.isSplit()) && hostAndPort != null) {
452               actualRegCount.incrementAndGet();
453             }
454             return true;
455           }
456         };
457         MetaScanner.metaScan(conf, visitor, desc.getName());
458         if (actualRegCount.get() != numRegs) {
459           if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
460             throw new RegionOfflineException("Only " + actualRegCount.get() +
461               " of " + numRegs + " regions are online; retries exhausted.");
462           }
463           try { // Sleep
464             Thread.sleep(getPauseTime(tries));
465           } catch (InterruptedException e) {
466             throw new InterruptedIOException("Interrupted when opening" +
467               " regions; " + actualRegCount.get() + " of " + numRegs +
468               " regions processed so far");
469           }
470           if (actualRegCount.get() > prevRegCount) { // Making progress
471             prevRegCount = actualRegCount.get();
472             tries = -1;
473           }
474         } else {
475           doneWithMetaScan = true;
476           tries = -1;
477         }
478       } else if (isTableEnabled(desc.getName())) {
479         return;
480       } else {
481         try { // Sleep
482           Thread.sleep(getPauseTime(tries));
483         } catch (InterruptedException e) {
484           throw new InterruptedIOException("Interrupted when waiting" +
485             " for table to be enabled; meta scan was done");
486         }
487       }
488     }
489     throw new TableNotEnabledException(
490       "Retries exhausted while still waiting for table: "
491       + desc.getNameAsString() + " to be enabled");
492   }
493 
494   /**
495    * Creates a new table but does not block and wait for it to come online.
496    * Asynchronous operation.  To check if the table exists, use
497    * {@link: #isTableAvailable} -- it is not safe to create an HTable
498    * instance to this table before it is available.
499    * Note : Avoid passing empty split key.
500    * @param desc table descriptor for table
501    *
502    * @throws IllegalArgumentException Bad table name, if the split keys
503    * are repeated and if the split key has empty byte array.
504    * @throws MasterNotRunningException if master is not running
505    * @throws TableExistsException if table already exists (If concurrent
506    * threads, the table may have been created between test-for-existence
507    * and attempt-at-creation).
508    * @throws IOException
509    */
510   public void createTableAsync(HTableDescriptor desc, byte [][] splitKeys)
511   throws IOException {
512     HTableDescriptor.isLegalTableName(desc.getName());
513     if(splitKeys != null && splitKeys.length > 0) {
514       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
515       // Verify there are no duplicate split keys
516       byte [] lastKey = null;
517       for(byte [] splitKey : splitKeys) {
518         if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
519           throw new IllegalArgumentException(
520               "Empty split key must not be passed in the split keys.");
521         }
522         if(lastKey != null && Bytes.equals(splitKey, lastKey)) {
523           throw new IllegalArgumentException("All split keys must be unique, " +
524             "found duplicate: " + Bytes.toStringBinary(splitKey) +
525             ", " + Bytes.toStringBinary(lastKey));
526         }
527         lastKey = splitKey;
528       }
529     }
530     try {
531       getMaster().createTable(desc, splitKeys);
532     } catch (RemoteException e) {
533       throw e.unwrapRemoteException();
534     }
535   }
536 
537   /**
538    * Deletes a table.
539    * Synchronous operation.
540    *
541    * @param tableName name of table to delete
542    * @throws IOException if a remote or network exception occurs
543    */
544   public void deleteTable(final String tableName) throws IOException {
545     deleteTable(Bytes.toBytes(tableName));
546   }
547 
548   /**
549    * Deletes a table.
550    * Synchronous operation.
551    *
552    * @param tableName name of table to delete
553    * @throws IOException if a remote or network exception occurs
554    */
555   public void deleteTable(final byte [] tableName) throws IOException {
556     isMasterRunning();
557     HTableDescriptor.isLegalTableName(tableName);
558     HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
559     boolean tableExists = true;
560     try {
561       getMaster().deleteTable(tableName);
562     } catch (RemoteException e) {
563       throw RemoteExceptionHandler.decodeRemoteException(e);
564     }
565     // Wait until all regions deleted
566     HRegionInterface server =
567       connection.getHRegionConnection(firstMetaServer.getHostname(), firstMetaServer.getPort());
568     List<String> tableNameAsList = new ArrayList<String>(1);
569     tableNameAsList.add(Bytes.toString(tableName));
570     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
571       long scannerId = -1L;
572       try {
573 
574         Scan scan = MetaReader.getScanForTableName(tableName);
575         scan.addColumn(HConstants.CATALOG_FAMILY,
576             HConstants.REGIONINFO_QUALIFIER);
577         scannerId = server.openScanner(
578           firstMetaServer.getRegionInfo().getRegionName(), scan);
579         // Get a batch at a time.
580         Result values = server.next(scannerId);
581 
582         // let us wait until .META. table is updated and
583         // HMaster removes the table from its HTableDescriptors
584         if (values == null) {
585           tableExists = false;
586           HTableDescriptor[] htds = getMaster().getHTableDescriptors(tableNameAsList);
587           tableExists = (htds != null && htds.length > 0);
588           if (!tableExists) {
589             break;
590           }
591         }
592       } catch (IOException ex) {
593         if(tries == numRetries - 1) {           // no more tries left
594           if (ex instanceof RemoteException) {
595             ex = RemoteExceptionHandler.decodeRemoteException((RemoteException) ex);
596           }
597           throw ex;
598         }
599       } finally {
600         if (scannerId != -1L) {
601           try {
602             server.close(scannerId);
603           } catch (Exception ex) {
604             LOG.warn(ex);
605           }
606         }
607       }
608       try {
609         Thread.sleep(getPauseTime(tries));
610       } catch (InterruptedException e) {
611         // continue
612       }
613     }
614 
615     if (tableExists) {
616       throw new IOException("Retries exhausted, it took too long to wait"+
617         " for the table " + Bytes.toString(tableName) + " to be deleted.");
618     }
619     // Delete cached information to prevent clients from using old locations
620     this.connection.clearRegionCache(tableName);
621     LOG.info("Deleted " + Bytes.toString(tableName));
622   }
623 
624   /**
625    * Deletes tables matching the passed in pattern and wait on completion.
626    *
627    * Warning: Use this method carefully, there is no prompting and the effect is
628    * immediate. Consider using {@link #listTables(java.lang.String)} and
629    * {@link #deleteTable(byte[])}
630    *
631    * @param regex The regular expression to match table names against
632    * @return Table descriptors for tables that couldn't be deleted
633    * @throws IOException
634    * @see #deleteTables(java.util.regex.Pattern)
635    * @see #deleteTable(java.lang.String)
636    */
637   public HTableDescriptor[] deleteTables(String regex) throws IOException {
638     return deleteTables(Pattern.compile(regex));
639   }
640 
641   /**
642    * Delete tables matching the passed in pattern and wait on completion.
643    *
644    * Warning: Use this method carefully, there is no prompting and the effect is
645    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
646    * {@link #deleteTable(byte[])}
647    *
648    * @param pattern The pattern to match table names against
649    * @return Table descriptors for tables that couldn't be deleted
650    * @throws IOException
651    */
652   public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
653     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
654     for (HTableDescriptor table : listTables(pattern)) {
655       try {
656         deleteTable(table.getName());
657       } catch (IOException ex) {
658         LOG.info("Failed to delete table " + table.getNameAsString(), ex);
659         failed.add(table);
660       }
661     }
662     return failed.toArray(new HTableDescriptor[failed.size()]);
663   }
664 
665 
666   public void enableTable(final String tableName)
667   throws IOException {
668     enableTable(Bytes.toBytes(tableName));
669   }
670 
671   /**
672    * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
673    * and {@link #isTableEnabled(byte[])} instead.
674    * The table has to be in disabled state for it to be enabled.
675    * @param tableName name of the table
676    * @throws IOException if a remote or network exception occurs
677    * There could be couple types of IOException
678    * TableNotFoundException means the table doesn't exist.
679    * TableNotDisabledException means the table isn't in disabled state.
680    * @see #isTableEnabled(byte[])
681    * @see #disableTable(byte[])
682    * @see #enableTableAsync(byte[])
683    */
684   public void enableTable(final byte [] tableName)
685   throws IOException {
686     enableTableAsync(tableName);
687 
688     // Wait until all regions are enabled
689     waitUntilTableIsEnabled(tableName);
690 
691     LOG.info("Enabled table " + Bytes.toString(tableName));
692   }
693 
694   /**
695    * Wait for the table to be enabled and available
696    * If enabling the table exceeds the retry period, an exception is thrown.
697    * @param tableName name of the table
698    * @throws IOException if a remote or network exception occurs or
699    *    table is not enabled after the retries period.
700    */
701   private void waitUntilTableIsEnabled(final byte[] tableName) throws IOException {
702     boolean enabled = false;
703     long start = EnvironmentEdgeManager.currentTimeMillis();
704     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
705       enabled = isTableEnabled(tableName) && isTableAvailable(tableName);
706       if (enabled) {
707         break;
708       }
709       long sleep = getPauseTime(tries);
710       if (LOG.isDebugEnabled()) {
711         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
712           "enabled in " + Bytes.toString(tableName));
713       }
714       try {
715         Thread.sleep(sleep);
716       } catch (InterruptedException e) {
717         Thread.currentThread().interrupt();
718         // Do this conversion rather than let it out because do not want to
719         // change the method signature.
720         throw new IOException("Interrupted", e);
721       }
722     }
723     if (!enabled) {
724       long msec = EnvironmentEdgeManager.currentTimeMillis() - start;
725       throw new IOException("Table '" + Bytes.toString(tableName) +
726         "' not yet enabled, after " + msec + "ms.");
727     }
728   }
729 
730   public void enableTableAsync(final String tableName)
731   throws IOException {
732     enableTableAsync(Bytes.toBytes(tableName));
733   }
734 
735   /**
736    * Brings a table on-line (enables it).  Method returns immediately though
737    * enable of table may take some time to complete, especially if the table
738    * is large (All regions are opened as part of enabling process).  Check
739    * {@link #isTableEnabled(byte[])} to learn when table is fully online.  If
740    * table is taking too long to online, check server logs.
741    * @param tableName
742    * @throws IOException
743    * @since 0.90.0
744    */
745   public void enableTableAsync(final byte [] tableName)
746   throws IOException {
747     HTableDescriptor.isLegalTableName(tableName);
748     isMasterRunning();
749     try {
750       getMaster().enableTable(tableName);
751     } catch (RemoteException e) {
752       throw e.unwrapRemoteException();
753     }
754     LOG.info("Started enable of " + Bytes.toString(tableName));
755   }
756 
757   /**
758    * Enable tables matching the passed in pattern and wait on completion.
759    *
760    * Warning: Use this method carefully, there is no prompting and the effect is
761    * immediate. Consider using {@link #listTables(java.lang.String)} and
762    * {@link #enableTable(byte[])}
763    *
764    * @param regex The regular expression to match table names against
765    * @throws IOException
766    * @see #enableTables(java.util.regex.Pattern)
767    * @see #enableTable(java.lang.String)
768    */
769   public HTableDescriptor[] enableTables(String regex) throws IOException {
770     return enableTables(Pattern.compile(regex));
771   }
772 
773   /**
774    * Enable tables matching the passed in pattern and wait on completion.
775    *
776    * Warning: Use this method carefully, there is no prompting and the effect is
777    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
778    * {@link #enableTable(byte[])}
779    *
780    * @param pattern The pattern to match table names against
781    * @throws IOException
782    */
783   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
784     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
785     for (HTableDescriptor table : listTables(pattern)) {
786       if (isTableDisabled(table.getName())) {
787         try {
788           enableTable(table.getName());
789         } catch (IOException ex) {
790           LOG.info("Failed to enable table " + table.getNameAsString(), ex);
791           failed.add(table);
792         }
793       }
794     }
795     return failed.toArray(new HTableDescriptor[failed.size()]);
796   }
797 
798   public void disableTableAsync(final String tableName) throws IOException {
799     disableTableAsync(Bytes.toBytes(tableName));
800   }
801 
802   /**
803    * Starts the disable of a table.  If it is being served, the master
804    * will tell the servers to stop serving it.  This method returns immediately.
805    * The disable of a table can take some time if the table is large (all
806    * regions are closed as part of table disable operation).
807    * Call {@link #isTableDisabled(byte[])} to check for when disable completes.
808    * If table is taking too long to online, check server logs.
809    * @param tableName name of table
810    * @throws IOException if a remote or network exception occurs
811    * @see #isTableDisabled(byte[])
812    * @see #isTableEnabled(byte[])
813    * @since 0.90.0
814    */
815   public void disableTableAsync(final byte [] tableName) throws IOException {
816     HTableDescriptor.isLegalTableName(tableName);
817     isMasterRunning();
818     try {
819       getMaster().disableTable(tableName);
820     } catch (RemoteException e) {
821       throw e.unwrapRemoteException();
822     }
823     LOG.info("Started disable of " + Bytes.toString(tableName));
824   }
825 
826   public void disableTable(final String tableName)
827   throws IOException {
828     disableTable(Bytes.toBytes(tableName));
829   }
830 
831   /**
832    * Disable table and wait on completion.  May timeout eventually.  Use
833    * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
834    * instead.
835    * The table has to be in enabled state for it to be disabled.
836    * @param tableName
837    * @throws IOException
838    * There could be couple types of IOException
839    * TableNotFoundException means the table doesn't exist.
840    * TableNotEnabledException means the table isn't in enabled state.
841    */
842   public void disableTable(final byte [] tableName)
843   throws IOException {
844     disableTableAsync(tableName);
845     // Wait until table is disabled
846     boolean disabled = false;
847     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
848       disabled = isTableDisabled(tableName);
849       if (disabled) {
850         break;
851       }
852       long sleep = getPauseTime(tries);
853       if (LOG.isDebugEnabled()) {
854         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
855           "disabled in " + Bytes.toString(tableName));
856       }
857       try {
858         Thread.sleep(sleep);
859       } catch (InterruptedException e) {
860         // Do this conversion rather than let it out because do not want to
861         // change the method signature.
862         Thread.currentThread().interrupt();
863         throw new IOException("Interrupted", e);
864       }
865     }
866     if (!disabled) {
867       throw new RegionException("Retries exhausted, it took too long to wait"+
868         " for the table " + Bytes.toString(tableName) + " to be disabled.");
869     }
870     LOG.info("Disabled " + Bytes.toString(tableName));
871   }
872 
873   /**
874    * Disable tables matching the passed in pattern and wait on completion.
875    *
876    * Warning: Use this method carefully, there is no prompting and the effect is
877    * immediate. Consider using {@link #listTables(java.lang.String)} and
878    * {@link #disableTable(byte[])}
879    *
880    * @param regex The regular expression to match table names against
881    * @return Table descriptors for tables that couldn't be disabled
882    * @throws IOException
883    * @see #disableTables(java.util.regex.Pattern)
884    * @see #disableTable(java.lang.String)
885    */
886   public HTableDescriptor[] disableTables(String regex) throws IOException {
887     return disableTables(Pattern.compile(regex));
888   }
889 
890   /**
891    * Disable tables matching the passed in pattern and wait on completion.
892    *
893    * Warning: Use this method carefully, there is no prompting and the effect is
894    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
895    * {@link #disableTable(byte[])}
896    *
897    * @param pattern The pattern to match table names against
898    * @return Table descriptors for tables that couldn't be disabled
899    * @throws IOException
900    */
901   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
902     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
903     for (HTableDescriptor table : listTables(pattern)) {
904       if (isTableEnabled(table.getName())) {
905         try {
906           disableTable(table.getName());
907         } catch (IOException ex) {
908           LOG.info("Failed to disable table " + table.getNameAsString(), ex);
909           failed.add(table);
910         }
911       }
912     }
913     return failed.toArray(new HTableDescriptor[failed.size()]);
914   }
915 
916   /**
917    * @param tableName name of table to check
918    * @return true if table is on-line
919    * @throws IOException if a remote or network exception occurs
920    */
921   public boolean isTableEnabled(String tableName) throws IOException {
922     return isTableEnabled(Bytes.toBytes(tableName));
923   }
924   /**
925    * @param tableName name of table to check
926    * @return true if table is on-line
927    * @throws IOException if a remote or network exception occurs
928    */
929   public boolean isTableEnabled(byte[] tableName) throws IOException {
930     if (!HTableDescriptor.isMetaTable(tableName)) {
931       HTableDescriptor.isLegalTableName(tableName);
932     }
933     if(!tableExists(tableName)){
934       throw new TableNotFoundException(Bytes.toString(tableName));
935     }
936     return connection.isTableEnabled(tableName);
937   }
938 
939   /**
940    * @param tableName name of table to check
941    * @return true if table is off-line
942    * @throws IOException if a remote or network exception occurs
943    */
944   public boolean isTableDisabled(final String tableName) throws IOException {
945     return isTableDisabled(Bytes.toBytes(tableName));
946   }
947 
948   /**
949    * @param tableName name of table to check
950    * @return true if table is off-line
951    * @throws IOException if a remote or network exception occurs
952    */
953   public boolean isTableDisabled(byte[] tableName) throws IOException {
954     if (!HTableDescriptor.isMetaTable(tableName)) {
955       HTableDescriptor.isLegalTableName(tableName);
956     }
957     return connection.isTableDisabled(tableName);
958   }
959 
960   /**
961    * @param tableName name of table to check
962    * @return true if all regions of the table are available
963    * @throws IOException if a remote or network exception occurs
964    */
965   public boolean isTableAvailable(byte[] tableName) throws IOException {
966     return connection.isTableAvailable(tableName);
967   }
968 
969   /**
970    * @param tableName name of table to check
971    * @return true if all regions of the table are available
972    * @throws IOException if a remote or network exception occurs
973    */
974   public boolean isTableAvailable(String tableName) throws IOException {
975     return connection.isTableAvailable(Bytes.toBytes(tableName));
976   }
977 
978   /**
979    * Get the status of alter command - indicates how many regions have received
980    * the updated schema Asynchronous operation.
981    *
982    * @param tableName
983    *          name of the table to get the status of
984    * @return Pair indicating the number of regions updated Pair.getFirst() is the
985    *         regions that are yet to be updated Pair.getSecond() is the total number
986    *         of regions of the table
987    * @throws IOException
988    *           if a remote or network exception occurs
989    */
990   public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
991   throws IOException {
992     HTableDescriptor.isLegalTableName(tableName);
993     try {
994       return getMaster().getAlterStatus(tableName);
995     } catch (RemoteException e) {
996       throw RemoteExceptionHandler.decodeRemoteException(e);
997     }
998   }
999 
1000   /**
1001    * Add a column to an existing table.
1002    * Asynchronous operation.
1003    *
1004    * @param tableName name of the table to add column to
1005    * @param column column descriptor of column to be added
1006    * @throws IOException if a remote or network exception occurs
1007    */
1008   public void addColumn(final String tableName, HColumnDescriptor column)
1009   throws IOException {
1010     addColumn(Bytes.toBytes(tableName), column);
1011   }
1012 
1013   /**
1014    * Add a column to an existing table.
1015    * Asynchronous operation.
1016    *
1017    * @param tableName name of the table to add column to
1018    * @param column column descriptor of column to be added
1019    * @throws IOException if a remote or network exception occurs
1020    */
1021   public void addColumn(final byte [] tableName, HColumnDescriptor column)
1022   throws IOException {
1023     HTableDescriptor.isLegalTableName(tableName);
1024     try {
1025       getMaster().addColumn(tableName, column);
1026     } catch (RemoteException e) {
1027       throw RemoteExceptionHandler.decodeRemoteException(e);
1028     }
1029   }
1030 
1031   /**
1032    * Delete a column from a table.
1033    * Asynchronous operation.
1034    *
1035    * @param tableName name of table
1036    * @param columnName name of column to be deleted
1037    * @throws IOException if a remote or network exception occurs
1038    */
1039   public void deleteColumn(final String tableName, final String columnName)
1040   throws IOException {
1041     deleteColumn(Bytes.toBytes(tableName), Bytes.toBytes(columnName));
1042   }
1043 
1044   /**
1045    * Delete a column from a table.
1046    * Asynchronous operation.
1047    *
1048    * @param tableName name of table
1049    * @param columnName name of column to be deleted
1050    * @throws IOException if a remote or network exception occurs
1051    */
1052   public void deleteColumn(final byte [] tableName, final byte [] columnName)
1053   throws IOException {
1054     try {
1055       getMaster().deleteColumn(tableName, columnName);
1056     } catch (RemoteException e) {
1057       throw RemoteExceptionHandler.decodeRemoteException(e);
1058     }
1059   }
1060 
1061   /**
1062    * Modify an existing column family on a table.
1063    * Asynchronous operation.
1064    *
1065    * @param tableName name of table
1066    * @param descriptor new column descriptor to use
1067    * @throws IOException if a remote or network exception occurs
1068    */
1069   public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
1070   throws IOException {
1071     modifyColumn(Bytes.toBytes(tableName), descriptor);
1072   }
1073 
1074   /**
1075    * Modify an existing column family on a table.
1076    * Asynchronous operation.
1077    *
1078    * @param tableName name of table
1079    * @param descriptor new column descriptor to use
1080    * @throws IOException if a remote or network exception occurs
1081    */
1082   public void modifyColumn(final byte [] tableName, HColumnDescriptor descriptor)
1083   throws IOException {
1084     try {
1085       getMaster().modifyColumn(tableName, descriptor);
1086     } catch (RemoteException re) {
1087       // Convert RE exceptions in here; client shouldn't have to deal with them,
1088       // at least w/ the type of exceptions that come out of this method:
1089       // TableNotFoundException, etc.
1090       throw RemoteExceptionHandler.decodeRemoteException(re);
1091     }
1092   }
1093 
1094   /**
1095    * Close a region. For expert-admins.  Runs close on the regionserver.  The
1096    * master will not be informed of the close.
1097    * @param regionname region name to close
1098    * @param serverName If supplied, we'll use this location rather than
1099    * the one currently in <code>.META.</code>
1100    * @throws IOException if a remote or network exception occurs
1101    */
1102   public void closeRegion(final String regionname, final String serverName)
1103   throws IOException {
1104     closeRegion(Bytes.toBytes(regionname), serverName);
1105   }
1106 
1107   /**
1108    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1109    * master will not be informed of the close.
1110    * @param regionname region name to close
1111    * @param serverName The servername of the regionserver.  If passed null we
1112    * will use servername found in the .META. table. A server name
1113    * is made of host, port and startcode.  Here is an example:
1114    * <code> host187.example.com,60020,1289493121758</code>
1115    * @throws IOException if a remote or network exception occurs
1116    */
1117   public void closeRegion(final byte [] regionname, final String serverName)
1118   throws IOException {
1119     CatalogTracker ct = getCatalogTracker();
1120     try {
1121       if (serverName != null) {
1122         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1123         if (pair == null || pair.getFirst() == null) {
1124           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1125         } else {
1126           closeRegion(new ServerName(serverName), pair.getFirst());
1127         }
1128       } else {
1129         Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, regionname);
1130         if (pair == null) {
1131           throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1132         } else if (pair.getSecond() == null) {
1133           throw new NoServerForRegionException(Bytes.toStringBinary(regionname));
1134         } else {
1135           closeRegion(pair.getSecond(), pair.getFirst());
1136         }
1137       }
1138     } finally {
1139       cleanupCatalogTracker(ct);
1140     }
1141   }
1142 
1143   /**
1144    * For expert-admins. Runs close on the regionserver. Closes a region based on
1145    * the encoded region name. The region server name is mandatory. If the
1146    * servername is provided then based on the online regions in the specified
1147    * regionserver the specified region will be closed. The master will not be
1148    * informed of the close. Note that the regionname is the encoded regionname.
1149    *
1150    * @param encodedRegionName
1151    *          The encoded region name; i.e. the hash that makes up the region
1152    *          name suffix: e.g. if regionname is
1153    *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
1154    *          , then the encoded region name is:
1155    *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
1156    * @param serverName
1157    *          The servername of the regionserver. A server name is made of host,
1158    *          port and startcode. This is mandatory. Here is an example:
1159    *          <code> host187.example.com,60020,1289493121758</code>
1160    * @return true if the region was closed, false if not.
1161    * @throws IOException
1162    *           if a remote or network exception occurs
1163    */
1164   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1165       final String serverName) throws IOException {
1166     byte[] encodedRegionNameInBytes = Bytes.toBytes(encodedRegionName);
1167     if (null == serverName || ("").equals(serverName.trim())) {
1168       throw new IllegalArgumentException(
1169           "The servername cannot be null or empty.");
1170     }
1171     ServerName sn = new ServerName(serverName);
1172     HRegionInterface rs = this.connection.getHRegionConnection(
1173         sn.getHostname(), sn.getPort());
1174     // Close the region without updating zk state.
1175     boolean isRegionClosed = rs.closeRegion(encodedRegionNameInBytes, false);
1176     if (false == isRegionClosed) {
1177       LOG.error("Not able to close the region " + encodedRegionName + ".");
1178     }
1179     return isRegionClosed;
1180   }
1181 
1182   /**
1183    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1184    * master will not be informed of the close.
1185    * @param sn
1186    * @param hri
1187    * @throws IOException
1188    */
1189   public void closeRegion(final ServerName sn, final HRegionInfo hri)
1190   throws IOException {
1191     HRegionInterface rs =
1192       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1193     // Close the region without updating zk state.
1194     rs.closeRegion(hri, false);
1195   }
1196 
1197   /**
1198    * Flush a table or an individual region.
1199    * Asynchronous operation.
1200    *
1201    * @param tableNameOrRegionName table or region to flush
1202    * @throws IOException if a remote or network exception occurs
1203    * @throws InterruptedException
1204    */
1205   public void flush(final String tableNameOrRegionName)
1206   throws IOException, InterruptedException {
1207     flush(Bytes.toBytes(tableNameOrRegionName));
1208   }
1209 
1210   /**
1211    * Flush a table or an individual region.
1212    * Asynchronous operation.
1213    *
1214    * @param tableNameOrRegionName table or region to flush
1215    * @throws IOException if a remote or network exception occurs
1216    * @throws InterruptedException
1217    */
1218   public void flush(final byte [] tableNameOrRegionName)
1219   throws IOException, InterruptedException {
1220     CatalogTracker ct = getCatalogTracker();
1221     try {
1222       Pair<HRegionInfo, ServerName> regionServerPair
1223         = getRegion(tableNameOrRegionName, ct);
1224       if (regionServerPair != null) {
1225         if (regionServerPair.getSecond() == null) {
1226           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1227         } else {
1228           flush(regionServerPair.getSecond(), regionServerPair.getFirst());
1229         }
1230       } else {
1231         final String tableName = tableNameString(tableNameOrRegionName, ct);
1232         List<Pair<HRegionInfo, ServerName>> pairs =
1233           MetaReader.getTableRegionsAndLocations(ct,
1234               tableName);
1235         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1236           if (pair.getFirst().isOffline()) continue;
1237           if (pair.getSecond() == null) continue;
1238           try {
1239             flush(pair.getSecond(), pair.getFirst());
1240           } catch (NotServingRegionException e) {
1241             if (LOG.isDebugEnabled()) {
1242               LOG.debug("Trying to flush " + pair.getFirst() + ": " +
1243                 StringUtils.stringifyException(e));
1244             }
1245           }
1246         }
1247       }
1248     } finally {
1249       cleanupCatalogTracker(ct);
1250     }
1251   }
1252 
1253   private void flush(final ServerName sn, final HRegionInfo hri)
1254   throws IOException {
1255     HRegionInterface rs =
1256       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1257     rs.flushRegion(hri);
1258   }
1259 
1260   /**
1261    * Compact a table or an individual region.
1262    * Asynchronous operation.
1263    *
1264    * @param tableNameOrRegionName table or region to compact
1265    * @throws IOException if a remote or network exception occurs
1266    * @throws InterruptedException
1267    */
1268   public void compact(final String tableNameOrRegionName)
1269   throws IOException, InterruptedException {
1270     compact(Bytes.toBytes(tableNameOrRegionName));
1271   }
1272 
1273   /**
1274    * Compact a table or an individual region.
1275    * Asynchronous operation.
1276    *
1277    * @param tableNameOrRegionName table or region to compact
1278    * @throws IOException if a remote or network exception occurs
1279    * @throws InterruptedException
1280    */
1281   public void compact(final byte [] tableNameOrRegionName)
1282   throws IOException, InterruptedException {
1283     compact(tableNameOrRegionName, null, false);
1284   }
1285 
1286   /**
1287    * Compact a column family within a table or region.
1288    * Asynchronous operation.
1289    *
1290    * @param tableOrRegionName table or region to compact
1291    * @param columnFamily column family within a table or region
1292    * @throws IOException if a remote or network exception occurs
1293    * @throws InterruptedException
1294    */
1295   public void compact(String tableOrRegionName, String columnFamily)
1296     throws IOException,  InterruptedException {
1297     compact(Bytes.toBytes(tableOrRegionName), Bytes.toBytes(columnFamily));
1298   }
1299 
1300   /**
1301    * Compact a column family within a table or region.
1302    * Asynchronous operation.
1303    *
1304    * @param tableNameOrRegionName table or region to compact
1305    * @param columnFamily column family within a table or region
1306    * @throws IOException if a remote or network exception occurs
1307    * @throws InterruptedException
1308    */
1309   public void compact(final byte [] tableNameOrRegionName, final byte[] columnFamily)
1310   throws IOException, InterruptedException {
1311     compact(tableNameOrRegionName, columnFamily, false);
1312   }
1313 
1314   /**
1315    * Major compact a table or an individual region.
1316    * Asynchronous operation.
1317    *
1318    * @param tableNameOrRegionName table or region to major compact
1319    * @throws IOException if a remote or network exception occurs
1320    * @throws InterruptedException
1321    */
1322   public void majorCompact(final String tableNameOrRegionName)
1323   throws IOException, InterruptedException {
1324     majorCompact(Bytes.toBytes(tableNameOrRegionName));
1325   }
1326 
1327   /**
1328    * Major compact a table or an individual region.
1329    * Asynchronous operation.
1330    *
1331    * @param tableNameOrRegionName table or region to major compact
1332    * @throws IOException if a remote or network exception occurs
1333    * @throws InterruptedException
1334    */
1335   public void majorCompact(final byte [] tableNameOrRegionName)
1336   throws IOException, InterruptedException {
1337     compact(tableNameOrRegionName, null, true);
1338   }
1339 
1340   /**
1341    * Major compact a column family within a table or region.
1342    * Asynchronous operation.
1343    *
1344    * @param tableNameOrRegionName table or region to major compact
1345    * @param columnFamily column family within a table or region
1346    * @throws IOException if a remote or network exception occurs
1347    * @throws InterruptedException
1348    */
1349   public void majorCompact(final String tableNameOrRegionName,
1350     final String columnFamily) throws IOException, InterruptedException {
1351     majorCompact(Bytes.toBytes(tableNameOrRegionName),
1352       Bytes.toBytes(columnFamily));
1353   }
1354 
1355   /**
1356    * Major compact a column family within a table or region.
1357    * Asynchronous operation.
1358    *
1359    * @param tableNameOrRegionName table or region to major compact
1360    * @param columnFamily column family within a table or region
1361    * @throws IOException if a remote or network exception occurs
1362    * @throws InterruptedException
1363    */
1364   public void majorCompact(final byte [] tableNameOrRegionName,
1365     final byte[] columnFamily) throws IOException, InterruptedException {
1366     compact(tableNameOrRegionName, columnFamily, true);
1367   }
1368 
1369   /**
1370    * Compact a table or an individual region.
1371    * Asynchronous operation.
1372    *
1373    * @param tableNameOrRegionName table or region to compact
1374    * @param columnFamily column family within a table or region
1375    * @param major True if we are to do a major compaction.
1376    * @throws IOException if a remote or network exception occurs
1377    * @throws InterruptedException
1378    */
1379   private void compact(final byte [] tableNameOrRegionName,
1380     final byte[] columnFamily, final boolean major)
1381   throws IOException, InterruptedException {
1382     CatalogTracker ct = getCatalogTracker();
1383     try {
1384       Pair<HRegionInfo, ServerName> regionServerPair
1385         = getRegion(tableNameOrRegionName, ct);
1386       if (regionServerPair != null) {
1387         if (regionServerPair.getSecond() == null) {
1388           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1389         } else {
1390           compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
1391         }
1392       } else {
1393         final String tableName = tableNameString(tableNameOrRegionName, ct);
1394         List<Pair<HRegionInfo, ServerName>> pairs =
1395           MetaReader.getTableRegionsAndLocations(ct,
1396               tableName);
1397         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1398           if (pair.getFirst().isOffline()) continue;
1399           if (pair.getSecond() == null) continue;
1400           try {
1401             compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
1402           } catch (NotServingRegionException e) {
1403             if (LOG.isDebugEnabled()) {
1404               LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
1405                 pair.getFirst() + ": " +
1406                 StringUtils.stringifyException(e));
1407             }
1408           }
1409         }
1410       }
1411     } finally {
1412       cleanupCatalogTracker(ct);
1413     }
1414   }
1415 
1416   private void compact(final ServerName sn, final HRegionInfo hri,
1417       final boolean major, final byte [] family)
1418   throws IOException {
1419     HRegionInterface rs =
1420       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1421     if (family != null) {
1422       try {
1423         rs.compactRegion(hri, major, family);
1424       } catch (IOException ioe) {
1425         String notFoundMsg = "java.lang.NoSuchMethodException: org.apache.hadoop.hbase.ipc.HRegionInterface."
1426           + "compactRegion(org.apache.hadoop.hbase.HRegionInfo, boolean, [B)";
1427         if (ioe.getMessage().contains(notFoundMsg)) {
1428           throw new IOException("per-column family compaction not supported on this version "
1429             + "of the HBase server.  You may still compact at the table or region level by "
1430           	+ "omitting the column family name.  Alternatively, you can upgrade the HBase server");
1431         }
1432         throw ioe;
1433       }
1434     } else {
1435       rs.compactRegion(hri, major);
1436     }
1437   }
1438 
1439   /**
1440    * Move the region <code>r</code> to <code>dest</code>.
1441    * @param encodedRegionName The encoded region name; i.e. the hash that makes
1442    * up the region name suffix: e.g. if regionname is
1443    * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
1444    * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
1445    * @param destServerName The servername of the destination regionserver.  If
1446    * passed the empty byte array we'll assign to a random server.  A server name
1447    * is made of host, port and startcode.  Here is an example:
1448    * <code> host187.example.com,60020,1289493121758</code>
1449    * @throws UnknownRegionException Thrown if we can't find a region named
1450    * <code>encodedRegionName</code>
1451    * @throws ZooKeeperConnectionException
1452    * @throws MasterNotRunningException
1453    */
1454   public void move(final byte [] encodedRegionName, final byte [] destServerName)
1455   throws UnknownRegionException, MasterNotRunningException, ZooKeeperConnectionException {
1456     getMaster().move(encodedRegionName, destServerName);
1457   }
1458 
1459   /**
1460    * @param regionName
1461    *          Region name to assign.
1462    * @throws MasterNotRunningException
1463    * @throws ZooKeeperConnectionException
1464    * @throws IOException
1465    */
1466   public void assign(final byte[] regionName) throws MasterNotRunningException,
1467       ZooKeeperConnectionException, IOException {
1468     getMaster().assign(regionName);
1469   }
1470 
1471   /**
1472    * Unassign a region from current hosting regionserver.  Region will then be
1473    * assigned to a regionserver chosen at random.  Region could be reassigned
1474    * back to the same server.  Use {@link #move(byte[], byte[])} if you want
1475    * to control the region movement.
1476    * @param regionName Region to unassign. Will clear any existing RegionPlan
1477    * if one found.
1478    * @param force If true, force unassign (Will remove region from
1479    * regions-in-transition too if present. If results in double assignment
1480    * use hbck -fix to resolve. To be used by experts).
1481    * @throws MasterNotRunningException
1482    * @throws ZooKeeperConnectionException
1483    * @throws IOException
1484    */
1485   public void unassign(final byte [] regionName, final boolean force)
1486   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
1487     getMaster().unassign(regionName, force);
1488   }
1489 
1490   /**
1491    * Turn the load balancer on or off.
1492    * @param b If true, enable balancer. If false, disable balancer.
1493    * @return Previous balancer value
1494    * @deprecated use setBalancerRunning(boolean, boolean) instead
1495    */
1496   @Deprecated
1497   public boolean balanceSwitch(final boolean b)
1498   throws MasterNotRunningException, ZooKeeperConnectionException {
1499     return getMaster().balanceSwitch(b);
1500   }
1501 
1502   /**
1503    * Turn the load balancer on or off.
1504    * @param on If true, enable balancer. If false, disable balancer.
1505    * @param synchronous If true, it waits until current balance() call, if outstanding, to return.
1506    * @return Previous balancer value
1507    */
1508   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
1509   throws MasterNotRunningException, ZooKeeperConnectionException {
1510     if (synchronous && synchronousBalanceSwitchSupported) {
1511       try {
1512         return getMaster().synchronousBalanceSwitch(on);
1513       } catch (UndeclaredThrowableException ute) {
1514         String error = ute.getCause().getMessage();
1515         if (error != null && error.matches(
1516             "(?s).+NoSuchMethodException:.+synchronousBalanceSwitch.+")) {
1517           LOG.info("HMaster doesn't support synchronousBalanceSwitch");
1518           synchronousBalanceSwitchSupported = false;
1519         } else {
1520           throw ute;
1521         }
1522       }
1523     }
1524     return balanceSwitch(on);
1525   }
1526 
1527   /**
1528    * Invoke the balancer.  Will run the balancer and if regions to move, it will
1529    * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
1530    * logs.
1531    * @return True if balancer ran, false otherwise.
1532    */
1533   public boolean balancer()
1534   throws MasterNotRunningException, ZooKeeperConnectionException {
1535     return getMaster().balance();
1536   }
1537 
1538   /**
1539    * Split a table or an individual region.
1540    * Asynchronous operation.
1541    *
1542    * @param tableNameOrRegionName table or region to split
1543    * @throws IOException if a remote or network exception occurs
1544    * @throws InterruptedException
1545    */
1546   public void split(final String tableNameOrRegionName)
1547   throws IOException, InterruptedException {
1548     split(Bytes.toBytes(tableNameOrRegionName));
1549   }
1550 
1551   /**
1552    * Split a table or an individual region.  Implicitly finds an optimal split
1553    * point.  Asynchronous operation.
1554    *
1555    * @param tableNameOrRegionName table to region to split
1556    * @throws IOException if a remote or network exception occurs
1557    * @throws InterruptedException
1558    */
1559   public void split(final byte [] tableNameOrRegionName)
1560   throws IOException, InterruptedException {
1561     split(tableNameOrRegionName, null);
1562   }
1563 
1564   public void split(final String tableNameOrRegionName,
1565     final String splitPoint) throws IOException, InterruptedException {
1566     split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
1567   }
1568 
1569   /**
1570    * Split a table or an individual region.
1571    * Asynchronous operation.
1572    *
1573    * @param tableNameOrRegionName table to region to split
1574    * @param splitPoint the explicit position to split on
1575    * @throws IOException if a remote or network exception occurs
1576    * @throws InterruptedException interrupt exception occurred
1577    */
1578   public void split(final byte [] tableNameOrRegionName,
1579       final byte [] splitPoint) throws IOException, InterruptedException {
1580     CatalogTracker ct = getCatalogTracker();
1581     try {
1582       Pair<HRegionInfo, ServerName> regionServerPair
1583         = getRegion(tableNameOrRegionName, ct);
1584       if (regionServerPair != null) {
1585         if (regionServerPair.getSecond() == null) {
1586             throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1587         } else {
1588           split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
1589         }
1590       } else {
1591         final String tableName = tableNameString(tableNameOrRegionName, ct);
1592         List<Pair<HRegionInfo, ServerName>> pairs =
1593           MetaReader.getTableRegionsAndLocations(ct,
1594               tableName);
1595         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1596           // May not be a server for a particular row
1597           if (pair.getSecond() == null) continue;
1598           HRegionInfo r = pair.getFirst();
1599           // check for parents
1600           if (r.isSplitParent()) continue;
1601           // if a split point given, only split that particular region
1602           if (splitPoint != null && !r.containsRow(splitPoint)) continue;
1603           // call out to region server to do split now
1604           split(pair.getSecond(), pair.getFirst(), splitPoint);
1605         }
1606       }
1607     } finally {
1608       cleanupCatalogTracker(ct);
1609     }
1610   }
1611 
1612   private void split(final ServerName sn, final HRegionInfo hri,
1613       byte[] splitPoint) throws IOException {
1614     HRegionInterface rs =
1615       this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1616     rs.splitRegion(hri, splitPoint);
1617   }
1618 
1619   /**
1620    * Modify an existing table, more IRB friendly version.
1621    * Asynchronous operation.  This means that it may be a while before your
1622    * schema change is updated across all of the table.
1623    *
1624    * @param tableName name of table.
1625    * @param htd modified description of the table
1626    * @throws IOException if a remote or network exception occurs
1627    */
1628   public void modifyTable(final byte [] tableName, HTableDescriptor htd)
1629   throws IOException {
1630     try {
1631       getMaster().modifyTable(tableName, htd);
1632     } catch (RemoteException re) {
1633       // Convert RE exceptions in here; client shouldn't have to deal with them,
1634       // at least w/ the type of exceptions that come out of this method:
1635       // TableNotFoundException, etc.
1636       throw RemoteExceptionHandler.decodeRemoteException(re);
1637     }
1638   }
1639 
1640   /**
1641    * @param tableNameOrRegionName Name of a table or name of a region.
1642    * @param ct A {@link CatalogTracker} instance (caller of this method usually has one).
1643    * @return a pair of HRegionInfo and ServerName if <code>tableNameOrRegionName</code> is
1644    *  a verified region name (we call {@link  MetaReader#getRegion( CatalogTracker, byte[])}
1645    *  else null.
1646    * Throw an exception if <code>tableNameOrRegionName</code> is null.
1647    * @throws IOException
1648    */
1649   Pair<HRegionInfo, ServerName> getRegion(final byte[] tableNameOrRegionName,
1650       final CatalogTracker ct) throws IOException {
1651     if (tableNameOrRegionName == null) {
1652       throw new IllegalArgumentException("Pass a table name or region name");
1653     }
1654     Pair<HRegionInfo, ServerName> pair = MetaReader.getRegion(ct, tableNameOrRegionName);
1655     if (pair == null) {
1656       final AtomicReference<Pair<HRegionInfo, ServerName>> result =
1657         new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
1658       final String encodedName = Bytes.toString(tableNameOrRegionName);
1659       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
1660         @Override
1661         public boolean processRow(Result data) throws IOException {
1662           if (data == null || data.size() <= 0) {
1663             return true;
1664           }
1665           HRegionInfo info = MetaReader.parseHRegionInfoFromCatalogResult(
1666             data, HConstants.REGIONINFO_QUALIFIER);
1667           if (info == null) {
1668             LOG.warn("No serialized HRegionInfo in " + data);
1669             return true;
1670           }
1671           if (!encodedName.equals(info.getEncodedName())) return true;
1672           ServerName sn = MetaReader.getServerNameFromCatalogResult(data);
1673           result.set(new Pair<HRegionInfo, ServerName>(info, sn));
1674           return false; // found the region, stop
1675         }
1676       };
1677 
1678       MetaScanner.metaScan(conf, visitor);
1679       pair = result.get();
1680     }
1681     return pair;
1682   }
1683 
1684   /**
1685    * Convert the table name byte array into a table name string and check if table
1686    * exists or not.
1687    * @param tableNameBytes Name of a table.
1688    * @param ct A {@link #CatalogTracker} instance (caller of this method usually has one).
1689    * @return tableName in string form.
1690    * @throws IOException if a remote or network exception occurs.
1691    * @throws TableNotFoundException if table does not exist.
1692    */
1693   private String tableNameString(final byte[] tableNameBytes, CatalogTracker ct)
1694       throws IOException {
1695     String tableNameString = Bytes.toString(tableNameBytes);
1696     if (!MetaReader.tableExists(ct, tableNameString)) {
1697       throw new TableNotFoundException(tableNameString);
1698     }
1699     return tableNameString;
1700   }
1701 
1702   /**
1703    * Shuts down the HBase cluster
1704    * @throws IOException if a remote or network exception occurs
1705    */
1706   public synchronized void shutdown() throws IOException {
1707     isMasterRunning();
1708     try {
1709       getMaster().shutdown();
1710     } catch (RemoteException e) {
1711       throw RemoteExceptionHandler.decodeRemoteException(e);
1712     }
1713   }
1714 
1715   /**
1716    * Shuts down the current HBase master only.
1717    * Does not shutdown the cluster.
1718    * @see #shutdown()
1719    * @throws IOException if a remote or network exception occurs
1720    */
1721   public synchronized void stopMaster() throws IOException {
1722     isMasterRunning();
1723     try {
1724       getMaster().stopMaster();
1725     } catch (RemoteException e) {
1726       throw RemoteExceptionHandler.decodeRemoteException(e);
1727     }
1728   }
1729 
1730   /**
1731    * Stop the designated regionserver
1732    * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
1733    * <code>example.org:1234</code>
1734    * @throws IOException if a remote or network exception occurs
1735    */
1736   public synchronized void stopRegionServer(final String hostnamePort)
1737   throws IOException {
1738     String hostname = Addressing.parseHostname(hostnamePort);
1739     int port = Addressing.parsePort(hostnamePort);
1740     HRegionInterface rs =
1741       this.connection.getHRegionConnection(hostname, port);
1742     rs.stop("Called by admin client " + this.connection.toString());
1743   }
1744 
1745   /**
1746    * @return cluster status
1747    * @throws IOException if a remote or network exception occurs
1748    */
1749   public ClusterStatus getClusterStatus() throws IOException {
1750     return getMaster().getClusterStatus();
1751   }
1752 
1753   private HRegionLocation getFirstMetaServerForTable(final byte [] tableName)
1754   throws IOException {
1755     return connection.locateRegion(HConstants.META_TABLE_NAME,
1756       HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
1757   }
1758 
1759   /**
1760    * @return Configuration used by the instance.
1761    */
1762   public Configuration getConfiguration() {
1763     return this.conf;
1764   }
1765 
1766   /**
1767    * Check to see if HBase is running. Throw an exception if not.
1768    *
1769    * @param conf system configuration
1770    * @throws MasterNotRunningException if the master is not running
1771    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
1772    */
1773   public static void checkHBaseAvailable(Configuration conf)
1774   throws MasterNotRunningException, ZooKeeperConnectionException {
1775     Configuration copyOfConf = HBaseConfiguration.create(conf);
1776     copyOfConf.setInt("hbase.client.retries.number", 1);
1777     HBaseAdmin admin = new HBaseAdmin(copyOfConf);
1778     try {
1779       admin.close();
1780     } catch (IOException ioe) {
1781       admin.LOG.info("Failed to close connection", ioe);
1782     }
1783   }
1784 
1785   /**
1786    * get the regions of a given table.
1787    *
1788    * @param tableName the name of the table
1789    * @return Ordered list of {@link HRegionInfo}.
1790    * @throws IOException
1791    */
1792   public List<HRegionInfo> getTableRegions(final byte[] tableName)
1793   throws IOException {
1794     CatalogTracker ct = getCatalogTracker();
1795     List<HRegionInfo> Regions = null;
1796     try {
1797       Regions = MetaReader.getTableRegions(ct, tableName, true);
1798     } finally {
1799       cleanupCatalogTracker(ct);
1800     }
1801     return Regions;
1802   }
1803 
1804   public void close() throws IOException {
1805     if (this.connection != null) {
1806       this.connection.close();
1807     }
1808   }
1809 
1810  /**
1811  * Get tableDescriptors
1812  * @param tableNames List of table names
1813  * @return HTD[] the tableDescriptor
1814  * @throws IOException if a remote or network exception occurs
1815  */
1816   public HTableDescriptor[] getTableDescriptors(List<String> tableNames)
1817   throws IOException {
1818     return this.connection.getHTableDescriptors(tableNames);
1819   }
1820 
1821   /**
1822    * Roll the log writer. That is, start writing log messages to a new file.
1823    *
1824    * @param serverName
1825    *          The servername of the regionserver. A server name is made of host,
1826    *          port and startcode. This is mandatory. Here is an example:
1827    *          <code> host187.example.com,60020,1289493121758</code>
1828    * @return If lots of logs, flush the returned regions so next time through
1829    * we can clean logs. Returns null if nothing to flush.  Names are actual
1830    * region names as returned by {@link HRegionInfo#getEncodedName()}
1831    * @throws IOException if a remote or network exception occurs
1832    * @throws FailedLogCloseException
1833    */
1834  public synchronized  byte[][] rollHLogWriter(String serverName)
1835       throws IOException, FailedLogCloseException {
1836     ServerName sn = new ServerName(serverName);
1837     HRegionInterface rs = this.connection.getHRegionConnection(
1838         sn.getHostname(), sn.getPort());
1839     return rs.rollHLogWriter();
1840   }
1841 
1842   public String[] getMasterCoprocessors() {
1843     try {
1844       return getClusterStatus().getMasterCoprocessors();
1845     } catch (IOException e) {
1846       LOG.error("Could not getClusterStatus()",e);
1847       return null;
1848     }
1849   }
1850 
1851   /**
1852    * Get the current compaction state of a table or region.
1853    * It could be in a major compaction, a minor compaction, both, or none.
1854    *
1855    * @param tableNameOrRegionName table or region to major compact
1856    * @throws IOException if a remote or network exception occurs
1857    * @throws InterruptedException
1858    * @return the current compaction state
1859    */
1860   public CompactionState getCompactionState(final String tableNameOrRegionName)
1861       throws IOException, InterruptedException {
1862     return getCompactionState(Bytes.toBytes(tableNameOrRegionName));
1863   }
1864 
1865   /**
1866    * Get the current compaction state of a table or region.
1867    * It could be in a major compaction, a minor compaction, both, or none.
1868    *
1869    * @param tableNameOrRegionName table or region to major compact
1870    * @throws IOException if a remote or network exception occurs
1871    * @throws InterruptedException
1872    * @return the current compaction state
1873    */
1874   public CompactionState getCompactionState(final byte [] tableNameOrRegionName)
1875       throws IOException, InterruptedException {
1876     CompactionState state = CompactionState.NONE;
1877     CatalogTracker ct = getCatalogTracker();
1878     try {
1879       Pair<HRegionInfo, ServerName> regionServerPair
1880         = getRegion(tableNameOrRegionName, ct);
1881       if (regionServerPair != null) {
1882         if (regionServerPair.getSecond() == null) {
1883           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1884         } else {
1885           ServerName sn = regionServerPair.getSecond();
1886           HRegionInterface rs =
1887             this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1888           return CompactionState.valueOf(
1889             rs.getCompactionState(regionServerPair.getFirst().getRegionName()));
1890         }
1891       } else {
1892         final String tableName = tableNameString(tableNameOrRegionName, ct);
1893         List<Pair<HRegionInfo, ServerName>> pairs =
1894           MetaReader.getTableRegionsAndLocations(ct, tableName);
1895         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1896           if (pair.getFirst().isOffline()) continue;
1897           if (pair.getSecond() == null) continue;
1898           try {
1899             ServerName sn = pair.getSecond();
1900             HRegionInterface rs =
1901               this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
1902             switch (CompactionState.valueOf(
1903               rs.getCompactionState(pair.getFirst().getRegionName()))) {
1904             case MAJOR_AND_MINOR:
1905               return CompactionState.MAJOR_AND_MINOR;
1906             case MAJOR:
1907               if (state == CompactionState.MINOR) {
1908                 return CompactionState.MAJOR_AND_MINOR;
1909               }
1910               state = CompactionState.MAJOR;
1911               break;
1912             case MINOR:
1913               if (state == CompactionState.MAJOR) {
1914                 return CompactionState.MAJOR_AND_MINOR;
1915               }
1916               state = CompactionState.MINOR;
1917               break;
1918             case NONE:
1919               default: // nothing, continue
1920             }
1921           } catch (NotServingRegionException e) {
1922             if (LOG.isDebugEnabled()) {
1923               LOG.debug("Trying to get compaction state of " +
1924                 pair.getFirst() + ": " +
1925                 StringUtils.stringifyException(e));
1926             }
1927           }
1928         }
1929       }
1930     } finally {
1931       cleanupCatalogTracker(ct);
1932     }
1933     return state;
1934   }
1935 
1936   /**
1937    * Creates and returns a proxy to the CoprocessorProtocol instance running in the
1938    * master.
1939    *
1940    * @param protocol The class or interface defining the remote protocol
1941    * @return A CoprocessorProtocol instance
1942    */
1943   public <T extends CoprocessorProtocol> T coprocessorProxy(
1944       Class<T> protocol) {
1945     return (T) Proxy.newProxyInstance(this.getClass().getClassLoader(),
1946         new Class[]{protocol},
1947         new MasterExecRPCInvoker(conf,
1948             connection,
1949             protocol));
1950   }
1951 
1952 
1953   /**
1954    * Create a timestamp consistent snapshot for the given table.
1955    * <p>
1956    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
1957    * snapshot with the same name (even a different type or with different parameters) will fail with
1958    * a {@link SnapshotCreationException} indicating the duplicate naming.
1959    * <p>
1960    * Snapshot names follow the same naming constraints as tables in HBase. See
1961    * {@link HTableDescriptor#isLegalTableName(byte[])}.
1962    * @param snapshotName name of the snapshot to be created
1963    * @param tableName name of the table for which snapshot is created
1964    * @throws IOException if a remote or network exception occurs
1965    * @throws SnapshotCreationException if snapshot creation failed
1966    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
1967    */
1968   public void snapshot(final String snapshotName, final String tableName) throws IOException,
1969       SnapshotCreationException, IllegalArgumentException {
1970     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
1971   }
1972 
1973   /**
1974    * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
1975    * taken. If the table is disabled, an offline snapshot is taken.
1976    * <p>
1977    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
1978    * snapshot with the same name (even a different type or with different parameters) will fail with
1979    * a {@link SnapshotCreationException} indicating the duplicate naming.
1980    * <p>
1981    * Snapshot names follow the same naming constraints as tables in HBase. See
1982    * {@link HTableDescriptor#isLegalTableName(byte[])}.
1983    * @param snapshotName name of the snapshot to be created
1984    * @param tableName name of the table for which snapshot is created
1985    * @throws IOException if a remote or network exception occurs
1986    * @throws SnapshotCreationException if snapshot creation failed
1987    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
1988    */
1989   public void snapshot(final byte[] snapshotName, final byte[] tableName) throws IOException,
1990       SnapshotCreationException, IllegalArgumentException {
1991     snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
1992   }
1993 
1994   /**
1995    * Create typed snapshot of the table.
1996    * <p>
1997    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
1998    * snapshot with the same name (even a different type or with different parameters) will fail with
1999    * a {@link SnapshotCreationException} indicating the duplicate naming.
2000    * <p>
2001    * Snapshot names follow the same naming constraints as tables in HBase. See
2002    * {@link HTableDescriptor#isLegalTableName(byte[])}.
2003    * <p>
2004    * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
2005    * snapshots stored on the cluster
2006    * @param tableName name of the table to snapshot
2007    * @param type type of snapshot to take
2008    * @throws IOException we fail to reach the master
2009    * @throws SnapshotCreationException if snapshot creation failed
2010    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2011    */
2012   public void snapshot(final String snapshotName, final String tableName,
2013       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2014       IllegalArgumentException {
2015     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
2016     builder.setTable(tableName);
2017     builder.setName(snapshotName);
2018     builder.setType(type);
2019     snapshot(builder.build());
2020   }
2021 
2022   /**
2023    * Take a snapshot and wait for the server to complete that snapshot (blocking).
2024    * <p>
2025    * Only a single snapshot should be taken at a time for an instance of HBase, or results may be
2026    * undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
2027    * time for a single cluster).
2028    * <p>
2029    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2030    * snapshot with the same name (even a different type or with different parameters) will fail with
2031    * a {@link SnapshotCreationException} indicating the duplicate naming.
2032    * <p>
2033    * Snapshot names follow the same naming constraints as tables in HBase. See
2034    * {@link HTableDescriptor#isLegalTableName(byte[])}.
2035    * <p>
2036    * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
2037    * unless you are sure about the type of snapshot that you want to take.
2038    * @param snapshot snapshot to take
2039    * @throws IOException or we lose contact with the master.
2040    * @throws SnapshotCreationException if snapshot failed to be taken
2041    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2042    */
2043   public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
2044       IllegalArgumentException {
2045     HSnapshotDescription snapshotWritable = new HSnapshotDescription(snapshot);
2046 
2047     try {
2048       // actually take the snapshot
2049       long max = takeSnapshotAsync(snapshot);
2050       long start = EnvironmentEdgeManager.currentTimeMillis();
2051       long maxPauseTime = max / this.numRetries;
2052       boolean done = false;
2053       int tries = 0;
2054       LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
2055           SnapshotDescriptionUtils.toString(snapshot) + "' to complete. (max " +
2056           maxPauseTime + " ms per retry)");
2057       while (tries == 0 || (EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done) {
2058         try {
2059           // sleep a backoff <= pauseTime amount
2060           long sleep = getPauseTime(tries++);
2061           sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2062           LOG.debug("(#" + tries + ") Sleeping: " + sleep +
2063             "ms while waiting for snapshot completion.");
2064           Thread.sleep(sleep);
2065 
2066         } catch (InterruptedException e) {
2067           LOG.debug("Interrupted while waiting for snapshot " + snapshot + " to complete");
2068           Thread.currentThread().interrupt();
2069         }
2070         LOG.debug("Getting current status of snapshot from master...");
2071         done = getMaster().isSnapshotDone(snapshotWritable);
2072       }
2073 
2074       if (!done) {
2075         throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
2076             + "' wasn't completed in expectedTime:" + max + " ms", snapshot);
2077       }
2078     } catch (RemoteException e) {
2079       throw RemoteExceptionHandler.decodeRemoteException(e);
2080     }
2081   }
2082 
2083   /**
2084    * Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
2085    * <p>
2086    * Only a single snapshot should be taken at a time, or results may be undefined.
2087    * @param snapshot snapshot to take
2088    * @return the max time in millis to wait for the snapshot
2089    * @throws IOException if the snapshot did not succeed or we lose contact with the master.
2090    * @throws SnapshotCreationException if snapshot creation failed
2091    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2092    */
2093   public long takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
2094       SnapshotCreationException {
2095     SnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
2096     HSnapshotDescription snapshotWritable = new HSnapshotDescription(snapshot);
2097     return getMaster().snapshot(snapshotWritable);
2098   }
2099 
2100   /**
2101    * Check the current state of the passed snapshot.
2102    * <p>
2103    * There are three possible states:
2104    * <ol>
2105    * <li>running - returns <tt>false</tt></li>
2106    * <li>finished - returns <tt>true</tt></li>
2107    * <li>finished with error - throws the exception that caused the snapshot to fail</li>
2108    * </ol>
2109    * <p>
2110    * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
2111    * run/started since the snapshot your are checking, you will recieve an
2112    * {@link UnknownSnapshotException}.
2113    * @param snapshot description of the snapshot to check
2114    * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
2115    * running
2116    * @throws IOException if we have a network issue
2117    * @throws HBaseSnapshotException if the snapshot failed
2118    * @throws UnknownSnapshotException if the requested snapshot is unknown
2119    */
2120   public boolean isSnapshotFinished(final SnapshotDescription snapshot)
2121       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
2122     try {
2123       return getMaster().isSnapshotDone(new HSnapshotDescription(snapshot));
2124     } catch (RemoteException e) {
2125       throw RemoteExceptionHandler.decodeRemoteException(e);
2126     }
2127   }
2128 
2129   /**
2130    * Restore the specified snapshot on the original table. (The table must be disabled)
2131    * Before restoring the table, a new snapshot with the current table state is created.
2132    * In case of failure, the table will be rolled back to the its original state.
2133    *
2134    * @param snapshotName name of the snapshot to restore
2135    * @throws IOException if a remote or network exception occurs
2136    * @throws RestoreSnapshotException if snapshot failed to be restored
2137    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2138    */
2139   public void restoreSnapshot(final byte[] snapshotName)
2140       throws IOException, RestoreSnapshotException {
2141     restoreSnapshot(Bytes.toString(snapshotName));
2142   }
2143 
2144   /**
2145    * Restore the specified snapshot on the original table. (The table must be disabled)
2146    * Before restoring the table, a new snapshot with the current table state is created.
2147    * In case of failure, the table will be rolled back to its original state.
2148    *
2149    * @param snapshotName name of the snapshot to restore
2150    * @throws IOException if a remote or network exception occurs
2151    * @throws RestoreSnapshotException if snapshot failed to be restored
2152    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2153    */
2154   public void restoreSnapshot(final String snapshotName)
2155       throws IOException, RestoreSnapshotException {
2156     String rollbackSnapshot = snapshotName + "-" + EnvironmentEdgeManager.currentTimeMillis();
2157 
2158     String tableName = null;
2159     for (SnapshotDescription snapshotInfo: listSnapshots()) {
2160       if (snapshotInfo.getName().equals(snapshotName)) {
2161         tableName = snapshotInfo.getTable();
2162         break;
2163       }
2164     }
2165 
2166     if (tableName == null) {
2167       throw new RestoreSnapshotException(
2168         "Unable to find the table name for snapshot=" + snapshotName);
2169     }
2170 
2171     // Take a snapshot of the current state
2172     snapshot(rollbackSnapshot, tableName);
2173 
2174     // Restore snapshot
2175     try {
2176       internalRestoreSnapshot(snapshotName, tableName);
2177     } catch (IOException e) {
2178       // Try to rollback
2179       try {
2180         String msg = "Restore snapshot=" + snapshotName +
2181           " failed. Rollback to snapshot=" + rollbackSnapshot + " succeeded.";
2182         LOG.error(msg, e);
2183         internalRestoreSnapshot(rollbackSnapshot, tableName);
2184         throw new RestoreSnapshotException(msg, e);
2185       } catch (IOException ex) {
2186         String msg = "Failed to restore and rollback to snapshot=" + rollbackSnapshot;
2187         LOG.error(msg, ex);
2188         throw new RestoreSnapshotException(msg, ex);
2189       }
2190     }
2191   }
2192 
2193   /**
2194    * Create a new table by cloning the snapshot content.
2195    *
2196    * @param snapshotName name of the snapshot to be cloned
2197    * @param tableName name of the table where the snapshot will be restored
2198    * @throws IOException if a remote or network exception occurs
2199    * @throws TableExistsException if table to be created already exists
2200    * @throws RestoreSnapshotException if snapshot failed to be cloned
2201    * @throws IllegalArgumentException if the specified table has not a valid name
2202    */
2203   public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
2204       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
2205     cloneSnapshot(Bytes.toString(snapshotName), Bytes.toString(tableName));
2206   }
2207 
2208   /**
2209    * Create a new table by cloning the snapshot content.
2210    *
2211    * @param snapshotName name of the snapshot to be cloned
2212    * @param tableName name of the table where the snapshot will be restored
2213    * @throws IOException if a remote or network exception occurs
2214    * @throws TableExistsException if table to be created already exists
2215    * @throws RestoreSnapshotException if snapshot failed to be cloned
2216    * @throws IllegalArgumentException if the specified table has not a valid name
2217    */
2218   public void cloneSnapshot(final String snapshotName, final String tableName)
2219       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
2220     if (tableExists(tableName)) {
2221       throw new TableExistsException("Table '" + tableName + " already exists");
2222     }
2223     internalRestoreSnapshot(snapshotName, tableName);
2224     waitUntilTableIsEnabled(Bytes.toBytes(tableName));
2225   }
2226 
2227   /**
2228    * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
2229    * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
2230    * create an HTable instance to this table before it is available.
2231    * @param snapshot snapshot to restore
2232    * @param tableName table name to restore the snapshot on
2233    * @throws IOException if a remote or network exception occurs
2234    * @throws RestoreSnapshotException if snapshot failed to be restored
2235    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2236    */
2237   private void internalRestoreSnapshot(final String snapshotName, final String tableName)
2238       throws IOException, RestoreSnapshotException {
2239     HSnapshotDescription snapshot = new HSnapshotDescription(
2240       SnapshotDescription.newBuilder().setName(snapshotName).setTable(tableName).build());
2241 
2242     try {
2243       // actually restore the snapshot
2244       getMaster().restoreSnapshot(snapshot);
2245 
2246       final long maxPauseTime = 5000;
2247       boolean done = false;
2248       int tries = 0;
2249       while (!done) {
2250         try {
2251           // sleep a backoff <= pauseTime amount
2252           long sleep = getPauseTime(tries++);
2253           sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2254           LOG.debug(tries + ") Sleeping: " + sleep + " ms while we wait for snapshot restore to complete.");
2255           Thread.sleep(sleep);
2256         } catch (InterruptedException e) {
2257           LOG.debug("Interrupted while waiting for snapshot " + snapshot + " restore to complete");
2258           Thread.currentThread().interrupt();
2259         }
2260         LOG.debug("Getting current status of snapshot restore from master...");
2261         done = getMaster().isRestoreSnapshotDone(snapshot);
2262       }
2263       if (!done) {
2264         throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + "' wasn't restored.");
2265       }
2266     } catch (RemoteException e) {
2267       throw RemoteExceptionHandler.decodeRemoteException(e);
2268     }
2269   }
2270 
2271   /**
2272    * List completed snapshots.
2273    * @return a list of snapshot descriptors for completed snapshots
2274    * @throws IOException if a network error occurs
2275    */
2276   public List<SnapshotDescription> listSnapshots() throws IOException {
2277     List<SnapshotDescription> snapshots = new LinkedList<SnapshotDescription>();
2278     try {
2279       for (HSnapshotDescription snapshot: getMaster().getCompletedSnapshots()) {
2280         snapshots.add(snapshot.getProto());
2281       }
2282     } catch (RemoteException e) {
2283       throw RemoteExceptionHandler.decodeRemoteException(e);
2284     }
2285     return snapshots;
2286   }
2287 
2288   /**
2289    * Delete an existing snapshot.
2290    * @param snapshotName name of the snapshot
2291    * @throws IOException if a remote or network exception occurs
2292    */
2293   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
2294     // make sure the snapshot is possibly valid
2295     HTableDescriptor.isLegalTableName(snapshotName);
2296     // do the delete
2297     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
2298       .setName(Bytes.toString(snapshotName)).build();
2299     try {
2300       getMaster().deleteSnapshot(new HSnapshotDescription(snapshot));
2301     } catch (RemoteException e) {
2302       throw RemoteExceptionHandler.decodeRemoteException(e);
2303     }
2304   }
2305 
2306   /**
2307    * Delete an existing snapshot.
2308    * @param snapshotName name of the snapshot
2309    * @throws IOException if a remote or network exception occurs
2310    */
2311   public void deleteSnapshot(final String snapshotName) throws IOException {
2312     deleteSnapshot(Bytes.toBytes(snapshotName));
2313   }
2314 }