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