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