View Javadoc

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