View Javadoc

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