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