View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.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.ArrayList;
26  import java.util.Arrays;
27  import java.util.HashMap;
28  import java.util.LinkedList;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.Map.Entry;
32  import java.util.concurrent.atomic.AtomicInteger;
33  import java.util.concurrent.atomic.AtomicReference;
34  import java.util.regex.Pattern;
35  
36  import org.apache.commons.logging.Log;
37  import org.apache.commons.logging.LogFactory;
38  import org.apache.hadoop.classification.InterfaceAudience;
39  import org.apache.hadoop.classification.InterfaceStability;
40  import org.apache.hadoop.conf.Configuration;
41  import org.apache.hadoop.hbase.Abortable;
42  import org.apache.hadoop.hbase.ClusterStatus;
43  import org.apache.hadoop.hbase.HBaseConfiguration;
44  import org.apache.hadoop.hbase.HBaseIOException;
45  import org.apache.hadoop.hbase.HColumnDescriptor;
46  import org.apache.hadoop.hbase.HConstants;
47  import org.apache.hadoop.hbase.HRegionInfo;
48  import org.apache.hadoop.hbase.HRegionLocation;
49  import org.apache.hadoop.hbase.HTableDescriptor;
50  import org.apache.hadoop.hbase.MasterNotRunningException;
51  import org.apache.hadoop.hbase.MetaTableAccessor;
52  import org.apache.hadoop.hbase.NamespaceDescriptor;
53  import org.apache.hadoop.hbase.NotServingRegionException;
54  import org.apache.hadoop.hbase.RegionException;
55  import org.apache.hadoop.hbase.RegionLocations;
56  import org.apache.hadoop.hbase.ServerName;
57  import org.apache.hadoop.hbase.TableExistsException;
58  import org.apache.hadoop.hbase.TableName;
59  import org.apache.hadoop.hbase.TableNotDisabledException;
60  import org.apache.hadoop.hbase.TableNotEnabledException;
61  import org.apache.hadoop.hbase.TableNotFoundException;
62  import org.apache.hadoop.hbase.UnknownRegionException;
63  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
64  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
65  import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
66  import org.apache.hadoop.hbase.exceptions.DeserializationException;
67  import org.apache.hadoop.hbase.exceptions.MergeRegionException;
68  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
69  import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
70  import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
71  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
72  import org.apache.hadoop.hbase.protobuf.RequestConverter;
73  import org.apache.hadoop.hbase.protobuf.ResponseConverter;
74  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
75  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
76  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
77  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
78  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
79  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
80  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
81  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
82  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
83  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
84  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
85  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService;
86  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanRequest;
87  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanResponse;
88  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
89  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
90  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
91  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
92  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
93  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
94  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
95  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
96  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
97  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
98  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
99  import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
100 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
101 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
102 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
103 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
104 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
105 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
106 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
107 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
108 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
109 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
110 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
111 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
112 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
113 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
114 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
115 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
116 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
117 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
118 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
119 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
120 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
121 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
122 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
123 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
124 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
125 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
126 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
127 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
128 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
129 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
130 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
131 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
132 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
133 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
134 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
135 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
136 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
137 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
138 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
139 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
140 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
141 import org.apache.hadoop.hbase.util.Addressing;
142 import org.apache.hadoop.hbase.util.Bytes;
143 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
144 import org.apache.hadoop.hbase.util.Pair;
145 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
146 import org.apache.hadoop.ipc.RemoteException;
147 import org.apache.hadoop.util.StringUtils;
148 import org.apache.zookeeper.KeeperException;
149 
150 import com.google.protobuf.ByteString;
151 import com.google.protobuf.ServiceException;
152 
153 /**
154  * Provides an interface to manage HBase database table metadata + general
155  * administrative functions.  Use HBaseAdmin to create, drop, list, enable and
156  * disable tables. Use it also to add and drop table column families.
157  *
158  * <p>See {@link HTable} to add, update, and delete data from an individual table.
159  * <p>Currently HBaseAdmin instances are not expected to be long-lived.  For
160  * example, an HBaseAdmin instance will not ride over a Master restart.
161  */
162 @InterfaceAudience.Public
163 @InterfaceStability.Evolving
164 public class HBaseAdmin implements Admin {
165   private static final Log LOG = LogFactory.getLog(HBaseAdmin.class);
166 
167   private static final String ZK_IDENTIFIER_PREFIX =  "hbase-admin-on-";
168 
169   // We use the implementation class rather then the interface because we
170   //  need the package protected functions to get the connection to master
171   private ClusterConnection connection;
172 
173   private volatile Configuration conf;
174   private final long pause;
175   private final int numRetries;
176   // Some operations can take a long time such as disable of big table.
177   // numRetries is for 'normal' stuff... Multiply by this factor when
178   // want to wait a long time.
179   private final int retryLongerMultiplier;
180   private boolean aborted;
181   private boolean cleanupConnectionOnClose = false; // close the connection in close()
182   private boolean closed = false;
183   private int operationTimeout;
184 
185   private RpcRetryingCallerFactory rpcCallerFactory;
186 
187   /**
188    * Constructor.
189    * See {@link #HBaseAdmin(HConnection connection)}
190    *
191    * @param c Configuration object. Copied internally.
192    */
193   public HBaseAdmin(Configuration c)
194   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
195     // Will not leak connections, as the new implementation of the constructor
196     // does not throw exceptions anymore.
197     this(ConnectionManager.getConnectionInternal(new Configuration(c)));
198     this.cleanupConnectionOnClose = true;
199   }
200 
201   @Override
202   public int getOperationTimeout() {
203     return operationTimeout;
204   }
205 
206 
207   /**
208    * Constructor for externally managed HConnections.
209    * The connection to master will be created when required by admin functions.
210    *
211    * @param connection The HConnection instance to use
212    * @throws MasterNotRunningException, ZooKeeperConnectionException are not
213    *  thrown anymore but kept into the interface for backward api compatibility
214    * @deprecated Do not use this internal ctor.
215    */
216   @Deprecated
217   public HBaseAdmin(HConnection connection)
218       throws MasterNotRunningException, ZooKeeperConnectionException {
219     this((ClusterConnection)connection);
220   }
221 
222   HBaseAdmin(ClusterConnection connection) {
223     this.conf = connection.getConfiguration();
224     this.connection = connection;
225 
226     this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
227         HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
228     this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
229         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
230     this.retryLongerMultiplier = this.conf.getInt(
231         "hbase.client.retries.longer.multiplier", 10);
232     this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
233         HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
234 
235     this.rpcCallerFactory = RpcRetryingCallerFactory.instantiate(this.conf);
236   }
237 
238   @Override
239   public void abort(String why, Throwable e) {
240     // Currently does nothing but throw the passed message and exception
241     this.aborted = true;
242     throw new RuntimeException(why, e);
243   }
244 
245   @Override
246   public boolean isAborted(){
247     return this.aborted;
248   }
249 
250   /** @return HConnection used by this object. */
251   @Override
252   public HConnection getConnection() {
253     return connection;
254   }
255 
256   /** @return - true if the master server is running. Throws an exception
257    *  otherwise.
258    * @throws ZooKeeperConnectionException
259    * @throws MasterNotRunningException
260    */
261   @Override
262   public boolean isMasterRunning()
263   throws MasterNotRunningException, ZooKeeperConnectionException {
264     return connection.isMasterRunning();
265   }
266 
267   /**
268    * @param tableName Table to check.
269    * @return True if table exists already.
270    * @throws IOException
271    */
272   public boolean tableExists(final TableName tableName) throws IOException {
273     return MetaTableAccessor.tableExists(connection, tableName);
274   }
275 
276   public boolean tableExists(final byte[] tableName)
277   throws IOException {
278     return tableExists(TableName.valueOf(tableName));
279   }
280 
281   public boolean tableExists(final String tableName)
282   throws IOException {
283     return tableExists(TableName.valueOf(tableName));
284   }
285 
286   /**
287    * List all the userspace tables.  In other words, scan the hbase:meta table.
288    *
289    * If we wanted this to be really fast, we could implement a special
290    * catalog table that just contains table names and their descriptors.
291    * Right now, it only exists as part of the hbase:meta table's region info.
292    *
293    * @return - returns an array of HTableDescriptors
294    * @throws IOException if a remote or network exception occurs
295    */
296   @Override
297   public HTableDescriptor[] listTables() throws IOException {
298     return this.connection.listTables();
299   }
300 
301   /**
302    * List all the userspace tables matching the given pattern.
303    *
304    * @param pattern The compiled regular expression to match against
305    * @return - returns an array of HTableDescriptors
306    * @throws IOException if a remote or network exception occurs
307    * @see #listTables()
308    */
309   @Override
310   public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
311     List<HTableDescriptor> matched = new LinkedList<HTableDescriptor>();
312     HTableDescriptor[] tables = listTables();
313     for (HTableDescriptor table : tables) {
314       if (pattern.matcher(table.getTableName().getNameAsString()).matches()) {
315         matched.add(table);
316       }
317     }
318     return matched.toArray(new HTableDescriptor[matched.size()]);
319   }
320 
321   /**
322    * List all the userspace tables matching the given regular expression.
323    *
324    * @param regex The regular expression to match against
325    * @return - returns an array of HTableDescriptors
326    * @throws IOException if a remote or network exception occurs
327    * @see #listTables(java.util.regex.Pattern)
328    */
329   @Override
330   public HTableDescriptor[] listTables(String regex) throws IOException {
331     return listTables(Pattern.compile(regex));
332   }
333 
334   /**
335    * List all of the names of userspace tables.
336    * @return String[] table names
337    * @throws IOException if a remote or network exception occurs
338    */
339   @Deprecated
340   public String[] getTableNames() throws IOException {
341     return this.connection.getTableNames();
342   }
343 
344   /**
345    * List all of the names of userspace tables matching the given regular expression.
346    * @param pattern The regular expression to match against
347    * @return String[] table names
348    * @throws IOException if a remote or network exception occurs
349    */
350   @Deprecated
351   public String[] getTableNames(Pattern pattern) throws IOException {
352     List<String> matched = new ArrayList<String>();
353     for (String name: this.connection.getTableNames()) {
354       if (pattern.matcher(name).matches()) {
355         matched.add(name);
356       }
357     }
358     return matched.toArray(new String[matched.size()]);
359   }
360 
361   /**
362    * List all of the names of userspace tables matching the given regular expression.
363    * @param regex The regular expression to match against
364    * @return String[] table names
365    * @throws IOException if a remote or network exception occurs
366    */
367   @Deprecated
368   public String[] getTableNames(String regex) throws IOException {
369     return getTableNames(Pattern.compile(regex));
370   }
371 
372   /**
373    * List all of the names of userspace tables.
374    * @return TableName[] table names
375    * @throws IOException if a remote or network exception occurs
376    */
377   @Override
378   public TableName[] listTableNames() throws IOException {
379     return this.connection.listTableNames();
380   }
381 
382   /**
383    * Method for getting the tableDescriptor
384    * @param tableName as a byte []
385    * @return the tableDescriptor
386    * @throws TableNotFoundException
387    * @throws IOException if a remote or network exception occurs
388    */
389   @Override
390   public HTableDescriptor getTableDescriptor(final TableName tableName)
391   throws TableNotFoundException, IOException {
392     return this.connection.getHTableDescriptor(tableName);
393   }
394 
395   public HTableDescriptor getTableDescriptor(final byte[] tableName)
396   throws TableNotFoundException, IOException {
397     return getTableDescriptor(TableName.valueOf(tableName));
398   }
399 
400   private long getPauseTime(int tries) {
401     int triesCount = tries;
402     if (triesCount >= HConstants.RETRY_BACKOFF.length) {
403       triesCount = HConstants.RETRY_BACKOFF.length - 1;
404     }
405     return this.pause * HConstants.RETRY_BACKOFF[triesCount];
406   }
407 
408   /**
409    * Creates a new table.
410    * Synchronous operation.
411    *
412    * @param desc table descriptor for table
413    *
414    * @throws IllegalArgumentException if the table name is reserved
415    * @throws MasterNotRunningException if master is not running
416    * @throws TableExistsException if table already exists (If concurrent
417    * threads, the table may have been created between test-for-existence
418    * and attempt-at-creation).
419    * @throws IOException if a remote or network exception occurs
420    */
421   @Override
422   public void createTable(HTableDescriptor desc)
423   throws IOException {
424     createTable(desc, null);
425   }
426 
427   /**
428    * Creates a new table with the specified number of regions.  The start key
429    * specified will become the end key of the first region of the table, and
430    * the end key specified will become the start key of the last region of the
431    * table (the first region has a null start key and the last region has a
432    * null end key).
433    *
434    * BigInteger math will be used to divide the key range specified into
435    * enough segments to make the required number of total regions.
436    *
437    * Synchronous operation.
438    *
439    * @param desc table descriptor for table
440    * @param startKey beginning of key range
441    * @param endKey end of key range
442    * @param numRegions the total number of regions to create
443    *
444    * @throws IllegalArgumentException if the table name is reserved
445    * @throws MasterNotRunningException if master is not running
446    * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
447    * threads, the table may have been created between test-for-existence
448    * and attempt-at-creation).
449    * @throws IOException
450    */
451   @Override
452   public void createTable(HTableDescriptor desc, byte [] startKey,
453       byte [] endKey, int numRegions)
454   throws IOException {
455     if(numRegions < 3) {
456       throw new IllegalArgumentException("Must create at least three regions");
457     } else if(Bytes.compareTo(startKey, endKey) >= 0) {
458       throw new IllegalArgumentException("Start key must be smaller than end key");
459     }
460     if (numRegions == 3) {
461       createTable(desc, new byte[][]{startKey, endKey});
462       return;
463     }
464     byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
465     if(splitKeys == null || splitKeys.length != numRegions - 1) {
466       throw new IllegalArgumentException("Unable to split key range into enough regions");
467     }
468     createTable(desc, splitKeys);
469   }
470 
471   /**
472    * Creates a new table with an initial set of empty regions defined by the
473    * specified split keys.  The total number of regions created will be the
474    * number of split keys plus one. Synchronous operation.
475    * Note : Avoid passing empty split key.
476    *
477    * @param desc table descriptor for table
478    * @param splitKeys array of split keys for the initial regions of the table
479    *
480    * @throws IllegalArgumentException if the table name is reserved, if the split keys
481    * are repeated and if the split key has empty byte array.
482    * @throws MasterNotRunningException if master is not running
483    * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
484    * threads, the table may have been created between test-for-existence
485    * and attempt-at-creation).
486    * @throws IOException
487    */
488   @Override
489   public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
490   throws IOException {
491     try {
492       createTableAsync(desc, splitKeys);
493     } catch (SocketTimeoutException ste) {
494       LOG.warn("Creating " + desc.getTableName() + " took too long", ste);
495     }
496     int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
497     int prevRegCount = 0;
498     boolean doneWithMetaScan = false;
499     for (int tries = 0; tries < this.numRetries * this.retryLongerMultiplier;
500       ++tries) {
501       if (!doneWithMetaScan) {
502         // Wait for new table to come on-line
503         final AtomicInteger actualRegCount = new AtomicInteger(0);
504         MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
505           @Override
506           public boolean processRow(Result rowResult) throws IOException {
507             RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
508             if (list == null) {
509               LOG.warn("No serialized HRegionInfo in " + rowResult);
510               return true;
511             }
512             HRegionLocation l = list.getRegionLocation();
513             if (l == null) {
514               return true;
515             }
516             if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
517               return false;
518             }
519             if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
520             HRegionLocation[] locations = list.getRegionLocations();
521             for (HRegionLocation location : locations) {
522               if (location == null) continue;
523               ServerName serverName = location.getServerName();
524               // Make sure that regions are assigned to server
525               if (serverName != null && serverName.getHostAndPort() != null) {
526                 actualRegCount.incrementAndGet();
527               }
528             }
529             return true;
530           }
531         };
532         MetaScanner.metaScan(conf, connection, visitor, desc.getTableName());
533         if (actualRegCount.get() < numRegs) {
534           if (tries == this.numRetries * this.retryLongerMultiplier - 1) {
535             throw new RegionOfflineException("Only " + actualRegCount.get() +
536               " of " + numRegs + " regions are online; retries exhausted.");
537           }
538           try { // Sleep
539             Thread.sleep(getPauseTime(tries));
540           } catch (InterruptedException e) {
541             throw new InterruptedIOException("Interrupted when opening" +
542               " regions; " + actualRegCount.get() + " of " + numRegs +
543               " regions processed so far");
544           }
545           if (actualRegCount.get() > prevRegCount) { // Making progress
546             prevRegCount = actualRegCount.get();
547             tries = -1;
548           }
549         } else {
550           doneWithMetaScan = true;
551           tries = -1;
552         }
553       } else if (isTableEnabled(desc.getTableName())) {
554         return;
555       } else {
556         try { // Sleep
557           Thread.sleep(getPauseTime(tries));
558         } catch (InterruptedException e) {
559           throw new InterruptedIOException("Interrupted when waiting" +
560             " for table to be enabled; meta scan was done");
561         }
562       }
563     }
564     throw new TableNotEnabledException(
565       "Retries exhausted while still waiting for table: "
566       + desc.getTableName() + " to be enabled");
567   }
568 
569   /**
570    * Creates a new table but does not block and wait for it to come online.
571    * Asynchronous operation.  To check if the table exists, use
572    * {@link #isTableAvailable} -- it is not safe to create an HTable
573    * instance to this table before it is available.
574    * Note : Avoid passing empty split key.
575    * @param desc table descriptor for table
576    *
577    * @throws IllegalArgumentException Bad table name, if the split keys
578    * are repeated and if the split key has empty byte array.
579    * @throws MasterNotRunningException if master is not running
580    * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
581    * threads, the table may have been created between test-for-existence
582    * and attempt-at-creation).
583    * @throws IOException
584    */
585   @Override
586   public void createTableAsync(
587     final HTableDescriptor desc, final byte [][] splitKeys)
588   throws IOException {
589     if(desc.getTableName() == null) {
590       throw new IllegalArgumentException("TableName cannot be null");
591     }
592     if(splitKeys != null && splitKeys.length > 0) {
593       Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
594       // Verify there are no duplicate split keys
595       byte [] lastKey = null;
596       for(byte [] splitKey : splitKeys) {
597         if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
598           throw new IllegalArgumentException(
599               "Empty split key must not be passed in the split keys.");
600         }
601         if(lastKey != null && Bytes.equals(splitKey, lastKey)) {
602           throw new IllegalArgumentException("All split keys must be unique, " +
603             "found duplicate: " + Bytes.toStringBinary(splitKey) +
604             ", " + Bytes.toStringBinary(lastKey));
605         }
606         lastKey = splitKey;
607       }
608     }
609 
610     executeCallable(new MasterCallable<Void>(getConnection()) {
611       @Override
612       public Void call(int callTimeout) throws ServiceException {
613         CreateTableRequest request = RequestConverter.buildCreateTableRequest(desc, splitKeys);
614         master.createTable(null, request);
615         return null;
616       }
617     });
618   }
619 
620   public void deleteTable(final String tableName) throws IOException {
621     deleteTable(TableName.valueOf(tableName));
622   }
623 
624   public void deleteTable(final byte[] tableName) throws IOException {
625     deleteTable(TableName.valueOf(tableName));
626   }
627 
628   /**
629    * Deletes a table.
630    * Synchronous operation.
631    *
632    * @param tableName name of table to delete
633    * @throws IOException if a remote or network exception occurs
634    */
635   @Override
636   public void deleteTable(final TableName tableName) throws IOException {
637     boolean tableExists = true;
638 
639     executeCallable(new MasterCallable<Void>(getConnection()) {
640       @Override
641       public Void call(int callTimeout) throws ServiceException {
642         DeleteTableRequest req = RequestConverter.buildDeleteTableRequest(tableName);
643         master.deleteTable(null,req);
644         return null;
645       }
646     });
647 
648     int failures = 0;
649     // Wait until all regions deleted
650     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
651       try {
652         HRegionLocation firstMetaServer = getFirstMetaServerForTable(tableName);
653         Scan scan = MetaTableAccessor.getScanForTableName(tableName);
654         scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
655         ScanRequest request = RequestConverter.buildScanRequest(
656           firstMetaServer.getRegionInfo().getRegionName(), scan, 1, true);
657         Result[] values = null;
658         // Get a batch at a time.
659         ClientService.BlockingInterface server = connection.getClient(firstMetaServer
660             .getServerName());
661         PayloadCarryingRpcController controller = new PayloadCarryingRpcController();
662         try {
663           controller.setPriority(tableName);
664           ScanResponse response = server.scan(controller, request);
665           values = ResponseConverter.getResults(controller.cellScanner(), response);
666         } catch (ServiceException se) {
667           throw ProtobufUtil.getRemoteException(se);
668         }
669 
670         // let us wait until hbase:meta table is updated and
671         // HMaster removes the table from its HTableDescriptors
672         if (values == null || values.length == 0) {
673           tableExists = false;
674           GetTableDescriptorsResponse htds;
675           MasterKeepAliveConnection master = connection.getKeepAliveMasterService();
676           try {
677             GetTableDescriptorsRequest req =
678               RequestConverter.buildGetTableDescriptorsRequest(tableName);
679             htds = master.getTableDescriptors(null, req);
680           } catch (ServiceException se) {
681             throw ProtobufUtil.getRemoteException(se);
682           } finally {
683             master.close();
684           }
685           tableExists = !htds.getTableSchemaList().isEmpty();
686           if (!tableExists) {
687             break;
688           }
689         }
690       } catch (IOException ex) {
691         failures++;
692         if(failures == numRetries - 1) {           // no more tries left
693           if (ex instanceof RemoteException) {
694             throw ((RemoteException) ex).unwrapRemoteException();
695           } else {
696             throw ex;
697           }
698         }
699       }
700       try {
701         Thread.sleep(getPauseTime(tries));
702       } catch (InterruptedException e) {
703         throw new InterruptedIOException("Interrupted when waiting" +
704             " for table to be deleted");
705       }
706     }
707 
708     if (tableExists) {
709       throw new IOException("Retries exhausted, it took too long to wait"+
710         " for the table " + tableName + " to be deleted.");
711     }
712     // Delete cached information to prevent clients from using old locations
713     this.connection.clearRegionCache(tableName);
714     LOG.info("Deleted " + tableName);
715   }
716 
717   /**
718    * Deletes tables matching the passed in pattern and wait on completion.
719    *
720    * Warning: Use this method carefully, there is no prompting and the effect is
721    * immediate. Consider using {@link #listTables(java.lang.String)} and
722    * {@link #deleteTable(byte[])}
723    *
724    * @param regex The regular expression to match table names against
725    * @return Table descriptors for tables that couldn't be deleted
726    * @throws IOException
727    * @see #deleteTables(java.util.regex.Pattern)
728    * @see #deleteTable(java.lang.String)
729    */
730   @Override
731   public HTableDescriptor[] deleteTables(String regex) throws IOException {
732     return deleteTables(Pattern.compile(regex));
733   }
734 
735   /**
736    * Delete tables matching the passed in pattern and wait on completion.
737    *
738    * Warning: Use this method carefully, there is no prompting and the effect is
739    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
740    * {@link #deleteTable(byte[])}
741    *
742    * @param pattern The pattern to match table names against
743    * @return Table descriptors for tables that couldn't be deleted
744    * @throws IOException
745    */
746   @Override
747   public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
748     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
749     for (HTableDescriptor table : listTables(pattern)) {
750       try {
751         deleteTable(table.getTableName());
752       } catch (IOException ex) {
753         LOG.info("Failed to delete table " + table.getTableName(), ex);
754         failed.add(table);
755       }
756     }
757     return failed.toArray(new HTableDescriptor[failed.size()]);
758   }
759 
760   /**
761    * Truncate a table.
762    * Synchronous operation.
763    *
764    * @param tableName name of table to truncate
765    * @param preserveSplits True if the splits should be preserved
766    * @throws IOException if a remote or network exception occurs
767    */
768   @Override
769   public void truncateTable(final TableName tableName, final boolean preserveSplits)
770       throws IOException {
771     executeCallable(new MasterCallable<Void>(getConnection()) {
772       @Override
773       public Void call(int callTimeout) throws ServiceException {
774         TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
775           tableName, preserveSplits);
776         master.truncateTable(null, req);
777         return null;
778       }
779     });
780   }
781 
782   /**
783    * Enable a table.  May timeout.  Use {@link #enableTableAsync(byte[])}
784    * and {@link #isTableEnabled(byte[])} instead.
785    * The table has to be in disabled state for it to be enabled.
786    * @param tableName name of the table
787    * @throws IOException if a remote or network exception occurs
788    * There could be couple types of IOException
789    * TableNotFoundException means the table doesn't exist.
790    * TableNotDisabledException means the table isn't in disabled state.
791    * @see #isTableEnabled(byte[])
792    * @see #disableTable(byte[])
793    * @see #enableTableAsync(byte[])
794    */
795   @Override
796   public void enableTable(final TableName tableName)
797   throws IOException {
798     enableTableAsync(tableName);
799 
800     // Wait until all regions are enabled
801     waitUntilTableIsEnabled(tableName);
802 
803     LOG.info("Enabled table " + tableName);
804   }
805 
806   public void enableTable(final byte[] tableName)
807   throws IOException {
808     enableTable(TableName.valueOf(tableName));
809   }
810 
811   public void enableTable(final String tableName)
812   throws IOException {
813     enableTable(TableName.valueOf(tableName));
814   }
815 
816   /**
817    * Wait for the table to be enabled and available
818    * If enabling the table exceeds the retry period, an exception is thrown.
819    * @param tableName name of the table
820    * @throws IOException if a remote or network exception occurs or
821    *    table is not enabled after the retries period.
822    */
823   private void waitUntilTableIsEnabled(final TableName tableName) throws IOException {
824     boolean enabled = false;
825     long start = EnvironmentEdgeManager.currentTimeMillis();
826     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
827       try {
828         enabled = isTableEnabled(tableName);
829       } catch (TableNotFoundException tnfe) {
830         // wait for table to be created
831         enabled = false;
832       }
833       enabled = enabled && isTableAvailable(tableName);
834       if (enabled) {
835         break;
836       }
837       long sleep = getPauseTime(tries);
838       if (LOG.isDebugEnabled()) {
839         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
840           "enabled in " + tableName);
841       }
842       try {
843         Thread.sleep(sleep);
844       } catch (InterruptedException e) {
845         // Do this conversion rather than let it out because do not want to
846         // change the method signature.
847         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
848       }
849     }
850     if (!enabled) {
851       long msec = EnvironmentEdgeManager.currentTimeMillis() - start;
852       throw new IOException("Table '" + tableName +
853         "' not yet enabled, after " + msec + "ms.");
854     }
855   }
856 
857   /**
858    * Brings a table on-line (enables it).  Method returns immediately though
859    * enable of table may take some time to complete, especially if the table
860    * is large (All regions are opened as part of enabling process).  Check
861    * {@link #isTableEnabled(byte[])} to learn when table is fully online.  If
862    * table is taking too long to online, check server logs.
863    * @param tableName
864    * @throws IOException
865    * @since 0.90.0
866    */
867   @Override
868   public void enableTableAsync(final TableName tableName)
869   throws IOException {
870     TableName.isLegalFullyQualifiedTableName(tableName.getName());
871     executeCallable(new MasterCallable<Void>(getConnection()) {
872       @Override
873       public Void call(int callTimeout) throws ServiceException {
874         LOG.info("Started enable of " + tableName);
875         EnableTableRequest req = RequestConverter.buildEnableTableRequest(tableName);
876         master.enableTable(null,req);
877         return null;
878       }
879     });
880   }
881 
882   public void enableTableAsync(final byte[] tableName)
883   throws IOException {
884     enableTable(TableName.valueOf(tableName));
885   }
886 
887   public void enableTableAsync(final String tableName)
888   throws IOException {
889     enableTableAsync(TableName.valueOf(tableName));
890   }
891 
892   /**
893    * Enable tables matching the passed in pattern and wait on completion.
894    *
895    * Warning: Use this method carefully, there is no prompting and the effect is
896    * immediate. Consider using {@link #listTables(java.lang.String)} and
897    * {@link #enableTable(byte[])}
898    *
899    * @param regex The regular expression to match table names against
900    * @throws IOException
901    * @see #enableTables(java.util.regex.Pattern)
902    * @see #enableTable(java.lang.String)
903    */
904   @Override
905   public HTableDescriptor[] enableTables(String regex) throws IOException {
906     return enableTables(Pattern.compile(regex));
907   }
908 
909   /**
910    * Enable tables matching the passed in pattern and wait on completion.
911    *
912    * Warning: Use this method carefully, there is no prompting and the effect is
913    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
914    * {@link #enableTable(byte[])}
915    *
916    * @param pattern The pattern to match table names against
917    * @throws IOException
918    */
919   @Override
920   public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
921     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
922     for (HTableDescriptor table : listTables(pattern)) {
923       if (isTableDisabled(table.getTableName())) {
924         try {
925           enableTable(table.getTableName());
926         } catch (IOException ex) {
927           LOG.info("Failed to enable table " + table.getTableName(), ex);
928           failed.add(table);
929         }
930       }
931     }
932     return failed.toArray(new HTableDescriptor[failed.size()]);
933   }
934 
935   /**
936    * Starts the disable of a table.  If it is being served, the master
937    * will tell the servers to stop serving it.  This method returns immediately.
938    * The disable of a table can take some time if the table is large (all
939    * regions are closed as part of table disable operation).
940    * Call {@link #isTableDisabled(byte[])} to check for when disable completes.
941    * If table is taking too long to online, check server logs.
942    * @param tableName name of table
943    * @throws IOException if a remote or network exception occurs
944    * @see #isTableDisabled(byte[])
945    * @see #isTableEnabled(byte[])
946    * @since 0.90.0
947    */
948   @Override
949   public void disableTableAsync(final TableName tableName) throws IOException {
950     TableName.isLegalFullyQualifiedTableName(tableName.getName());
951     executeCallable(new MasterCallable<Void>(getConnection()) {
952       @Override
953       public Void call(int callTimeout) throws ServiceException {
954         LOG.info("Started disable of " + tableName);
955         DisableTableRequest req = RequestConverter.buildDisableTableRequest(tableName);
956         master.disableTable(null,req);
957         return null;
958       }
959     });
960   }
961 
962   public void disableTableAsync(final byte[] tableName) throws IOException {
963     disableTableAsync(TableName.valueOf(tableName));
964   }
965 
966   public void disableTableAsync(final String tableName) throws IOException {
967     disableTableAsync(TableName.valueOf(tableName));
968   }
969 
970   /**
971    * Disable table and wait on completion.  May timeout eventually.  Use
972    * {@link #disableTableAsync(byte[])} and {@link #isTableDisabled(String)}
973    * instead.
974    * The table has to be in enabled state for it to be disabled.
975    * @param tableName
976    * @throws IOException
977    * There could be couple types of IOException
978    * TableNotFoundException means the table doesn't exist.
979    * TableNotEnabledException means the table isn't in enabled state.
980    */
981   @Override
982   public void disableTable(final TableName tableName)
983   throws IOException {
984     disableTableAsync(tableName);
985     // Wait until table is disabled
986     boolean disabled = false;
987     for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
988       disabled = isTableDisabled(tableName);
989       if (disabled) {
990         break;
991       }
992       long sleep = getPauseTime(tries);
993       if (LOG.isDebugEnabled()) {
994         LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
995           "disabled in " + tableName);
996       }
997       try {
998         Thread.sleep(sleep);
999       } catch (InterruptedException e) {
1000         // Do this conversion rather than let it out because do not want to
1001         // change the method signature.
1002         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
1003       }
1004     }
1005     if (!disabled) {
1006       throw new RegionException("Retries exhausted, it took too long to wait"+
1007         " for the table " + tableName + " to be disabled.");
1008     }
1009     LOG.info("Disabled " + tableName);
1010   }
1011 
1012   public void disableTable(final byte[] tableName)
1013   throws IOException {
1014     disableTable(TableName.valueOf(tableName));
1015   }
1016 
1017   public void disableTable(final String tableName)
1018   throws IOException {
1019     disableTable(TableName.valueOf(tableName));
1020   }
1021 
1022   /**
1023    * Disable tables matching the passed in pattern and wait on completion.
1024    *
1025    * Warning: Use this method carefully, there is no prompting and the effect is
1026    * immediate. Consider using {@link #listTables(java.lang.String)} and
1027    * {@link #disableTable(byte[])}
1028    *
1029    * @param regex The regular expression to match table names against
1030    * @return Table descriptors for tables that couldn't be disabled
1031    * @throws IOException
1032    * @see #disableTables(java.util.regex.Pattern)
1033    * @see #disableTable(java.lang.String)
1034    */
1035   @Override
1036   public HTableDescriptor[] disableTables(String regex) throws IOException {
1037     return disableTables(Pattern.compile(regex));
1038   }
1039 
1040   /**
1041    * Disable tables matching the passed in pattern and wait on completion.
1042    *
1043    * Warning: Use this method carefully, there is no prompting and the effect is
1044    * immediate. Consider using {@link #listTables(java.util.regex.Pattern) } and
1045    * {@link #disableTable(byte[])}
1046    *
1047    * @param pattern The pattern to match table names against
1048    * @return Table descriptors for tables that couldn't be disabled
1049    * @throws IOException
1050    */
1051   @Override
1052   public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
1053     List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
1054     for (HTableDescriptor table : listTables(pattern)) {
1055       if (isTableEnabled(table.getTableName())) {
1056         try {
1057           disableTable(table.getTableName());
1058         } catch (IOException ex) {
1059           LOG.info("Failed to disable table " + table.getTableName(), ex);
1060           failed.add(table);
1061         }
1062       }
1063     }
1064     return failed.toArray(new HTableDescriptor[failed.size()]);
1065   }
1066 
1067   /*
1068    * Checks whether table exists. If not, throws TableNotFoundException
1069    * @param tableName
1070    */
1071   private void checkTableExistence(TableName tableName) throws IOException {
1072     if (!tableExists(tableName)) {
1073       throw new TableNotFoundException(tableName);
1074     }
1075   }
1076 
1077   /**
1078    * @param tableName name of table to check
1079    * @return true if table is on-line
1080    * @throws IOException if a remote or network exception occurs
1081    */
1082   @Override
1083   public boolean isTableEnabled(TableName tableName) throws IOException {
1084     checkTableExistence(tableName);
1085     return connection.isTableEnabled(tableName);
1086   }
1087 
1088   public boolean isTableEnabled(byte[] tableName) throws IOException {
1089     return isTableEnabled(TableName.valueOf(tableName));
1090   }
1091 
1092   public boolean isTableEnabled(String tableName) throws IOException {
1093     return isTableEnabled(TableName.valueOf(tableName));
1094   }
1095 
1096 
1097 
1098   /**
1099    * @param tableName name of table to check
1100    * @return true if table is off-line
1101    * @throws IOException if a remote or network exception occurs
1102    */
1103   @Override
1104   public boolean isTableDisabled(TableName tableName) throws IOException {
1105     checkTableExistence(tableName);
1106     return connection.isTableDisabled(tableName);
1107   }
1108 
1109   public boolean isTableDisabled(byte[] tableName) throws IOException {
1110     return isTableDisabled(TableName.valueOf(tableName));
1111   }
1112 
1113   public boolean isTableDisabled(String tableName) throws IOException {
1114     return isTableDisabled(TableName.valueOf(tableName));
1115   }
1116 
1117   /**
1118    * @param tableName name of table to check
1119    * @return true if all regions of the table are available
1120    * @throws IOException if a remote or network exception occurs
1121    */
1122   @Override
1123   public boolean isTableAvailable(TableName tableName) throws IOException {
1124     return connection.isTableAvailable(tableName);
1125   }
1126 
1127   public boolean isTableAvailable(byte[] tableName) throws IOException {
1128     return isTableAvailable(TableName.valueOf(tableName));
1129   }
1130 
1131   public boolean isTableAvailable(String tableName) throws IOException {
1132     return isTableAvailable(TableName.valueOf(tableName));
1133   }
1134 
1135   /**
1136    * Use this api to check if the table has been created with the specified number of
1137    * splitkeys which was used while creating the given table.
1138    * Note : If this api is used after a table's region gets splitted, the api may return
1139    * false.
1140    * @param tableName
1141    *          name of table to check
1142    * @param splitKeys
1143    *          keys to check if the table has been created with all split keys
1144    * @throws IOException
1145    *           if a remote or network excpetion occurs
1146    */
1147   @Override
1148   public boolean isTableAvailable(TableName tableName,
1149                                   byte[][] splitKeys) throws IOException {
1150     return connection.isTableAvailable(tableName, splitKeys);
1151   }
1152 
1153   public boolean isTableAvailable(byte[] tableName,
1154                                   byte[][] splitKeys) throws IOException {
1155     return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1156   }
1157 
1158   public boolean isTableAvailable(String tableName,
1159                                   byte[][] splitKeys) throws IOException {
1160     return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1161   }
1162 
1163   /**
1164    * Get the status of alter command - indicates how many regions have received
1165    * the updated schema Asynchronous operation.
1166    *
1167    * @param tableName TableName instance
1168    * @return Pair indicating the number of regions updated Pair.getFirst() is the
1169    *         regions that are yet to be updated Pair.getSecond() is the total number
1170    *         of regions of the table
1171    * @throws IOException
1172    *           if a remote or network exception occurs
1173    */
1174   @Override
1175   public Pair<Integer, Integer> getAlterStatus(final TableName tableName)
1176   throws IOException {
1177     return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
1178       @Override
1179       public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
1180         GetSchemaAlterStatusRequest req = RequestConverter
1181             .buildGetSchemaAlterStatusRequest(tableName);
1182         GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(null, req);
1183         Pair<Integer, Integer> pair = new Pair<Integer, Integer>(Integer.valueOf(ret
1184             .getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions()));
1185         return pair;
1186       }
1187     });
1188   }
1189 
1190   /**
1191    * Get the status of alter command - indicates how many regions have received
1192    * the updated schema Asynchronous operation.
1193    *
1194    * @param tableName
1195    *          name of the table to get the status of
1196    * @return Pair indicating the number of regions updated Pair.getFirst() is the
1197    *         regions that are yet to be updated Pair.getSecond() is the total number
1198    *         of regions of the table
1199    * @throws IOException
1200    *           if a remote or network exception occurs
1201    */
1202   @Override
1203   public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
1204    throws IOException {
1205     return getAlterStatus(TableName.valueOf(tableName));
1206   }
1207 
1208   /**
1209    * Add a column to an existing table.
1210    * Asynchronous operation.
1211    *
1212    * @param tableName name of the table to add column to
1213    * @param column column descriptor of column to be added
1214    * @throws IOException if a remote or network exception occurs
1215    */
1216   public void addColumn(final byte[] tableName, HColumnDescriptor column)
1217   throws IOException {
1218     addColumn(TableName.valueOf(tableName), column);
1219   }
1220 
1221 
1222   /**
1223    * Add a column to an existing table.
1224    * Asynchronous operation.
1225    *
1226    * @param tableName name of the table to add column to
1227    * @param column column descriptor of column to be added
1228    * @throws IOException if a remote or network exception occurs
1229    */
1230   public void addColumn(final String tableName, HColumnDescriptor column)
1231   throws IOException {
1232     addColumn(TableName.valueOf(tableName), column);
1233   }
1234 
1235   /**
1236    * Add a column to an existing table.
1237    * Asynchronous operation.
1238    *
1239    * @param tableName name of the table to add column to
1240    * @param column column descriptor of column to be added
1241    * @throws IOException if a remote or network exception occurs
1242    */
1243   @Override
1244   public void addColumn(final TableName tableName, final HColumnDescriptor column)
1245   throws IOException {
1246     executeCallable(new MasterCallable<Void>(getConnection()) {
1247       @Override
1248       public Void call(int callTimeout) throws ServiceException {
1249         AddColumnRequest req = RequestConverter.buildAddColumnRequest(tableName, column);
1250         master.addColumn(null,req);
1251         return null;
1252       }
1253     });
1254   }
1255 
1256   /**
1257    * Delete a column from a table.
1258    * Asynchronous operation.
1259    *
1260    * @param tableName name of table
1261    * @param columnName name of column to be deleted
1262    * @throws IOException if a remote or network exception occurs
1263    */
1264   public void deleteColumn(final byte[] tableName, final String columnName)
1265   throws IOException {
1266     deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
1267   }
1268 
1269   /**
1270    * Delete a column from a table.
1271    * Asynchronous operation.
1272    *
1273    * @param tableName name of table
1274    * @param columnName name of column to be deleted
1275    * @throws IOException if a remote or network exception occurs
1276    */
1277   public void deleteColumn(final String tableName, final String columnName)
1278   throws IOException {
1279     deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
1280   }
1281 
1282   /**
1283    * Delete a column from a table.
1284    * Asynchronous operation.
1285    *
1286    * @param tableName name of table
1287    * @param columnName name of column to be deleted
1288    * @throws IOException if a remote or network exception occurs
1289    */
1290   @Override
1291   public void deleteColumn(final TableName tableName, final byte [] columnName)
1292   throws IOException {
1293     executeCallable(new MasterCallable<Void>(getConnection()) {
1294       @Override
1295       public Void call(int callTimeout) throws ServiceException {
1296         DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(tableName, columnName);
1297         master.deleteColumn(null,req);
1298         return null;
1299       }
1300     });
1301   }
1302 
1303   /**
1304    * Modify an existing column family on a table.
1305    * Asynchronous operation.
1306    *
1307    * @param tableName name of table
1308    * @param descriptor new column descriptor to use
1309    * @throws IOException if a remote or network exception occurs
1310    */
1311   public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
1312   throws IOException {
1313     modifyColumn(TableName.valueOf(tableName), descriptor);
1314   }
1315 
1316   /**
1317    * Modify an existing column family on a table.
1318    * Asynchronous operation.
1319    *
1320    * @param tableName name of table
1321    * @param descriptor new column descriptor to use
1322    * @throws IOException if a remote or network exception occurs
1323    */
1324   public void modifyColumn(final byte[] tableName, HColumnDescriptor descriptor)
1325   throws IOException {
1326     modifyColumn(TableName.valueOf(tableName), descriptor);
1327   }
1328 
1329 
1330 
1331   /**
1332    * Modify an existing column family on a table.
1333    * Asynchronous operation.
1334    *
1335    * @param tableName name of table
1336    * @param descriptor new column descriptor to use
1337    * @throws IOException if a remote or network exception occurs
1338    */
1339   @Override
1340   public void modifyColumn(final TableName tableName, final HColumnDescriptor descriptor)
1341   throws IOException {
1342     executeCallable(new MasterCallable<Void>(getConnection()) {
1343       @Override
1344       public Void call(int callTimeout) throws ServiceException {
1345         ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(tableName, descriptor);
1346         master.modifyColumn(null,req);
1347         return null;
1348       }
1349     });
1350   }
1351 
1352   /**
1353    * Close a region. For expert-admins.  Runs close on the regionserver.  The
1354    * master will not be informed of the close.
1355    * @param regionname region name to close
1356    * @param serverName If supplied, we'll use this location rather than
1357    * the one currently in <code>hbase:meta</code>
1358    * @throws IOException if a remote or network exception occurs
1359    */
1360   @Override
1361   public void closeRegion(final String regionname, final String serverName)
1362   throws IOException {
1363     closeRegion(Bytes.toBytes(regionname), serverName);
1364   }
1365 
1366   /**
1367    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1368    * master will not be informed of the close.
1369    * @param regionname region name to close
1370    * @param serverName The servername of the regionserver.  If passed null we
1371    * will use servername found in the hbase:meta table. A server name
1372    * is made of host, port and startcode.  Here is an example:
1373    * <code> host187.example.com,60020,1289493121758</code>
1374    * @throws IOException if a remote or network exception occurs
1375    */
1376   @Override
1377   public void closeRegion(final byte [] regionname, final String serverName)
1378       throws IOException {
1379     if (serverName != null) {
1380       Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
1381       if (pair == null || pair.getFirst() == null) {
1382         throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1383       } else {
1384         closeRegion(ServerName.valueOf(serverName), pair.getFirst());
1385       }
1386     } else {
1387       Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
1388       if (pair == null) {
1389         throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1390       } else if (pair.getSecond() == null) {
1391         throw new NoServerForRegionException(Bytes.toStringBinary(regionname));
1392       } else {
1393         closeRegion(pair.getSecond(), pair.getFirst());
1394       }
1395     }
1396   }
1397 
1398   /**
1399    * For expert-admins. Runs close on the regionserver. Closes a region based on
1400    * the encoded region name. The region server name is mandatory. If the
1401    * servername is provided then based on the online regions in the specified
1402    * regionserver the specified region will be closed. The master will not be
1403    * informed of the close. Note that the regionname is the encoded regionname.
1404    *
1405    * @param encodedRegionName
1406    *          The encoded region name; i.e. the hash that makes up the region
1407    *          name suffix: e.g. if regionname is
1408    *          <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>
1409    *          , then the encoded region name is:
1410    *          <code>527db22f95c8a9e0116f0cc13c680396</code>.
1411    * @param serverName
1412    *          The servername of the regionserver. A server name is made of host,
1413    *          port and startcode. This is mandatory. Here is an example:
1414    *          <code> host187.example.com,60020,1289493121758</code>
1415    * @return true if the region was closed, false if not.
1416    * @throws IOException
1417    *           if a remote or network exception occurs
1418    */
1419   @Override
1420   public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1421       final String serverName) throws IOException {
1422     if (null == serverName || ("").equals(serverName.trim())) {
1423       throw new IllegalArgumentException(
1424           "The servername cannot be null or empty.");
1425     }
1426     ServerName sn = ServerName.valueOf(serverName);
1427     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1428     // Close the region without updating zk state.
1429     CloseRegionRequest request =
1430       RequestConverter.buildCloseRegionRequest(sn, encodedRegionName, false);
1431     try {
1432       CloseRegionResponse response = admin.closeRegion(null, request);
1433       boolean isRegionClosed = response.getClosed();
1434       if (false == isRegionClosed) {
1435         LOG.error("Not able to close the region " + encodedRegionName + ".");
1436       }
1437       return isRegionClosed;
1438     } catch (ServiceException se) {
1439       throw ProtobufUtil.getRemoteException(se);
1440     }
1441   }
1442 
1443   /**
1444    * Close a region.  For expert-admins  Runs close on the regionserver.  The
1445    * master will not be informed of the close.
1446    * @param sn
1447    * @param hri
1448    * @throws IOException
1449    */
1450   @Override
1451   public void closeRegion(final ServerName sn, final HRegionInfo hri)
1452   throws IOException {
1453     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1454     // Close the region without updating zk state.
1455     ProtobufUtil.closeRegion(admin, sn, hri.getRegionName(), false);
1456   }
1457 
1458   /**
1459    * Get all the online regions on a region server.
1460    */
1461   @Override
1462   public List<HRegionInfo> getOnlineRegions(
1463       final ServerName sn) throws IOException {
1464     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1465     return ProtobufUtil.getOnlineRegions(admin);
1466   }
1467 
1468   /**
1469    * Flush a table or an individual region.
1470    * Synchronous operation.
1471    *
1472    * @param tableNameOrRegionName table or region to flush
1473    * @throws IOException if a remote or network exception occurs
1474    * @throws InterruptedException
1475    */
1476   @Override
1477   public void flush(final String tableNameOrRegionName)
1478   throws IOException, InterruptedException {
1479     flush(Bytes.toBytes(tableNameOrRegionName));
1480   }
1481 
1482   /**
1483    * Flush a table or an individual region.
1484    * Synchronous operation.
1485    *
1486    * @param tableNameOrRegionName table or region to flush
1487    * @throws IOException if a remote or network exception occurs
1488    * @throws InterruptedException
1489    */
1490   @Override
1491   public void flush(final byte[] tableNameOrRegionName)
1492   throws IOException, InterruptedException {
1493     Pair<HRegionInfo, ServerName> regionServerPair
1494       = getRegion(tableNameOrRegionName);
1495     if (regionServerPair != null) {
1496       if (regionServerPair.getSecond() == null) {
1497         throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1498       } else {
1499         flush(regionServerPair.getSecond(), regionServerPair.getFirst());
1500       }
1501     } else {
1502       final TableName tableName = checkTableExists(
1503           TableName.valueOf(tableNameOrRegionName));
1504       if (isTableDisabled(tableName)) {
1505         LOG.info("Table is disabled: " + tableName.getNameAsString());
1506         return;
1507       }
1508       execProcedure("flush-table-proc", tableName.getNameAsString(),
1509         new HashMap<String, String>());
1510     }
1511   }
1512 
1513   private void flush(final ServerName sn, final HRegionInfo hri)
1514   throws IOException {
1515     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1516     FlushRegionRequest request =
1517       RequestConverter.buildFlushRegionRequest(hri.getRegionName());
1518     try {
1519       admin.flushRegion(null, request);
1520     } catch (ServiceException se) {
1521       throw ProtobufUtil.getRemoteException(se);
1522     }
1523   }
1524 
1525   /**
1526    * Compact a table or an individual region.
1527    * Asynchronous operation.
1528    *
1529    * @param tableNameOrRegionName table or region to compact
1530    * @throws IOException if a remote or network exception occurs
1531    * @throws InterruptedException
1532    */
1533   @Override
1534   public void compact(final String tableNameOrRegionName)
1535   throws IOException, InterruptedException {
1536     compact(Bytes.toBytes(tableNameOrRegionName));
1537   }
1538 
1539   /**
1540    * Compact a table or an individual region.
1541    * Asynchronous operation.
1542    *
1543    * @param tableNameOrRegionName table or region to compact
1544    * @throws IOException if a remote or network exception occurs
1545    * @throws InterruptedException
1546    */
1547   @Override
1548   public void compact(final byte[] tableNameOrRegionName)
1549   throws IOException, InterruptedException {
1550     compact(tableNameOrRegionName, null, false);
1551   }
1552 
1553   /**
1554    * Compact a column family within a table or region.
1555    * Asynchronous operation.
1556    *
1557    * @param tableOrRegionName table or region to compact
1558    * @param columnFamily column family within a table or region
1559    * @throws IOException if a remote or network exception occurs
1560    * @throws InterruptedException
1561    */
1562   @Override
1563   public void compact(String tableOrRegionName, String columnFamily)
1564     throws IOException,  InterruptedException {
1565     compact(Bytes.toBytes(tableOrRegionName), Bytes.toBytes(columnFamily));
1566   }
1567 
1568   /**
1569    * Compact a column family within a table or region.
1570    * Asynchronous operation.
1571    *
1572    * @param tableNameOrRegionName table or region to compact
1573    * @param columnFamily column family within a table or region
1574    * @throws IOException if a remote or network exception occurs
1575    * @throws InterruptedException
1576    */
1577   @Override
1578   public void compact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
1579   throws IOException, InterruptedException {
1580     compact(tableNameOrRegionName, columnFamily, false);
1581   }
1582 
1583   /**
1584    * Major compact a table or an individual region.
1585    * Asynchronous operation.
1586    *
1587    * @param tableNameOrRegionName table or region to major compact
1588    * @throws IOException if a remote or network exception occurs
1589    * @throws InterruptedException
1590    */
1591   @Override
1592   public void majorCompact(final String tableNameOrRegionName)
1593   throws IOException, InterruptedException {
1594     majorCompact(Bytes.toBytes(tableNameOrRegionName));
1595   }
1596 
1597   /**
1598    * Major compact a table or an individual region.
1599    * Asynchronous operation.
1600    *
1601    * @param tableNameOrRegionName table or region to major compact
1602    * @throws IOException if a remote or network exception occurs
1603    * @throws InterruptedException
1604    */
1605   @Override
1606   public void majorCompact(final byte[] tableNameOrRegionName)
1607   throws IOException, InterruptedException {
1608     compact(tableNameOrRegionName, null, true);
1609   }
1610 
1611   /**
1612    * Major compact a column family within a table or region.
1613    * Asynchronous operation.
1614    *
1615    * @param tableNameOrRegionName table or region to major compact
1616    * @param columnFamily column family within a table or region
1617    * @throws IOException if a remote or network exception occurs
1618    * @throws InterruptedException
1619    */
1620   @Override
1621   public void majorCompact(final String tableNameOrRegionName,
1622     final String columnFamily) throws IOException, InterruptedException {
1623     majorCompact(Bytes.toBytes(tableNameOrRegionName),
1624       Bytes.toBytes(columnFamily));
1625   }
1626 
1627   /**
1628    * Major compact a column family within a table or region.
1629    * Asynchronous operation.
1630    *
1631    * @param tableNameOrRegionName table or region to major compact
1632    * @param columnFamily column family within a table or region
1633    * @throws IOException if a remote or network exception occurs
1634    * @throws InterruptedException
1635    */
1636   @Override
1637   public void majorCompact(final byte[] tableNameOrRegionName,
1638     final byte[] columnFamily) throws IOException, InterruptedException {
1639     compact(tableNameOrRegionName, columnFamily, true);
1640   }
1641 
1642   /**
1643    * Compact a table or an individual region.
1644    * Asynchronous operation.
1645    *
1646    * @param tableNameOrRegionName table or region to compact
1647    * @param columnFamily column family within a table or region
1648    * @param major True if we are to do a major compaction.
1649    * @throws IOException if a remote or network exception occurs
1650    * @throws InterruptedException
1651    */
1652   private void compact(final byte[] tableNameOrRegionName,
1653     final byte[] columnFamily,final boolean major)
1654   throws IOException, InterruptedException {
1655     ZooKeeperWatcher zookeeper = null;
1656     try {
1657       Pair<HRegionInfo, ServerName> regionServerPair
1658         = getRegion(tableNameOrRegionName);
1659       if (regionServerPair != null) {
1660         if (regionServerPair.getSecond() == null) {
1661           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
1662         } else {
1663           compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
1664         }
1665       } else {
1666         final TableName tableName =
1667             checkTableExists(TableName.valueOf(tableNameOrRegionName));
1668         zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
1669             new ThrowableAbortable());
1670         List<Pair<HRegionInfo, ServerName>> pairs =
1671           MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection,
1672             tableName);
1673         for (Pair<HRegionInfo, ServerName> pair: pairs) {
1674           if (pair.getFirst().isOffline()) continue;
1675           if (pair.getSecond() == null) continue;
1676           try {
1677             compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
1678           } catch (NotServingRegionException e) {
1679             if (LOG.isDebugEnabled()) {
1680               LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
1681                 pair.getFirst() + ": " +
1682                 StringUtils.stringifyException(e));
1683             }
1684           }
1685         }
1686       }
1687     } finally {
1688       if (zookeeper != null) {
1689         zookeeper.close();
1690       }
1691     }
1692   }
1693 
1694   private void compact(final ServerName sn, final HRegionInfo hri,
1695       final boolean major, final byte [] family)
1696   throws IOException {
1697     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1698     CompactRegionRequest request =
1699       RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
1700     try {
1701       admin.compactRegion(null, request);
1702     } catch (ServiceException se) {
1703       throw ProtobufUtil.getRemoteException(se);
1704     }
1705   }
1706 
1707   /**
1708    * Move the region <code>r</code> to <code>dest</code>.
1709    * @param encodedRegionName The encoded region name; i.e. the hash that makes
1710    * up the region name suffix: e.g. if regionname is
1711    * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>,
1712    * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>.
1713    * @param destServerName The servername of the destination regionserver.  If
1714    * passed the empty byte array we'll assign to a random server.  A server name
1715    * is made of host, port and startcode.  Here is an example:
1716    * <code> host187.example.com,60020,1289493121758</code>
1717    * @throws UnknownRegionException Thrown if we can't find a region named
1718    * <code>encodedRegionName</code>
1719    * @throws ZooKeeperConnectionException
1720    * @throws MasterNotRunningException
1721    */
1722   @Override
1723   public void move(final byte [] encodedRegionName, final byte [] destServerName)
1724   throws HBaseIOException, MasterNotRunningException, ZooKeeperConnectionException {
1725     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1726     try {
1727       MoveRegionRequest request =
1728         RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
1729       stub.moveRegion(null,request);
1730     } catch (ServiceException se) {
1731       IOException ioe = ProtobufUtil.getRemoteException(se);
1732       if (ioe instanceof HBaseIOException) {
1733         throw (HBaseIOException)ioe;
1734       }
1735       LOG.error("Unexpected exception: " + se + " from calling HMaster.moveRegion");
1736     } catch (DeserializationException de) {
1737       LOG.error("Could not parse destination server name: " + de);
1738     } finally {
1739       stub.close();
1740     }
1741   }
1742 
1743   /**
1744    * @param regionName
1745    *          Region name to assign.
1746    * @throws MasterNotRunningException
1747    * @throws ZooKeeperConnectionException
1748    * @throws IOException
1749    */
1750   @Override
1751   public void assign(final byte[] regionName) throws MasterNotRunningException,
1752       ZooKeeperConnectionException, IOException {
1753     final byte[] toBeAssigned = getRegionName(regionName);
1754     executeCallable(new MasterCallable<Void>(getConnection()) {
1755       @Override
1756       public Void call(int callTimeout) throws ServiceException {
1757         AssignRegionRequest request =
1758           RequestConverter.buildAssignRegionRequest(toBeAssigned);
1759         master.assignRegion(null,request);
1760         return null;
1761       }
1762     });
1763   }
1764 
1765   /**
1766    * Unassign a region from current hosting regionserver.  Region will then be
1767    * assigned to a regionserver chosen at random.  Region could be reassigned
1768    * back to the same server.  Use {@link #move(byte[], byte[])} if you want
1769    * to control the region movement.
1770    * @param regionName Region to unassign. Will clear any existing RegionPlan
1771    * if one found.
1772    * @param force If true, force unassign (Will remove region from
1773    * regions-in-transition too if present. If results in double assignment
1774    * use hbck -fix to resolve. To be used by experts).
1775    * @throws MasterNotRunningException
1776    * @throws ZooKeeperConnectionException
1777    * @throws IOException
1778    */
1779   @Override
1780   public void unassign(final byte [] regionName, final boolean force)
1781   throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
1782     final byte[] toBeUnassigned = getRegionName(regionName);
1783     executeCallable(new MasterCallable<Void>(getConnection()) {
1784       @Override
1785       public Void call(int callTimeout) throws ServiceException {
1786         UnassignRegionRequest request =
1787           RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
1788         master.unassignRegion(null,request);
1789         return null;
1790       }
1791     });
1792   }
1793 
1794   /**
1795    * Offline specified region from master's in-memory state. It will not attempt to reassign the
1796    * region as in unassign. This API can be used when a region not served by any region server and
1797    * still online as per Master's in memory state. If this API is incorrectly used on active region
1798    * then master will loose track of that region.
1799    *
1800    * This is a special method that should be used by experts or hbck.
1801    *
1802    * @param regionName
1803    *          Region to offline.
1804    * @throws IOException
1805    */
1806   @Override
1807   public void offline(final byte [] regionName)
1808   throws IOException {
1809     MasterKeepAliveConnection master = connection.getKeepAliveMasterService();
1810     try {
1811       master.offlineRegion(null,RequestConverter.buildOfflineRegionRequest(regionName));
1812     } catch (ServiceException se) {
1813       throw ProtobufUtil.getRemoteException(se);
1814     } finally {
1815       master.close();
1816     }
1817   }
1818 
1819   /**
1820    * Turn the load balancer on or off.
1821    * @param on If true, enable balancer. If false, disable balancer.
1822    * @param synchronous If true, it waits until current balance() call, if outstanding, to return.
1823    * @return Previous balancer value
1824    */
1825   @Override
1826   public boolean setBalancerRunning(final boolean on, final boolean synchronous)
1827   throws MasterNotRunningException, ZooKeeperConnectionException {
1828     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1829     try {
1830       SetBalancerRunningRequest req =
1831         RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
1832       return stub.setBalancerRunning(null, req).getPrevBalanceValue();
1833     } catch (ServiceException se) {
1834       IOException ioe = ProtobufUtil.getRemoteException(se);
1835       if (ioe instanceof MasterNotRunningException) {
1836         throw (MasterNotRunningException)ioe;
1837       }
1838       if (ioe instanceof ZooKeeperConnectionException) {
1839         throw (ZooKeeperConnectionException)ioe;
1840       }
1841 
1842       // Throwing MasterNotRunningException even though not really valid in order to not
1843       // break interface by adding additional exception type.
1844       throw new MasterNotRunningException("Unexpected exception when calling balanceSwitch",se);
1845     } finally {
1846       stub.close();
1847     }
1848   }
1849 
1850   /**
1851    * Invoke the balancer.  Will run the balancer and if regions to move, it will
1852    * go ahead and do the reassignments.  Can NOT run for various reasons.  Check
1853    * logs.
1854    * @return True if balancer ran, false otherwise.
1855    */
1856   @Override
1857   public boolean balancer()
1858   throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException {
1859     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1860     try {
1861       return stub.balance(null,RequestConverter.buildBalanceRequest()).getBalancerRan();
1862     } finally {
1863       stub.close();
1864     }
1865   }
1866 
1867   /**
1868    * Enable/Disable the catalog janitor
1869    * @param enable if true enables the catalog janitor
1870    * @return the previous state
1871    * @throws ServiceException
1872    * @throws MasterNotRunningException
1873    */
1874   @Override
1875   public boolean enableCatalogJanitor(boolean enable)
1876       throws ServiceException, MasterNotRunningException {
1877     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1878     try {
1879       return stub.enableCatalogJanitor(null,
1880           RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
1881     } finally {
1882       stub.close();
1883     }
1884   }
1885 
1886   /**
1887    * Ask for a scan of the catalog table
1888    * @return the number of entries cleaned
1889    * @throws ServiceException
1890    * @throws MasterNotRunningException
1891    */
1892   @Override
1893   public int runCatalogScan() throws ServiceException, MasterNotRunningException {
1894     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1895     try {
1896       return stub.runCatalogScan(null,
1897           RequestConverter.buildCatalogScanRequest()).getScanResult();
1898     } finally {
1899       stub.close();
1900     }
1901   }
1902 
1903   /**
1904    * Query on the catalog janitor state (Enabled/Disabled?)
1905    * @throws ServiceException
1906    * @throws org.apache.hadoop.hbase.MasterNotRunningException
1907    */
1908   @Override
1909   public boolean isCatalogJanitorEnabled() throws ServiceException, MasterNotRunningException {
1910     MasterKeepAliveConnection stub = connection.getKeepAliveMasterService();
1911     try {
1912       return stub.isCatalogJanitorEnabled(null,
1913           RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
1914     } finally {
1915       stub.close();
1916     }
1917   }
1918 
1919   /**
1920    * Merge two regions. Asynchronous operation.
1921    * @param encodedNameOfRegionA encoded name of region a
1922    * @param encodedNameOfRegionB encoded name of region b
1923    * @param forcible true if do a compulsory merge, otherwise we will only merge
1924    *          two adjacent regions
1925    * @throws IOException
1926    */
1927   @Override
1928   public void mergeRegions(final byte[] encodedNameOfRegionA,
1929       final byte[] encodedNameOfRegionB, final boolean forcible)
1930       throws IOException {
1931     MasterKeepAliveConnection master = connection
1932         .getKeepAliveMasterService();
1933     try {
1934       DispatchMergingRegionsRequest request = RequestConverter
1935           .buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
1936               encodedNameOfRegionB, forcible);
1937       master.dispatchMergingRegions(null, request);
1938     } catch (ServiceException se) {
1939       IOException ioe = ProtobufUtil.getRemoteException(se);
1940       if (ioe instanceof UnknownRegionException) {
1941         throw (UnknownRegionException) ioe;
1942       }
1943       if (ioe instanceof MergeRegionException) {
1944         throw (MergeRegionException) ioe;
1945       }
1946       LOG.error("Unexpected exception: " + se
1947           + " from calling HMaster.dispatchMergingRegions");
1948     } catch (DeserializationException de) {
1949       LOG.error("Could not parse destination server name: " + de);
1950     } finally {
1951       master.close();
1952     }
1953   }
1954 
1955   /**
1956    * Split a table or an individual region.
1957    * Asynchronous operation.
1958    *
1959    * @param tableNameOrRegionName table or region to split
1960    * @throws IOException if a remote or network exception occurs
1961    * @throws InterruptedException
1962    */
1963   @Override
1964   public void split(final String tableNameOrRegionName)
1965   throws IOException, InterruptedException {
1966     split(Bytes.toBytes(tableNameOrRegionName));
1967   }
1968 
1969   /**
1970    * Split a table or an individual region.  Implicitly finds an optimal split
1971    * point.  Asynchronous operation.
1972    *
1973    * @param tableNameOrRegionName table to region to split
1974    * @throws IOException if a remote or network exception occurs
1975    * @throws InterruptedException
1976    */
1977   @Override
1978   public void split(final byte[] tableNameOrRegionName)
1979   throws IOException, InterruptedException {
1980     split(tableNameOrRegionName, null);
1981   }
1982 
1983   @Override
1984   public void split(final String tableNameOrRegionName,
1985     final String splitPoint) throws IOException, InterruptedException {
1986     split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
1987   }
1988 
1989   /**
1990    * Split a table or an individual region.
1991    * Asynchronous operation.
1992    *
1993    * @param tableNameOrRegionName table to region to split
1994    * @param splitPoint the explicit position to split on
1995    * @throws IOException if a remote or network exception occurs
1996    * @throws InterruptedException interrupt exception occurred
1997    */
1998   @Override
1999   public void split(final byte[] tableNameOrRegionName,
2000       final byte [] splitPoint) throws IOException, InterruptedException {
2001     ZooKeeperWatcher zookeeper = null;
2002     try {
2003       Pair<HRegionInfo, ServerName> regionServerPair
2004         = getRegion(tableNameOrRegionName);
2005       if (regionServerPair != null) {
2006         if (regionServerPair.getSecond() == null) {
2007             throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
2008         } else {
2009           split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
2010         }
2011       } else {
2012         final TableName tableName =
2013             checkTableExists(TableName.valueOf(tableNameOrRegionName));
2014         zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2015             new ThrowableAbortable());
2016         List<Pair<HRegionInfo, ServerName>> pairs =
2017           MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection,
2018             tableName);
2019         for (Pair<HRegionInfo, ServerName> pair: pairs) {
2020           // May not be a server for a particular row
2021           if (pair.getSecond() == null) continue;
2022           HRegionInfo r = pair.getFirst();
2023           // check for parents
2024           if (r.isSplitParent()) continue;
2025           // if a split point given, only split that particular region
2026           if (splitPoint != null && !r.containsRow(splitPoint)) continue;
2027           // call out to region server to do split now
2028           split(pair.getSecond(), pair.getFirst(), splitPoint);
2029         }
2030       }
2031     } finally {
2032       if (zookeeper != null) {
2033         zookeeper.close();
2034       }
2035     }
2036   }
2037 
2038   private void split(final ServerName sn, final HRegionInfo hri,
2039       byte[] splitPoint) throws IOException {
2040     if (hri.getStartKey() != null && splitPoint != null &&
2041          Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
2042        throw new IOException("should not give a splitkey which equals to startkey!");
2043     }
2044     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2045     ProtobufUtil.split(admin, hri, splitPoint);
2046   }
2047 
2048   /**
2049    * Modify an existing table, more IRB friendly version.
2050    * Asynchronous operation.  This means that it may be a while before your
2051    * schema change is updated across all of the table.
2052    *
2053    * @param tableName name of table.
2054    * @param htd modified description of the table
2055    * @throws IOException if a remote or network exception occurs
2056    */
2057   @Override
2058   public void modifyTable(final TableName tableName, final HTableDescriptor htd)
2059   throws IOException {
2060     if (!tableName.equals(htd.getTableName())) {
2061       throw new IllegalArgumentException("the specified table name '" + tableName +
2062         "' doesn't match with the HTD one: " + htd.getTableName());
2063     }
2064 
2065     executeCallable(new MasterCallable<Void>(getConnection()) {
2066       @Override
2067       public Void call(int callTimeout) throws ServiceException {
2068         ModifyTableRequest request = RequestConverter.buildModifyTableRequest(tableName, htd);
2069         master.modifyTable(null, request);
2070         return null;
2071       }
2072     });
2073   }
2074 
2075   public void modifyTable(final byte[] tableName, final HTableDescriptor htd)
2076   throws IOException {
2077     modifyTable(TableName.valueOf(tableName), htd);
2078   }
2079 
2080   public void modifyTable(final String tableName, final HTableDescriptor htd)
2081   throws IOException {
2082     modifyTable(TableName.valueOf(tableName), htd);
2083   }
2084 
2085   /**
2086    * @param tableNameOrRegionName Name of a table or name of a region.
2087    * @return a pair of HRegionInfo and ServerName if <code>tableNameOrRegionName</code> is
2088    *  a verified region name (we call {@link
2089    *  MetaTableAccessor#getRegion(HConnection, byte[])}
2090    *  else null.
2091    * Throw an exception if <code>tableNameOrRegionName</code> is null.
2092    * @throws IOException
2093    */
2094   Pair<HRegionInfo, ServerName> getRegion(final byte[] tableNameOrRegionName) throws IOException {
2095     if (tableNameOrRegionName == null) {
2096       throw new IllegalArgumentException("Pass a table name or region name");
2097     }
2098     Pair<HRegionInfo, ServerName> pair =
2099       MetaTableAccessor.getRegion(connection, tableNameOrRegionName);
2100     if (pair == null) {
2101       final AtomicReference<Pair<HRegionInfo, ServerName>> result =
2102         new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
2103       final String encodedName = Bytes.toString(tableNameOrRegionName);
2104       MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
2105         @Override
2106         public boolean processRow(Result data) throws IOException {
2107           HRegionInfo info = HRegionInfo.getHRegionInfo(data);
2108           if (info == null) {
2109             LOG.warn("No serialized HRegionInfo in " + data);
2110             return true;
2111           }
2112           if (!encodedName.equals(info.getEncodedName())) return true;
2113           ServerName sn = HRegionInfo.getServerName(data);
2114           result.set(new Pair<HRegionInfo, ServerName>(info, sn));
2115           return false; // found the region, stop
2116         }
2117       };
2118 
2119       MetaScanner.metaScan(conf, connection, visitor, null);
2120       pair = result.get();
2121     }
2122     return pair;
2123   }
2124 
2125   /**
2126    * If the input is a region name, it is returned as is. If it's an
2127    * encoded region name, the corresponding region is found from meta
2128    * and its region name is returned. If we can't find any region in
2129    * meta matching the input as either region name or encoded region
2130    * name, the input is returned as is. We don't throw unknown
2131    * region exception.
2132    */
2133   private byte[] getRegionName(
2134       final byte[] regionNameOrEncodedRegionName) throws IOException {
2135     if (Bytes.equals(regionNameOrEncodedRegionName,
2136         HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
2137           || Bytes.equals(regionNameOrEncodedRegionName,
2138             HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
2139       return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
2140     }
2141     byte[] tmp = regionNameOrEncodedRegionName;
2142     Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
2143     if (regionServerPair != null && regionServerPair.getFirst() != null) {
2144       tmp = regionServerPair.getFirst().getRegionName();
2145     }
2146     return tmp;
2147   }
2148 
2149   /**
2150    * Check if table exists or not
2151    * @param tableName Name of a table.
2152    * @return tableName instance
2153    * @throws IOException if a remote or network exception occurs.
2154    * @throws TableNotFoundException if table does not exist.
2155    */
2156   private TableName checkTableExists(final TableName tableName)
2157       throws IOException {
2158     if (!MetaTableAccessor.tableExists(connection, tableName)) {
2159       throw new TableNotFoundException(tableName);
2160     }
2161     return tableName;
2162   }
2163 
2164   /**
2165    * Shuts down the HBase cluster
2166    * @throws IOException if a remote or network exception occurs
2167    */
2168   @Override
2169   public synchronized void shutdown() throws IOException {
2170     executeCallable(new MasterCallable<Void>(getConnection()) {
2171       @Override
2172       public Void call(int callTimeout) throws ServiceException {
2173         master.shutdown(null,ShutdownRequest.newBuilder().build());
2174         return null;
2175       }
2176     });
2177   }
2178 
2179   /**
2180    * Shuts down the current HBase master only.
2181    * Does not shutdown the cluster.
2182    * @see #shutdown()
2183    * @throws IOException if a remote or network exception occurs
2184    */
2185   @Override
2186   public synchronized void stopMaster() throws IOException {
2187     executeCallable(new MasterCallable<Void>(getConnection()) {
2188       @Override
2189       public Void call(int callTimeout) throws ServiceException {
2190         master.stopMaster(null,StopMasterRequest.newBuilder().build());
2191         return null;
2192       }
2193     });
2194   }
2195 
2196   /**
2197    * Stop the designated regionserver
2198    * @param hostnamePort Hostname and port delimited by a <code>:</code> as in
2199    * <code>example.org:1234</code>
2200    * @throws IOException if a remote or network exception occurs
2201    */
2202   @Override
2203   public synchronized void stopRegionServer(final String hostnamePort)
2204   throws IOException {
2205     String hostname = Addressing.parseHostname(hostnamePort);
2206     int port = Addressing.parsePort(hostnamePort);
2207     AdminService.BlockingInterface admin =
2208       this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
2209     StopServerRequest request = RequestConverter.buildStopServerRequest(
2210       "Called by admin client " + this.connection.toString());
2211     try {
2212       admin.stopServer(null, request);
2213     } catch (ServiceException se) {
2214       throw ProtobufUtil.getRemoteException(se);
2215     }
2216   }
2217 
2218 
2219   /**
2220    * @return cluster status
2221    * @throws IOException if a remote or network exception occurs
2222    */
2223   @Override
2224   public ClusterStatus getClusterStatus() throws IOException {
2225     return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
2226       @Override
2227       public ClusterStatus call(int callTimeout) throws ServiceException {
2228         GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
2229         return ClusterStatus.convert(master.getClusterStatus(null,req).getClusterStatus());
2230       }
2231     });
2232   }
2233 
2234   private HRegionLocation getFirstMetaServerForTable(final TableName tableName)
2235   throws IOException {
2236     return connection.locateRegion(TableName.META_TABLE_NAME,
2237       HRegionInfo.createRegionName(tableName, null, HConstants.NINES, false));
2238   }
2239 
2240   /**
2241    * @return Configuration used by the instance.
2242    */
2243   @Override
2244   public Configuration getConfiguration() {
2245     return this.conf;
2246   }
2247 
2248   /**
2249    * Create a new namespace
2250    * @param descriptor descriptor which describes the new namespace
2251    * @throws IOException
2252    */
2253   @Override
2254   public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
2255     executeCallable(new MasterCallable<Void>(getConnection()) {
2256       @Override
2257       public Void call(int callTimeout) throws Exception {
2258         master.createNamespace(null,
2259           CreateNamespaceRequest.newBuilder()
2260                 .setNamespaceDescriptor(ProtobufUtil
2261                     .toProtoNamespaceDescriptor(descriptor)).build());
2262         return null;
2263       }
2264     });
2265   }
2266 
2267   /**
2268    * Modify an existing namespace
2269    * @param descriptor descriptor which describes the new namespace
2270    * @throws IOException
2271    */
2272   @Override
2273   public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
2274     executeCallable(new MasterCallable<Void>(getConnection()) {
2275       @Override
2276       public Void call(int callTimeout) throws Exception {
2277         master.modifyNamespace(null, ModifyNamespaceRequest.newBuilder().
2278           setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
2279         return null;
2280       }
2281     });
2282   }
2283 
2284   /**
2285    * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
2286    * @param name namespace name
2287    * @throws IOException
2288    */
2289   @Override
2290   public void deleteNamespace(final String name) throws IOException {
2291     executeCallable(new MasterCallable<Void>(getConnection()) {
2292       @Override
2293       public Void call(int callTimeout) throws Exception {
2294         master.deleteNamespace(null, DeleteNamespaceRequest.newBuilder().
2295           setNamespaceName(name).build());
2296         return null;
2297       }
2298     });
2299   }
2300 
2301   /**
2302    * Get a namespace descriptor by name
2303    * @param name name of namespace descriptor
2304    * @return A descriptor
2305    * @throws IOException
2306    */
2307   @Override
2308   public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
2309     return
2310         executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
2311           @Override
2312           public NamespaceDescriptor call(int callTimeout) throws Exception {
2313             return ProtobufUtil.toNamespaceDescriptor(
2314               master.getNamespaceDescriptor(null, GetNamespaceDescriptorRequest.newBuilder().
2315                 setNamespaceName(name).build()).getNamespaceDescriptor());
2316           }
2317         });
2318   }
2319 
2320   /**
2321    * List available namespace descriptors
2322    * @return List of descriptors
2323    * @throws IOException
2324    */
2325   @Override
2326   public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
2327     return
2328         executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
2329           @Override
2330           public NamespaceDescriptor[] call(int callTimeout) throws Exception {
2331             List<HBaseProtos.NamespaceDescriptor> list =
2332               master.listNamespaceDescriptors(null, ListNamespaceDescriptorsRequest.newBuilder().
2333                 build()).getNamespaceDescriptorList();
2334             NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
2335             for(int i = 0; i < list.size(); i++) {
2336               res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
2337             }
2338             return res;
2339           }
2340         });
2341   }
2342 
2343   /**
2344    * Get list of table descriptors by namespace
2345    * @param name namespace name
2346    * @return A descriptor
2347    * @throws IOException
2348    */
2349   @Override
2350   public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
2351     return
2352         executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
2353           @Override
2354           public HTableDescriptor[] call(int callTimeout) throws Exception {
2355             List<TableSchema> list =
2356               master.listTableDescriptorsByNamespace(null, ListTableDescriptorsByNamespaceRequest.
2357                 newBuilder().setNamespaceName(name).build()).getTableSchemaList();
2358             HTableDescriptor[] res = new HTableDescriptor[list.size()];
2359             for(int i=0; i < list.size(); i++) {
2360 
2361               res[i] = HTableDescriptor.convert(list.get(i));
2362             }
2363             return res;
2364           }
2365         });
2366   }
2367 
2368   /**
2369    * Get list of table names by namespace
2370    * @param name namespace name
2371    * @return The list of table names in the namespace
2372    * @throws IOException
2373    */
2374   @Override
2375   public TableName[] listTableNamesByNamespace(final String name) throws IOException {
2376     return
2377         executeCallable(new MasterCallable<TableName[]>(getConnection()) {
2378           @Override
2379           public TableName[] call(int callTimeout) throws Exception {
2380             List<HBaseProtos.TableName> tableNames =
2381               master.listTableNamesByNamespace(null, ListTableNamesByNamespaceRequest.
2382                 newBuilder().setNamespaceName(name).build())
2383                 .getTableNameList();
2384             TableName[] result = new TableName[tableNames.size()];
2385             for (int i = 0; i < tableNames.size(); i++) {
2386               result[i] = ProtobufUtil.toTableName(tableNames.get(i));
2387             }
2388             return result;
2389           }
2390         });
2391   }
2392 
2393   /**
2394    * Check to see if HBase is running. Throw an exception if not.
2395    * We consider that HBase is running if ZooKeeper and Master are running.
2396    *
2397    * @param conf system configuration
2398    * @throws MasterNotRunningException if the master is not running
2399    * @throws ZooKeeperConnectionException if unable to connect to zookeeper
2400    */
2401   public static void checkHBaseAvailable(Configuration conf)
2402     throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException, IOException {
2403     Configuration copyOfConf = HBaseConfiguration.create(conf);
2404 
2405     // We set it to make it fail as soon as possible if HBase is not available
2406     copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
2407     copyOfConf.setInt("zookeeper.recovery.retry", 0);
2408 
2409     ConnectionManager.HConnectionImplementation connection
2410       = (ConnectionManager.HConnectionImplementation)
2411       HConnectionManager.getConnection(copyOfConf);
2412 
2413     try {
2414       // Check ZK first.
2415       // If the connection exists, we may have a connection to ZK that does
2416       //  not work anymore
2417       ZooKeeperKeepAliveConnection zkw = null;
2418       try {
2419         zkw = connection.getKeepAliveZooKeeperWatcher();
2420         zkw.getRecoverableZooKeeper().getZooKeeper().exists(
2421           zkw.baseZNode, false);
2422 
2423       } catch (IOException e) {
2424         throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
2425       } catch (InterruptedException e) {
2426         throw (InterruptedIOException)
2427             new InterruptedIOException("Can't connect to ZooKeeper").initCause(e);
2428       } catch (KeeperException e) {
2429         throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
2430       } finally {
2431         if (zkw != null) {
2432           zkw.close();
2433         }
2434       }
2435 
2436       // Check Master
2437       connection.isMasterRunning();
2438 
2439     } finally {
2440       connection.close();
2441     }
2442   }
2443 
2444   /**
2445    * get the regions of a given table.
2446    *
2447    * @param tableName the name of the table
2448    * @return Ordered list of {@link HRegionInfo}.
2449    * @throws IOException
2450    */
2451   @Override
2452   public List<HRegionInfo> getTableRegions(final TableName tableName)
2453   throws IOException {
2454     ZooKeeperWatcher zookeeper =
2455       new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2456         new ThrowableAbortable());
2457     List<HRegionInfo> Regions = null;
2458     try {
2459       Regions = MetaTableAccessor.getTableRegions(zookeeper, connection, tableName, true);
2460     } finally {
2461       zookeeper.close();
2462     }
2463     return Regions;
2464   }
2465 
2466   public List<HRegionInfo> getTableRegions(final byte[] tableName)
2467   throws IOException {
2468     return getTableRegions(TableName.valueOf(tableName));
2469   }
2470 
2471   @Override
2472   public synchronized void close() throws IOException {
2473     if (cleanupConnectionOnClose && this.connection != null && !this.closed) {
2474       this.connection.close();
2475       this.closed = true;
2476     }
2477   }
2478 
2479   /**
2480    * Get tableDescriptors
2481    * @param tableNames List of table names
2482    * @return HTD[] the tableDescriptor
2483    * @throws IOException if a remote or network exception occurs
2484    */
2485   @Override
2486   public HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames)
2487   throws IOException {
2488     return this.connection.getHTableDescriptorsByTableName(tableNames);
2489   }
2490 
2491   /**
2492    * Get tableDescriptors
2493    * @param names List of table names
2494    * @return HTD[] the tableDescriptor
2495    * @throws IOException if a remote or network exception occurs
2496    */
2497   @Override
2498   public HTableDescriptor[] getTableDescriptors(List<String> names)
2499   throws IOException {
2500     List<TableName> tableNames = new ArrayList<TableName>(names.size());
2501     for(String name : names) {
2502       tableNames.add(TableName.valueOf(name));
2503     }
2504     return getTableDescriptorsByTableName(tableNames);
2505   }
2506 
2507   /**
2508    * Roll the log writer. That is, start writing log messages to a new file.
2509    *
2510    * @param serverName
2511    *          The servername of the regionserver. A server name is made of host,
2512    *          port and startcode. This is mandatory. Here is an example:
2513    *          <code> host187.example.com,60020,1289493121758</code>
2514    * @return If lots of logs, flush the returned regions so next time through
2515    * we can clean logs. Returns null if nothing to flush.  Names are actual
2516    * region names as returned by {@link HRegionInfo#getEncodedName()}
2517    * @throws IOException if a remote or network exception occurs
2518    * @throws FailedLogCloseException
2519    */
2520  @Override
2521 public synchronized  byte[][] rollHLogWriter(String serverName)
2522       throws IOException, FailedLogCloseException {
2523     ServerName sn = ServerName.valueOf(serverName);
2524     AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2525     RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
2526     try {
2527       RollWALWriterResponse response = admin.rollWALWriter(null, request);
2528       int regionCount = response.getRegionToFlushCount();
2529       byte[][] regionsToFlush = new byte[regionCount][];
2530       for (int i = 0; i < regionCount; i++) {
2531         ByteString region = response.getRegionToFlush(i);
2532         regionsToFlush[i] = region.toByteArray();
2533       }
2534       return regionsToFlush;
2535     } catch (ServiceException se) {
2536       throw ProtobufUtil.getRemoteException(se);
2537     }
2538   }
2539 
2540   @Override
2541   public String[] getMasterCoprocessors() {
2542     try {
2543       return getClusterStatus().getMasterCoprocessors();
2544     } catch (IOException e) {
2545       LOG.error("Could not getClusterStatus()",e);
2546       return null;
2547     }
2548   }
2549 
2550   /**
2551    * Get the current compaction state of a table or region.
2552    * It could be in a major compaction, a minor compaction, both, or none.
2553    *
2554    * @param tableNameOrRegionName table or region to major compact
2555    * @throws IOException if a remote or network exception occurs
2556    * @throws InterruptedException
2557    * @return the current compaction state
2558    */
2559   @Override
2560   public CompactionState getCompactionState(final String tableNameOrRegionName)
2561       throws IOException, InterruptedException {
2562     return getCompactionState(Bytes.toBytes(tableNameOrRegionName));
2563   }
2564 
2565   /**
2566    * Get the current compaction state of a table or region.
2567    * It could be in a major compaction, a minor compaction, both, or none.
2568    *
2569    * @param tableNameOrRegionName table or region to major compact
2570    * @throws IOException if a remote or network exception occurs
2571    * @throws InterruptedException
2572    * @return the current compaction state
2573    */
2574   @Override
2575   public CompactionState getCompactionState(final byte[] tableNameOrRegionName)
2576       throws IOException, InterruptedException {
2577     CompactionState state = CompactionState.NONE;
2578     ZooKeeperWatcher zookeeper =
2579       new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2580         new ThrowableAbortable());
2581     try {
2582       Pair<HRegionInfo, ServerName> regionServerPair = getRegion(tableNameOrRegionName);
2583       if (regionServerPair != null) {
2584         if (regionServerPair.getSecond() == null) {
2585           throw new NoServerForRegionException(Bytes.toStringBinary(tableNameOrRegionName));
2586         } else {
2587           ServerName sn = regionServerPair.getSecond();
2588           AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2589           GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
2590             regionServerPair.getFirst().getRegionName(), true);
2591           GetRegionInfoResponse response = admin.getRegionInfo(null, request);
2592           return response.getCompactionState();
2593         }
2594       } else {
2595         final TableName tableName =
2596             checkTableExists(TableName.valueOf(tableNameOrRegionName));
2597         List<Pair<HRegionInfo, ServerName>> pairs =
2598           MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection, tableName);
2599         for (Pair<HRegionInfo, ServerName> pair: pairs) {
2600           if (pair.getFirst().isOffline()) continue;
2601           if (pair.getSecond() == null) continue;
2602           try {
2603             ServerName sn = pair.getSecond();
2604             AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2605             GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
2606               pair.getFirst().getRegionName(), true);
2607             GetRegionInfoResponse response = admin.getRegionInfo(null, request);
2608             switch (response.getCompactionState()) {
2609             case MAJOR_AND_MINOR:
2610               return CompactionState.MAJOR_AND_MINOR;
2611             case MAJOR:
2612               if (state == CompactionState.MINOR) {
2613                 return CompactionState.MAJOR_AND_MINOR;
2614               }
2615               state = CompactionState.MAJOR;
2616               break;
2617             case MINOR:
2618               if (state == CompactionState.MAJOR) {
2619                 return CompactionState.MAJOR_AND_MINOR;
2620               }
2621               state = CompactionState.MINOR;
2622               break;
2623             case NONE:
2624               default: // nothing, continue
2625             }
2626           } catch (NotServingRegionException e) {
2627             if (LOG.isDebugEnabled()) {
2628               LOG.debug("Trying to get compaction state of " +
2629                 pair.getFirst() + ": " +
2630                 StringUtils.stringifyException(e));
2631             }
2632           } catch (RemoteException e) {
2633             if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
2634               if (LOG.isDebugEnabled()) {
2635                 LOG.debug("Trying to get compaction state of " + pair.getFirst() + ": "
2636                     + StringUtils.stringifyException(e));
2637               }
2638             } else {
2639               throw e;
2640             }
2641           }
2642         }
2643       }
2644     } catch (ServiceException se) {
2645       throw ProtobufUtil.getRemoteException(se);
2646     } finally {
2647       zookeeper.close();
2648     }
2649     return state;
2650   }
2651 
2652   /**
2653    * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
2654    * taken. If the table is disabled, an offline snapshot is taken.
2655    * <p>
2656    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2657    * snapshot with the same name (even a different type or with different parameters) will fail with
2658    * a {@link SnapshotCreationException} indicating the duplicate naming.
2659    * <p>
2660    * Snapshot names follow the same naming constraints as tables in HBase. See
2661    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
2662    * @param snapshotName name of the snapshot to be created
2663    * @param tableName name of the table for which snapshot is created
2664    * @throws IOException if a remote or network exception occurs
2665    * @throws SnapshotCreationException if snapshot creation failed
2666    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2667    */
2668   @Override
2669   public void snapshot(final String snapshotName,
2670                        final TableName tableName) throws IOException,
2671       SnapshotCreationException, IllegalArgumentException {
2672     snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
2673   }
2674 
2675   public void snapshot(final String snapshotName,
2676                        final String tableName) throws IOException,
2677       SnapshotCreationException, IllegalArgumentException {
2678     snapshot(snapshotName, TableName.valueOf(tableName),
2679         SnapshotDescription.Type.FLUSH);
2680   }
2681 
2682   /**
2683    * Create snapshot for the given table of given flush type.
2684    * <p>
2685    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2686    * snapshot with the same name (even a different type or with different parameters) will fail with
2687    * a {@link SnapshotCreationException} indicating the duplicate naming.
2688    * <p>
2689    * Snapshot names follow the same naming constraints as tables in HBase.
2690    * @param snapshotName name of the snapshot to be created
2691    * @param tableName name of the table for which snapshot is created
2692    * @param flushType if the snapshot should be taken without flush memstore first
2693    * @throws IOException if a remote or network exception occurs
2694    * @throws SnapshotCreationException if snapshot creation failed
2695    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2696    */
2697    public void snapshot(final byte[] snapshotName, final byte[] tableName,
2698                        final SnapshotDescription.Type flushType) throws
2699       IOException, SnapshotCreationException, IllegalArgumentException {
2700       snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName), flushType);
2701   }
2702   /**
2703    public void snapshot(final String snapshotName,
2704     * Create a timestamp consistent snapshot for the given table.
2705                         final byte[] tableName) throws IOException,
2706     * <p>
2707     * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2708     * snapshot with the same name (even a different type or with different parameters) will fail with
2709     * a {@link SnapshotCreationException} indicating the duplicate naming.
2710     * <p>
2711     * Snapshot names follow the same naming constraints as tables in HBase.
2712     * @param snapshotName name of the snapshot to be created
2713     * @param tableName name of the table for which snapshot is created
2714     * @throws IOException if a remote or network exception occurs
2715     * @throws SnapshotCreationException if snapshot creation failed
2716     * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2717     */
2718   @Override
2719   public void snapshot(final byte[] snapshotName,
2720                        final TableName tableName) throws IOException,
2721       SnapshotCreationException, IllegalArgumentException {
2722     snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH);
2723   }
2724 
2725   public void snapshot(final byte[] snapshotName,
2726                        final byte[] tableName) throws IOException,
2727       SnapshotCreationException, IllegalArgumentException {
2728     snapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName),
2729         SnapshotDescription.Type.FLUSH);
2730   }
2731 
2732   /**
2733    * Create typed snapshot of the table.
2734    * <p>
2735    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2736    * snapshot with the same name (even a different type or with different parameters) will fail with
2737    * a {@link SnapshotCreationException} indicating the duplicate naming.
2738    * <p>
2739    * Snapshot names follow the same naming constraints as tables in HBase. See
2740    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
2741    * <p>
2742    * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
2743    *          snapshots stored on the cluster
2744    * @param tableName name of the table to snapshot
2745    * @param type type of snapshot to take
2746    * @throws IOException we fail to reach the master
2747    * @throws SnapshotCreationException if snapshot creation failed
2748    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2749    */
2750   @Override
2751   public void snapshot(final String snapshotName,
2752                        final TableName tableName,
2753                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2754       IllegalArgumentException {
2755     SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
2756     builder.setTable(tableName.getNameAsString());
2757     builder.setName(snapshotName);
2758     builder.setType(type);
2759     snapshot(builder.build());
2760   }
2761 
2762   public void snapshot(final String snapshotName,
2763                        final String tableName,
2764                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2765       IllegalArgumentException {
2766     snapshot(snapshotName, TableName.valueOf(tableName), type);
2767   }
2768 
2769   public void snapshot(final String snapshotName,
2770                        final byte[] tableName,
2771                       SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
2772       IllegalArgumentException {
2773     snapshot(snapshotName, TableName.valueOf(tableName), type);
2774   }
2775 
2776   /**
2777    * Take a snapshot and wait for the server to complete that snapshot (blocking).
2778    * <p>
2779    * Only a single snapshot should be taken at a time for an instance of HBase, or results may be
2780    * undefined (you can tell multiple HBase clusters to snapshot at the same time, but only one at a
2781    * time for a single cluster).
2782    * <p>
2783    * Snapshots are considered unique based on <b>the name of the snapshot</b>. Attempts to take a
2784    * snapshot with the same name (even a different type or with different parameters) will fail with
2785    * a {@link SnapshotCreationException} indicating the duplicate naming.
2786    * <p>
2787    * Snapshot names follow the same naming constraints as tables in HBase. See
2788    * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
2789    * <p>
2790    * You should probably use {@link #snapshot(String, String)} or {@link #snapshot(byte[], byte[])}
2791    * unless you are sure about the type of snapshot that you want to take.
2792    * @param snapshot snapshot to take
2793    * @throws IOException or we lose contact with the master.
2794    * @throws SnapshotCreationException if snapshot failed to be taken
2795    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2796    */
2797   @Override
2798   public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
2799       IllegalArgumentException {
2800     // actually take the snapshot
2801     SnapshotResponse response = takeSnapshotAsync(snapshot);
2802     final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
2803         .build();
2804     IsSnapshotDoneResponse done = null;
2805     long start = EnvironmentEdgeManager.currentTimeMillis();
2806     long max = response.getExpectedTimeout();
2807     long maxPauseTime = max / this.numRetries;
2808     int tries = 0;
2809     LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
2810         ClientSnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " +
2811         maxPauseTime + " ms per retry)");
2812     while (tries == 0
2813         || ((EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done.getDone())) {
2814       try {
2815         // sleep a backoff <= pauseTime amount
2816         long sleep = getPauseTime(tries++);
2817         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
2818         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
2819           "ms while waiting for snapshot completion.");
2820         Thread.sleep(sleep);
2821       } catch (InterruptedException e) {
2822         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
2823       }
2824       LOG.debug("Getting current status of snapshot from master...");
2825       done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
2826         @Override
2827         public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
2828           return master.isSnapshotDone(null, request);
2829         }
2830       });
2831     }
2832     if (!done.getDone()) {
2833       throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
2834           + "' wasn't completed in expectedTime:" + max + " ms", snapshot);
2835     }
2836   }
2837 
2838   /**
2839    * Take a snapshot without waiting for the server to complete that snapshot (asynchronous)
2840    * <p>
2841    * Only a single snapshot should be taken at a time, or results may be undefined.
2842    * @param snapshot snapshot to take
2843    * @return response from the server indicating the max time to wait for the snapshot
2844    * @throws IOException if the snapshot did not succeed or we lose contact with the master.
2845    * @throws SnapshotCreationException if snapshot creation failed
2846    * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
2847    */
2848   @Override
2849   public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
2850       SnapshotCreationException {
2851     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
2852     final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
2853         .build();
2854     // run the snapshot on the master
2855     return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
2856       @Override
2857       public SnapshotResponse call(int callTimeout) throws ServiceException {
2858         return master.snapshot(null, request);
2859       }
2860     });
2861   }
2862 
2863   /**
2864    * Check the current state of the passed snapshot.
2865    * <p>
2866    * There are three possible states:
2867    * <ol>
2868    * <li>running - returns <tt>false</tt></li>
2869    * <li>finished - returns <tt>true</tt></li>
2870    * <li>finished with error - throws the exception that caused the snapshot to fail</li>
2871    * </ol>
2872    * <p>
2873    * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
2874    * run/started since the snapshot your are checking, you will recieve an
2875    * {@link UnknownSnapshotException}.
2876    * @param snapshot description of the snapshot to check
2877    * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
2878    *         running
2879    * @throws IOException if we have a network issue
2880    * @throws HBaseSnapshotException if the snapshot failed
2881    * @throws UnknownSnapshotException if the requested snapshot is unknown
2882    */
2883   @Override
2884   public boolean isSnapshotFinished(final SnapshotDescription snapshot)
2885       throws IOException, HBaseSnapshotException, UnknownSnapshotException {
2886 
2887     return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
2888       @Override
2889       public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
2890         return master.isSnapshotDone(null,
2891           IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
2892       }
2893     }).getDone();
2894   }
2895 
2896   /**
2897    * Restore the specified snapshot on the original table. (The table must be disabled)
2898    * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
2899    * is set to true, a snapshot of the current table is taken
2900    * before executing the restore operation.
2901    * In case of restore failure, the failsafe snapshot will be restored.
2902    * If the restore completes without problem the failsafe snapshot is deleted.
2903    *
2904    * @param snapshotName name of the snapshot to restore
2905    * @throws IOException if a remote or network exception occurs
2906    * @throws RestoreSnapshotException if snapshot failed to be restored
2907    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2908    */
2909   @Override
2910   public void restoreSnapshot(final byte[] snapshotName)
2911       throws IOException, RestoreSnapshotException {
2912     restoreSnapshot(Bytes.toString(snapshotName));
2913   }
2914 
2915   /**
2916    * Restore the specified snapshot on the original table. (The table must be disabled)
2917    * If the "hbase.snapshot.restore.take.failsafe.snapshot" configuration property
2918    * is set to true, a snapshot of the current table is taken
2919    * before executing the restore operation.
2920    * In case of restore failure, the failsafe snapshot will be restored.
2921    * If the restore completes without problem the failsafe snapshot is deleted.
2922    *
2923    * @param snapshotName name of the snapshot to restore
2924    * @throws IOException if a remote or network exception occurs
2925    * @throws RestoreSnapshotException if snapshot failed to be restored
2926    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2927    */
2928   @Override
2929   public void restoreSnapshot(final String snapshotName)
2930       throws IOException, RestoreSnapshotException {
2931     boolean takeFailSafeSnapshot =
2932       conf.getBoolean("hbase.snapshot.restore.take.failsafe.snapshot", false);
2933     restoreSnapshot(snapshotName, takeFailSafeSnapshot);
2934   }
2935 
2936   /**
2937    * Restore the specified snapshot on the original table. (The table must be disabled)
2938    * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
2939    * before executing the restore operation.
2940    * In case of restore failure, the failsafe snapshot will be restored.
2941    * If the restore completes without problem the failsafe snapshot is deleted.
2942    *
2943    * The failsafe snapshot name is configurable by using the property
2944    * "hbase.snapshot.restore.failsafe.name".
2945    *
2946    * @param snapshotName name of the snapshot to restore
2947    * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
2948    * @throws IOException if a remote or network exception occurs
2949    * @throws RestoreSnapshotException if snapshot failed to be restored
2950    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2951    */
2952   @Override
2953   public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
2954       throws IOException, RestoreSnapshotException {
2955     restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
2956   }
2957 
2958   /**
2959    * Restore the specified snapshot on the original table. (The table must be disabled)
2960    * If 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken
2961    * before executing the restore operation.
2962    * In case of restore failure, the failsafe snapshot will be restored.
2963    * If the restore completes without problem the failsafe snapshot is deleted.
2964    *
2965    * The failsafe snapshot name is configurable by using the property
2966    * "hbase.snapshot.restore.failsafe.name".
2967    *
2968    * @param snapshotName name of the snapshot to restore
2969    * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
2970    * @throws IOException if a remote or network exception occurs
2971    * @throws RestoreSnapshotException if snapshot failed to be restored
2972    * @throws IllegalArgumentException if the restore request is formatted incorrectly
2973    */
2974   @Override
2975   public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot)
2976       throws IOException, RestoreSnapshotException {
2977     TableName tableName = null;
2978     for (SnapshotDescription snapshotInfo: listSnapshots()) {
2979       if (snapshotInfo.getName().equals(snapshotName)) {
2980         tableName = TableName.valueOf(snapshotInfo.getTable());
2981         break;
2982       }
2983     }
2984 
2985     if (tableName == null) {
2986       throw new RestoreSnapshotException(
2987         "Unable to find the table name for snapshot=" + snapshotName);
2988     }
2989 
2990     // The table does not exists, switch to clone.
2991     if (!tableExists(tableName)) {
2992       try {
2993         cloneSnapshot(snapshotName, tableName);
2994       } catch (InterruptedException e) {
2995         throw new InterruptedIOException("Interrupted when restoring a nonexistent table: " +
2996           e.getMessage());
2997       }
2998       return;
2999     }
3000 
3001     // Check if the table is disabled
3002     if (!isTableDisabled(tableName)) {
3003       throw new TableNotDisabledException(tableName);
3004     }
3005 
3006     // Take a snapshot of the current state
3007     String failSafeSnapshotSnapshotName = null;
3008     if (takeFailSafeSnapshot) {
3009       failSafeSnapshotSnapshotName = conf.get("hbase.snapshot.restore.failsafe.name",
3010         "hbase-failsafe-{snapshot.name}-{restore.timestamp}");
3011       failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotName
3012         .replace("{snapshot.name}", snapshotName)
3013         .replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.'))
3014         .replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTimeMillis()));
3015       LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
3016       snapshot(failSafeSnapshotSnapshotName, tableName);
3017     }
3018 
3019     try {
3020       // Restore snapshot
3021       internalRestoreSnapshot(snapshotName, tableName);
3022     } catch (IOException e) {
3023       // Somthing went wrong during the restore...
3024       // if the pre-restore snapshot is available try to rollback
3025       if (takeFailSafeSnapshot) {
3026         try {
3027           internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName);
3028           String msg = "Restore snapshot=" + snapshotName +
3029             " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + " succeeded.";
3030           LOG.error(msg, e);
3031           throw new RestoreSnapshotException(msg, e);
3032         } catch (IOException ex) {
3033           String msg = "Failed to restore and rollback to snapshot=" + failSafeSnapshotSnapshotName;
3034           LOG.error(msg, ex);
3035           throw new RestoreSnapshotException(msg, e);
3036         }
3037       } else {
3038         throw new RestoreSnapshotException("Failed to restore snapshot=" + snapshotName, e);
3039       }
3040     }
3041 
3042     // If the restore is succeeded, delete the pre-restore snapshot
3043     if (takeFailSafeSnapshot) {
3044       try {
3045         LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
3046         deleteSnapshot(failSafeSnapshotSnapshotName);
3047       } catch (IOException e) {
3048         LOG.error("Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, e);
3049       }
3050     }
3051   }
3052 
3053   /**
3054    * Create a new table by cloning the snapshot content.
3055    *
3056    * @param snapshotName name of the snapshot to be cloned
3057    * @param tableName name of the table where the snapshot will be restored
3058    * @throws IOException if a remote or network exception occurs
3059    * @throws TableExistsException if table to be created already exists
3060    * @throws RestoreSnapshotException if snapshot failed to be cloned
3061    * @throws IllegalArgumentException if the specified table has not a valid name
3062    */
3063   public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
3064       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
3065     cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName));
3066   }
3067 
3068   /**
3069    * Create a new table by cloning the snapshot content.
3070    *
3071    * @param snapshotName name of the snapshot to be cloned
3072    * @param tableName name of the table where the snapshot will be restored
3073    * @throws IOException if a remote or network exception occurs
3074    * @throws TableExistsException if table to be created already exists
3075    * @throws RestoreSnapshotException if snapshot failed to be cloned
3076    * @throws IllegalArgumentException if the specified table has not a valid name
3077    */
3078   @Override
3079   public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
3080       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
3081     cloneSnapshot(Bytes.toString(snapshotName), tableName);
3082   }
3083 
3084 
3085 
3086   /**
3087    * Create a new table by cloning the snapshot content.
3088    *
3089    * @param snapshotName name of the snapshot to be cloned
3090    * @param tableName name of the table where the snapshot will be restored
3091    * @throws IOException if a remote or network exception occurs
3092    * @throws TableExistsException if table to be created already exists
3093    * @throws RestoreSnapshotException if snapshot failed to be cloned
3094    * @throws IllegalArgumentException if the specified table has not a valid name
3095    */
3096   public void cloneSnapshot(final String snapshotName, final String tableName)
3097       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
3098     cloneSnapshot(snapshotName, TableName.valueOf(tableName));
3099   }
3100 
3101   /**
3102    * Create a new table by cloning the snapshot content.
3103    *
3104    * @param snapshotName name of the snapshot to be cloned
3105    * @param tableName name of the table where the snapshot will be restored
3106    * @throws IOException if a remote or network exception occurs
3107    * @throws TableExistsException if table to be created already exists
3108    * @throws RestoreSnapshotException if snapshot failed to be cloned
3109    * @throws IllegalArgumentException if the specified table has not a valid name
3110    */
3111   @Override
3112   public void cloneSnapshot(final String snapshotName, final TableName tableName)
3113       throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
3114     if (tableExists(tableName)) {
3115       throw new TableExistsException(tableName);
3116     }
3117     internalRestoreSnapshot(snapshotName, tableName);
3118     waitUntilTableIsEnabled(tableName);
3119   }
3120 
3121   /**
3122    * Execute a distributed procedure on a cluster synchronously with return data
3123    *
3124    * @param signature A distributed procedure is uniquely identified
3125    * by its signature (default the root ZK node name of the procedure).
3126    * @param instance The instance name of the procedure. For some procedures, this parameter is
3127    * optional.
3128    * @param props Property/Value pairs of properties passing to the procedure
3129    * @return data returned after procedure execution. null if no return data.
3130    * @throws IOException
3131    */
3132   @Override
3133   public byte[] execProcedureWithRet(String signature, String instance,
3134       Map<String, String> props) throws IOException {
3135     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
3136     builder.setSignature(signature).setInstance(instance);
3137     for (Entry<String, String> entry : props.entrySet()) {
3138       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
3139           .setValue(entry.getValue()).build();
3140       builder.addConfiguration(pair);
3141     }
3142 
3143     final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
3144         .setProcedure(builder.build()).build();
3145     // run the procedure on the master
3146     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
3147         getConnection()) {
3148       @Override
3149       public ExecProcedureResponse call(int callTimeout) throws ServiceException {
3150         return master.execProcedureWithRet(null, request);
3151       }
3152     });
3153 
3154     return response.hasReturnData() ? response.getReturnData().toByteArray() : null;
3155   }
3156   /**
3157    * Execute a distributed procedure on a cluster.
3158    *
3159    * @param signature A distributed procedure is uniquely identified
3160    * by its signature (default the root ZK node name of the procedure).
3161    * @param instance The instance name of the procedure. For some procedures, this parameter is
3162    * optional.
3163    * @param props Property/Value pairs of properties passing to the procedure
3164    * @throws IOException
3165    */
3166   @Override
3167   public void execProcedure(String signature, String instance,
3168       Map<String, String> props) throws IOException {
3169     ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
3170     builder.setSignature(signature).setInstance(instance);
3171     for (Entry<String, String> entry : props.entrySet()) {
3172       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
3173           .setValue(entry.getValue()).build();
3174       builder.addConfiguration(pair);
3175     }
3176 
3177     final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
3178         .setProcedure(builder.build()).build();
3179     // run the procedure on the master
3180     ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
3181         getConnection()) {
3182       @Override
3183       public ExecProcedureResponse call(int callTimeout) throws ServiceException {
3184         return master.execProcedure(null, request);
3185       }
3186     });
3187 
3188     long start = EnvironmentEdgeManager.currentTimeMillis();
3189     long max = response.getExpectedTimeout();
3190     long maxPauseTime = max / this.numRetries;
3191     int tries = 0;
3192     LOG.debug("Waiting a max of " + max + " ms for procedure '" +
3193         signature + " : " + instance + "'' to complete. (max " + maxPauseTime + " ms per retry)");
3194     boolean done = false;
3195     while (tries == 0
3196         || ((EnvironmentEdgeManager.currentTimeMillis() - start) < max && !done)) {
3197       try {
3198         // sleep a backoff <= pauseTime amount
3199         long sleep = getPauseTime(tries++);
3200         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
3201         LOG.debug("(#" + tries + ") Sleeping: " + sleep +
3202           "ms while waiting for procedure completion.");
3203         Thread.sleep(sleep);
3204       } catch (InterruptedException e) {
3205         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
3206       }
3207       LOG.debug("Getting current status of procedure from master...");
3208       done = isProcedureFinished(signature, instance, props);
3209     }
3210     if (!done) {
3211       throw new IOException("Procedure '" + signature + " : " + instance
3212           + "' wasn't completed in expectedTime:" + max + " ms");
3213     }
3214   }
3215 
3216   /**
3217    * Check the current state of the specified procedure.
3218    * <p>
3219    * There are three possible states:
3220    * <ol>
3221    * <li>running - returns <tt>false</tt></li>
3222    * <li>finished - returns <tt>true</tt></li>
3223    * <li>finished with error - throws the exception that caused the procedure to fail</li>
3224    * </ol>
3225    * <p>
3226    *
3227    * @param signature The signature that uniquely identifies a procedure
3228    * @param instance The instance name of the procedure
3229    * @param props Property/Value pairs of properties passing to the procedure
3230    * @return true if the specified procedure is finished successfully, false if it is still running
3231    * @throws IOException if the specified procedure finished with error
3232    */
3233   @Override
3234   public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
3235       throws IOException {
3236     final ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
3237     builder.setSignature(signature).setInstance(instance);
3238     for (Entry<String, String> entry : props.entrySet()) {
3239       NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
3240           .setValue(entry.getValue()).build();
3241       builder.addConfiguration(pair);
3242     }
3243     final ProcedureDescription desc = builder.build();
3244     return executeCallable(
3245         new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
3246           @Override
3247           public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
3248             return master.isProcedureDone(null, IsProcedureDoneRequest
3249                 .newBuilder().setProcedure(desc).build());
3250           }
3251         }).getDone();
3252   }
3253 
3254   /**
3255    * Execute Restore/Clone snapshot and wait for the server to complete (blocking).
3256    * To check if the cloned table exists, use {@link #isTableAvailable} -- it is not safe to
3257    * create an HTable instance to this table before it is available.
3258    * @param snapshotName snapshot to restore
3259    * @param tableName table name to restore the snapshot on
3260    * @throws IOException if a remote or network exception occurs
3261    * @throws RestoreSnapshotException if snapshot failed to be restored
3262    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3263    */
3264   private void internalRestoreSnapshot(final String snapshotName, final TableName
3265       tableName)
3266       throws IOException, RestoreSnapshotException {
3267     SnapshotDescription snapshot = SnapshotDescription.newBuilder()
3268         .setName(snapshotName).setTable(tableName.getNameAsString()).build();
3269 
3270     // actually restore the snapshot
3271     internalRestoreSnapshotAsync(snapshot);
3272 
3273     final IsRestoreSnapshotDoneRequest request = IsRestoreSnapshotDoneRequest.newBuilder()
3274         .setSnapshot(snapshot).build();
3275     IsRestoreSnapshotDoneResponse done = IsRestoreSnapshotDoneResponse.newBuilder()
3276         .setDone(false).buildPartial();
3277     final long maxPauseTime = 5000;
3278     int tries = 0;
3279     while (!done.getDone()) {
3280       try {
3281         // sleep a backoff <= pauseTime amount
3282         long sleep = getPauseTime(tries++);
3283         sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
3284         LOG.debug(tries + ") Sleeping: " + sleep + " ms while we wait for snapshot restore to complete.");
3285         Thread.sleep(sleep);
3286       } catch (InterruptedException e) {
3287         throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
3288       }
3289       LOG.debug("Getting current status of snapshot restore from master...");
3290       done = executeCallable(new MasterCallable<IsRestoreSnapshotDoneResponse>(
3291           getConnection()) {
3292         @Override
3293         public IsRestoreSnapshotDoneResponse call(int callTimeout) throws ServiceException {
3294           return master.isRestoreSnapshotDone(null, request);
3295         }
3296       });
3297     }
3298     if (!done.getDone()) {
3299       throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + "' wasn't restored.");
3300     }
3301   }
3302 
3303   /**
3304    * Execute Restore/Clone snapshot and wait for the server to complete (asynchronous)
3305    * <p>
3306    * Only a single snapshot should be restored at a time, or results may be undefined.
3307    * @param snapshot snapshot to restore
3308    * @return response from the server indicating the max time to wait for the snapshot
3309    * @throws IOException if a remote or network exception occurs
3310    * @throws RestoreSnapshotException if snapshot failed to be restored
3311    * @throws IllegalArgumentException if the restore request is formatted incorrectly
3312    */
3313   private RestoreSnapshotResponse internalRestoreSnapshotAsync(final SnapshotDescription snapshot)
3314       throws IOException, RestoreSnapshotException {
3315     ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
3316 
3317     final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
3318         .build();
3319 
3320     // run the snapshot restore on the master
3321     return executeCallable(new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
3322       @Override
3323       public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
3324         return master.restoreSnapshot(null, request);
3325       }
3326     });
3327   }
3328 
3329   /**
3330    * List completed snapshots.
3331    * @return a list of snapshot descriptors for completed snapshots
3332    * @throws IOException if a network error occurs
3333    */
3334   @Override
3335   public List<SnapshotDescription> listSnapshots() throws IOException {
3336     return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
3337       @Override
3338       public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
3339         return master.getCompletedSnapshots(null, GetCompletedSnapshotsRequest.newBuilder().build())
3340             .getSnapshotsList();
3341       }
3342     });
3343   }
3344 
3345   /**
3346    * List all the completed snapshots matching the given regular expression.
3347    *
3348    * @param regex The regular expression to match against
3349    * @return - returns a List of SnapshotDescription
3350    * @throws IOException if a remote or network exception occurs
3351    */
3352   @Override
3353   public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
3354     return listSnapshots(Pattern.compile(regex));
3355   }
3356 
3357   /**
3358    * List all the completed snapshots matching the given pattern.
3359    *
3360    * @param pattern The compiled regular expression to match against
3361    * @return - returns a List of SnapshotDescription
3362    * @throws IOException if a remote or network exception occurs
3363    */
3364   @Override
3365   public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
3366     List<SnapshotDescription> matched = new LinkedList<SnapshotDescription>();
3367     List<SnapshotDescription> snapshots = listSnapshots();
3368     for (SnapshotDescription snapshot : snapshots) {
3369       if (pattern.matcher(snapshot.getName()).matches()) {
3370         matched.add(snapshot);
3371       }
3372     }
3373     return matched;
3374   }
3375 
3376   /**
3377    * Delete an existing snapshot.
3378    * @param snapshotName name of the snapshot
3379    * @throws IOException if a remote or network exception occurs
3380    */
3381   @Override
3382   public void deleteSnapshot(final byte[] snapshotName) throws IOException {
3383     deleteSnapshot(Bytes.toString(snapshotName));
3384   }
3385 
3386   /**
3387    * Delete an existing snapshot.
3388    * @param snapshotName name of the snapshot
3389    * @throws IOException if a remote or network exception occurs
3390    */
3391   @Override
3392   public void deleteSnapshot(final String snapshotName) throws IOException {
3393     // make sure the snapshot is possibly valid
3394     TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
3395     // do the delete
3396     executeCallable(new MasterCallable<Void>(getConnection()) {
3397       @Override
3398       public Void call(int callTimeout) throws ServiceException {
3399         master.deleteSnapshot(null,
3400           DeleteSnapshotRequest.newBuilder().
3401             setSnapshot(SnapshotDescription.newBuilder().setName(snapshotName).build()).build());
3402         return null;
3403       }
3404     });
3405   }
3406 
3407   /**
3408    * Delete existing snapshots whose names match the pattern passed.
3409    * @param regex The regular expression to match against
3410    * @throws IOException if a remote or network exception occurs
3411    */
3412   @Override
3413   public void deleteSnapshots(final String regex) throws IOException {
3414     deleteSnapshots(Pattern.compile(regex));
3415   }
3416 
3417   /**
3418    * Delete existing snapshots whose names match the pattern passed.
3419    * @param pattern pattern for names of the snapshot to match
3420    * @throws IOException if a remote or network exception occurs
3421    */
3422   @Override
3423   public void deleteSnapshots(final Pattern pattern) throws IOException {
3424     List<SnapshotDescription> snapshots = listSnapshots(pattern);
3425     for (final SnapshotDescription snapshot : snapshots) {
3426       // do the delete
3427       executeCallable(new MasterCallable<Void>(getConnection()) {
3428         @Override
3429         public Void call(int callTimeout) throws ServiceException {
3430           this.master.deleteSnapshot(null,
3431             DeleteSnapshotRequest.newBuilder().setSnapshot(snapshot).build());
3432           return null;
3433         }
3434       });
3435     }
3436   }
3437 
3438   /**
3439    * Parent of {@link MasterCallable} and {@link MasterCallable}.
3440    * Has common methods.
3441    * @param <V>
3442    */
3443   abstract static class MasterCallable<V> implements RetryingCallable<V>, Closeable {
3444     protected HConnection connection;
3445     protected MasterKeepAliveConnection master;
3446 
3447     public MasterCallable(final HConnection connection) {
3448       this.connection = connection;
3449     }
3450 
3451     @Override
3452     public void prepare(boolean reload) throws IOException {
3453       this.master = this.connection.getKeepAliveMasterService();
3454     }
3455 
3456     @Override
3457     public void close() throws IOException {
3458       // The above prepare could fail but this would still be called though masterAdmin is null
3459       if (this.master != null) this.master.close();
3460     }
3461 
3462     @Override
3463     public void throwable(Throwable t, boolean retrying) {
3464     }
3465 
3466     @Override
3467     public String getExceptionMessageAdditionalDetail() {
3468       return "";
3469     }
3470 
3471     @Override
3472     public long sleep(long pause, int tries) {
3473       return ConnectionUtils.getPauseTime(pause, tries);
3474     }
3475   }
3476 
3477   private <V> V executeCallable(MasterCallable<V> callable) throws IOException {
3478     RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
3479     try {
3480       return caller.callWithRetries(callable, operationTimeout);
3481     } finally {
3482       callable.close();
3483     }
3484   }
3485 
3486   /**
3487    * Creates and returns a {@link com.google.protobuf.RpcChannel} instance
3488    * connected to the active master.
3489    *
3490    * <p>
3491    * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published
3492    * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations:
3493    * </p>
3494    *
3495    * <div style="background-color: #cccccc; padding: 2px">
3496    * <blockquote><pre>
3497    * CoprocessorRpcChannel channel = myAdmin.coprocessorService();
3498    * MyService.BlockingInterface service = MyService.newBlockingStub(channel);
3499    * MyCallRequest request = MyCallRequest.newBuilder()
3500    *     ...
3501    *     .build();
3502    * MyCallResponse response = service.myCall(null, request);
3503    * </pre></blockquote></div>
3504    *
3505    * @return A MasterCoprocessorRpcChannel instance
3506    */
3507   @Override
3508   public CoprocessorRpcChannel coprocessorService() {
3509     return new MasterCoprocessorRpcChannel(connection);
3510   }
3511 
3512   /**
3513    * Simple {@link Abortable}, throwing RuntimeException on abort.
3514    */
3515   private static class ThrowableAbortable implements Abortable {
3516 
3517     @Override
3518     public void abort(String why, Throwable e) {
3519       throw new RuntimeException(why, e);
3520     }
3521 
3522     @Override
3523     public boolean isAborted() {
3524       return true;
3525     }
3526   }
3527 }