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