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