1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.client;
20
21 import java.io.Closeable;
22 import java.io.IOException;
23 import java.io.InterruptedIOException;
24 import java.net.SocketTimeoutException;
25 import java.util.ArrayList;
26 import java.util.Arrays;
27 import java.util.HashMap;
28 import java.util.LinkedList;
29 import java.util.List;
30 import java.util.Map;
31 import java.util.Map.Entry;
32 import java.util.concurrent.atomic.AtomicInteger;
33 import java.util.concurrent.atomic.AtomicReference;
34 import java.util.concurrent.ExecutionException;
35 import java.util.concurrent.Future;
36 import java.util.concurrent.TimeUnit;
37 import java.util.concurrent.TimeoutException;
38 import java.util.regex.Pattern;
39
40 import org.apache.commons.logging.Log;
41 import org.apache.commons.logging.LogFactory;
42 import org.apache.hadoop.conf.Configuration;
43 import org.apache.hadoop.hbase.Abortable;
44 import org.apache.hadoop.hbase.ClusterStatus;
45 import org.apache.hadoop.hbase.DoNotRetryIOException;
46 import org.apache.hadoop.hbase.HBaseConfiguration;
47 import org.apache.hadoop.hbase.HColumnDescriptor;
48 import org.apache.hadoop.hbase.HConstants;
49 import org.apache.hadoop.hbase.HRegionInfo;
50 import org.apache.hadoop.hbase.HRegionLocation;
51 import org.apache.hadoop.hbase.HTableDescriptor;
52 import org.apache.hadoop.hbase.MasterNotRunningException;
53 import org.apache.hadoop.hbase.MetaTableAccessor;
54 import org.apache.hadoop.hbase.NamespaceDescriptor;
55 import org.apache.hadoop.hbase.NotServingRegionException;
56 import org.apache.hadoop.hbase.ProcedureInfo;
57 import org.apache.hadoop.hbase.RegionException;
58 import org.apache.hadoop.hbase.RegionLocations;
59 import org.apache.hadoop.hbase.ServerName;
60 import org.apache.hadoop.hbase.TableExistsException;
61 import org.apache.hadoop.hbase.TableName;
62 import org.apache.hadoop.hbase.TableNotDisabledException;
63 import org.apache.hadoop.hbase.TableNotEnabledException;
64 import org.apache.hadoop.hbase.TableNotFoundException;
65 import org.apache.hadoop.hbase.UnknownRegionException;
66 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
67 import org.apache.hadoop.hbase.backup.BackupRequest;
68 import org.apache.hadoop.hbase.backup.util.BackupClientUtil;
69 import org.apache.hadoop.hbase.classification.InterfaceAudience;
70 import org.apache.hadoop.hbase.classification.InterfaceStability;
71 import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
72 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor;
73 import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase;
74 import org.apache.hadoop.hbase.exceptions.DeserializationException;
75 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
76 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
77 import org.apache.hadoop.hbase.ipc.MasterCoprocessorRpcChannel;
78 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
79 import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
80 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
81 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
82 import org.apache.hadoop.hbase.protobuf.RequestConverter;
83 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
84 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
85 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
86 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionRequest;
87 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
88 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
89 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
90 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
91 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
92 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
93 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
94 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.UpdateConfigurationRequest;
95 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesRequest;
96 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BackupTablesResponse;
97 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
98 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
99 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
100 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
101 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
102 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
103 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
104 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
105 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureResponse;
106 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
107 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AssignRegionRequest;
108 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateNamespaceRequest;
109 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableRequest;
110 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.CreateTableResponse;
111 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteColumnRequest;
112 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteNamespaceRequest;
113 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteSnapshotRequest;
114 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableRequest;
115 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DeleteTableResponse;
116 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableRequest;
117 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DisableTableResponse;
118 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.DispatchMergingRegionsRequest;
119 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableRequest;
120 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.EnableTableResponse;
121 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
122 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureResponse;
123 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetClusterStatusRequest;
124 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetCompletedSnapshotsRequest;
125 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
126 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultRequest;
127 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetProcedureResultResponse;
128 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
129 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetSchemaAlterStatusResponse;
130 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
131 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableDescriptorsResponse;
132 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.GetTableNamesRequest;
133 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneRequest;
134 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsProcedureDoneResponse;
135 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneRequest;
136 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsRestoreSnapshotDoneResponse;
137 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
138 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
139 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListNamespaceDescriptorsRequest;
140 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListProceduresRequest;
141 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest;
142 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest;
143 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampForRegionRequest;
144 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MajorCompactionTimestampRequest;
145 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyColumnRequest;
146 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyNamespaceRequest;
147 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ModifyTableRequest;
148 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest;
149 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotRequest;
150 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RestoreSnapshotResponse;
151 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
152 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningRequest;
153 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
154 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
155 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
156 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
157 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
158 import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
159 import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos;
160 import org.apache.hadoop.hbase.quotas.QuotaFilter;
161 import org.apache.hadoop.hbase.quotas.QuotaRetriever;
162 import org.apache.hadoop.hbase.quotas.QuotaSettings;
163 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
164 import org.apache.hadoop.hbase.snapshot.ClientSnapshotDescriptionUtils;
165 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
166 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
167 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
168 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
169 import org.apache.hadoop.hbase.util.Addressing;
170 import org.apache.hadoop.hbase.util.Bytes;
171 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
172 import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
173 import org.apache.hadoop.hbase.util.Pair;
174 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
175 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
176 import org.apache.hadoop.ipc.RemoteException;
177 import org.apache.hadoop.util.StringUtils;
178 import org.apache.zookeeper.KeeperException;
179
180 import com.google.common.annotations.VisibleForTesting;
181 import com.google.protobuf.ByteString;
182 import com.google.protobuf.ServiceException;
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200 @InterfaceAudience.Private
201 @InterfaceStability.Evolving
202 public class HBaseAdmin implements Admin {
203 private static final Log LOG = LogFactory.getLog(HBaseAdmin.class);
204
205 private static final String ZK_IDENTIFIER_PREFIX = "hbase-admin-on-";
206
207 private ClusterConnection connection;
208
209 private volatile Configuration conf;
210 private final long pause;
211 private final int numRetries;
212
213
214
215 private final int retryLongerMultiplier;
216 private final int syncWaitTimeout;
217 private final long backupWaitTimeout;
218 private boolean aborted;
219 private boolean cleanupConnectionOnClose = false;
220 private boolean closed = false;
221 private int operationTimeout;
222
223 private RpcRetryingCallerFactory rpcCallerFactory;
224 private RpcControllerFactory rpcControllerFactory;
225
226
227
228
229
230
231
232
233
234 @Deprecated
235 public HBaseAdmin(Configuration c)
236 throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
237
238
239 this(ConnectionManager.getConnectionInternal(new Configuration(c)));
240 this.cleanupConnectionOnClose = true;
241 }
242
243 @Override
244 public int getOperationTimeout() {
245 return operationTimeout;
246 }
247
248
249
250
251
252
253
254
255
256
257
258
259 @Deprecated
260 public HBaseAdmin(Connection connection)
261 throws MasterNotRunningException, ZooKeeperConnectionException {
262 this((ClusterConnection)connection);
263 }
264
265 HBaseAdmin(ClusterConnection connection) {
266 this.conf = connection.getConfiguration();
267 this.connection = connection;
268
269
270 this.pause = this.conf.getLong(HConstants.HBASE_CLIENT_PAUSE,
271 HConstants.DEFAULT_HBASE_CLIENT_PAUSE);
272 this.numRetries = this.conf.getInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
273 HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
274 this.retryLongerMultiplier = this.conf.getInt(
275 "hbase.client.retries.longer.multiplier", 10);
276 this.operationTimeout = this.conf.getInt(HConstants.HBASE_CLIENT_OPERATION_TIMEOUT,
277 HConstants.DEFAULT_HBASE_CLIENT_OPERATION_TIMEOUT);
278 this.syncWaitTimeout = this.conf.getInt(
279 "hbase.client.sync.wait.timeout.msec", 10 * 60000);
280 this.backupWaitTimeout = this.conf.getInt(
281 "hbase.client.backup.wait.timeout.sec", 24 * 3600);
282 this.rpcCallerFactory = connection.getRpcRetryingCallerFactory();
283 this.rpcControllerFactory = connection.getRpcControllerFactory();
284 }
285
286 @Override
287 public void abort(String why, Throwable e) {
288
289 this.aborted = true;
290 throw new RuntimeException(why, e);
291 }
292
293 @Override
294 public boolean isAborted(){
295 return this.aborted;
296 }
297
298
299
300
301
302
303
304
305 @Override
306 public boolean abortProcedure(
307 final long procId,
308 final boolean mayInterruptIfRunning) throws IOException {
309 Future<Boolean> future = abortProcedureAsync(procId, mayInterruptIfRunning);
310 try {
311 return future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
312 } catch (InterruptedException e) {
313 throw new InterruptedIOException("Interrupted when waiting for procedure to be cancelled");
314 } catch (TimeoutException e) {
315 throw new TimeoutIOException(e);
316 } catch (ExecutionException e) {
317 if (e.getCause() instanceof IOException) {
318 throw (IOException)e.getCause();
319 } else {
320 throw new IOException(e.getCause());
321 }
322 }
323 }
324
325
326
327
328
329
330
331
332
333
334
335
336
337 @Override
338 public Future<Boolean> abortProcedureAsync(
339 final long procId,
340 final boolean mayInterruptIfRunning) throws IOException {
341 Boolean abortProcResponse = executeCallable(
342 new MasterCallable<AbortProcedureResponse>(getConnection()) {
343 @Override
344 public AbortProcedureResponse call(int callTimeout) throws ServiceException {
345 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
346 controller.setCallTimeout(callTimeout);
347 AbortProcedureRequest abortProcRequest =
348 AbortProcedureRequest.newBuilder().setProcId(procId).build();
349 return master.abortProcedure(controller, abortProcRequest);
350 }
351 }).getIsProcedureAborted();
352
353 AbortProcedureFuture abortProcFuture =
354 new AbortProcedureFuture(this, procId, abortProcResponse);
355 return abortProcFuture;
356 }
357
358 private static class AbortProcedureFuture extends ProcedureFuture<Boolean> {
359 private boolean isAbortInProgress;
360
361 public AbortProcedureFuture(
362 final HBaseAdmin admin,
363 final Long procId,
364 final Boolean abortProcResponse) {
365 super(admin, procId);
366 this.isAbortInProgress = abortProcResponse;
367 }
368
369 @Override
370 public Boolean get(long timeout, TimeUnit unit)
371 throws InterruptedException, ExecutionException, TimeoutException {
372 if (!this.isAbortInProgress) {
373 return false;
374 }
375 super.get(timeout, unit);
376 return true;
377 }
378 }
379
380
381 @Override
382 public HConnection getConnection() {
383 return connection;
384 }
385
386
387
388
389
390
391
392 @Deprecated
393 public boolean isMasterRunning()
394 throws MasterNotRunningException, ZooKeeperConnectionException {
395 return connection.isMasterRunning();
396 }
397
398
399
400
401
402
403 @Override
404 public boolean tableExists(final TableName tableName) throws IOException {
405 return MetaTableAccessor.tableExists(connection, tableName);
406 }
407
408 public boolean tableExists(final byte[] tableName)
409 throws IOException {
410 return tableExists(TableName.valueOf(tableName));
411 }
412
413 public boolean tableExists(final String tableName)
414 throws IOException {
415 return tableExists(TableName.valueOf(tableName));
416 }
417
418 @Override
419 public HTableDescriptor[] listTables() throws IOException {
420 return listTables((Pattern)null, false);
421 }
422
423 @Override
424 public HTableDescriptor[] listTables(Pattern pattern) throws IOException {
425 return listTables(pattern, false);
426 }
427
428 @Override
429 public HTableDescriptor[] listTables(String regex) throws IOException {
430 return listTables(Pattern.compile(regex), false);
431 }
432
433 @Override
434 public HTableDescriptor[] listTables(final Pattern pattern, final boolean includeSysTables)
435 throws IOException {
436 return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
437 @Override
438 public HTableDescriptor[] call(int callTimeout) throws ServiceException {
439 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
440 controller.setCallTimeout(callTimeout);
441 GetTableDescriptorsRequest req =
442 RequestConverter.buildGetTableDescriptorsRequest(pattern, includeSysTables);
443 return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
444 }
445 });
446 }
447
448 @Override
449 public HTableDescriptor[] listTables(String regex, boolean includeSysTables)
450 throws IOException {
451 return listTables(Pattern.compile(regex), includeSysTables);
452 }
453
454
455
456
457
458
459
460 @Deprecated
461 public String[] getTableNames() throws IOException {
462 TableName[] tableNames = listTableNames();
463 String result[] = new String[tableNames.length];
464 for (int i = 0; i < tableNames.length; i++) {
465 result[i] = tableNames[i].getNameAsString();
466 }
467 return result;
468 }
469
470
471
472
473
474
475
476
477 @Deprecated
478 public String[] getTableNames(Pattern pattern) throws IOException {
479 TableName[] tableNames = listTableNames(pattern);
480 String result[] = new String[tableNames.length];
481 for (int i = 0; i < tableNames.length; i++) {
482 result[i] = tableNames[i].getNameAsString();
483 }
484 return result;
485 }
486
487
488
489
490
491
492
493
494 @Deprecated
495 public String[] getTableNames(String regex) throws IOException {
496 return getTableNames(Pattern.compile(regex));
497 }
498
499 @Override
500 public TableName[] listTableNames() throws IOException {
501 return listTableNames((Pattern)null, false);
502 }
503
504 @Override
505 public TableName[] listTableNames(Pattern pattern) throws IOException {
506 return listTableNames(pattern, false);
507 }
508
509 @Override
510 public TableName[] listTableNames(String regex) throws IOException {
511 return listTableNames(Pattern.compile(regex), false);
512 }
513
514 @Override
515 public TableName[] listTableNames(final Pattern pattern, final boolean includeSysTables)
516 throws IOException {
517 return executeCallable(new MasterCallable<TableName[]>(getConnection()) {
518 @Override
519 public TableName[] call(int callTimeout) throws ServiceException {
520 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
521 controller.setCallTimeout(callTimeout);
522 GetTableNamesRequest req =
523 RequestConverter.buildGetTableNamesRequest(pattern, includeSysTables);
524 return ProtobufUtil.getTableNameArray(master.getTableNames(controller, req)
525 .getTableNamesList());
526 }
527 });
528 }
529
530 @Override
531 public TableName[] listTableNames(final String regex, final boolean includeSysTables)
532 throws IOException {
533 return listTableNames(Pattern.compile(regex), includeSysTables);
534 }
535
536
537
538
539
540
541
542
543 @Override
544 public HTableDescriptor getTableDescriptor(final TableName tableName) throws IOException {
545 return getTableDescriptor(tableName, getConnection(), rpcCallerFactory, rpcControllerFactory,
546 operationTimeout);
547 }
548
549 static HTableDescriptor getTableDescriptor(final TableName tableName, HConnection connection,
550 RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory,
551 int operationTimeout) throws IOException {
552 if (tableName == null) return null;
553 HTableDescriptor htd = executeCallable(new MasterCallable<HTableDescriptor>(connection) {
554 @Override
555 public HTableDescriptor call(int callTimeout) throws ServiceException {
556 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
557 controller.setCallTimeout(callTimeout);
558 GetTableDescriptorsResponse htds;
559 GetTableDescriptorsRequest req =
560 RequestConverter.buildGetTableDescriptorsRequest(tableName);
561 htds = master.getTableDescriptors(controller, req);
562
563 if (!htds.getTableSchemaList().isEmpty()) {
564 return HTableDescriptor.convert(htds.getTableSchemaList().get(0));
565 }
566 return null;
567 }
568 }, rpcCallerFactory, operationTimeout);
569 if (htd != null) {
570 return htd;
571 }
572 throw new TableNotFoundException(tableName.getNameAsString());
573 }
574
575 public HTableDescriptor getTableDescriptor(final byte[] tableName)
576 throws TableNotFoundException, IOException {
577 return getTableDescriptor(TableName.valueOf(tableName));
578 }
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593 @Override
594 public void createTable(HTableDescriptor desc)
595 throws IOException {
596 createTable(desc, null);
597 }
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623 @Override
624 public void createTable(HTableDescriptor desc, byte [] startKey,
625 byte [] endKey, int numRegions)
626 throws IOException {
627 if(numRegions < 3) {
628 throw new IllegalArgumentException("Must create at least three regions");
629 } else if(Bytes.compareTo(startKey, endKey) >= 0) {
630 throw new IllegalArgumentException("Start key must be smaller than end key");
631 }
632 if (numRegions == 3) {
633 createTable(desc, new byte[][]{startKey, endKey});
634 return;
635 }
636 byte [][] splitKeys = Bytes.split(startKey, endKey, numRegions - 3);
637 if(splitKeys == null || splitKeys.length != numRegions - 1) {
638 throw new IllegalArgumentException("Unable to split key range into enough regions");
639 }
640 createTable(desc, splitKeys);
641 }
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660 @Override
661 public void createTable(final HTableDescriptor desc, byte [][] splitKeys)
662 throws IOException {
663 Future<Void> future = createTableAsyncV2(desc, splitKeys);
664 try {
665
666 future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
667 } catch (InterruptedException e) {
668 throw new InterruptedIOException("Interrupted when waiting" +
669 " for table to be enabled; meta scan was done");
670 } catch (TimeoutException e) {
671 throw new TimeoutIOException(e);
672 } catch (ExecutionException e) {
673 if (e.getCause() instanceof IOException) {
674 throw (IOException)e.getCause();
675 } else {
676 throw new IOException(e.getCause());
677 }
678 }
679 }
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697 @Override
698 public void createTableAsync(final HTableDescriptor desc, final byte [][] splitKeys)
699 throws IOException {
700 createTableAsyncV2(desc, splitKeys);
701 }
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719 private Future<Void> createTableAsyncV2(final HTableDescriptor desc, final byte[][] splitKeys)
720 throws IOException {
721 if (desc.getTableName() == null) {
722 throw new IllegalArgumentException("TableName cannot be null");
723 }
724 if (splitKeys != null && splitKeys.length > 0) {
725 Arrays.sort(splitKeys, Bytes.BYTES_COMPARATOR);
726
727 byte[] lastKey = null;
728 for (byte[] splitKey : splitKeys) {
729 if (Bytes.compareTo(splitKey, HConstants.EMPTY_BYTE_ARRAY) == 0) {
730 throw new IllegalArgumentException(
731 "Empty split key must not be passed in the split keys.");
732 }
733 if (lastKey != null && Bytes.equals(splitKey, lastKey)) {
734 throw new IllegalArgumentException("All split keys must be unique, " +
735 "found duplicate: " + Bytes.toStringBinary(splitKey) +
736 ", " + Bytes.toStringBinary(lastKey));
737 }
738 lastKey = splitKey;
739 }
740 }
741
742 CreateTableResponse response = executeCallable(
743 new MasterCallable<CreateTableResponse>(getConnection()) {
744 @Override
745 public CreateTableResponse call(int callTimeout) throws ServiceException {
746 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
747 controller.setCallTimeout(callTimeout);
748 controller.setPriority(desc.getTableName());
749 CreateTableRequest request = RequestConverter.buildCreateTableRequest(
750 desc, splitKeys);
751 return master.createTable(controller, request);
752 }
753 });
754 return new CreateTableFuture(this, desc, splitKeys, response);
755 }
756
757 private static class CreateTableFuture extends ProcedureFuture<Void> {
758 private final HTableDescriptor desc;
759 private final byte[][] splitKeys;
760
761 public CreateTableFuture(final HBaseAdmin admin, final HTableDescriptor desc,
762 final byte[][] splitKeys, final CreateTableResponse response) {
763 super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
764 this.splitKeys = splitKeys;
765 this.desc = desc;
766 }
767
768 @Override
769 protected Void waitOperationResult(final long deadlineTs)
770 throws IOException, TimeoutException {
771 waitForTableEnabled(deadlineTs);
772 waitForAllRegionsOnline(deadlineTs);
773 return null;
774 }
775
776 @Override
777 protected Void postOperationResult(final Void result, final long deadlineTs)
778 throws IOException, TimeoutException {
779 LOG.info("Created " + desc.getTableName());
780 return result;
781 }
782
783 private void waitForTableEnabled(final long deadlineTs)
784 throws IOException, TimeoutException {
785 waitForState(deadlineTs, new WaitForStateCallable() {
786 @Override
787 public boolean checkState(int tries) throws IOException {
788 try {
789 if (getAdmin().isTableAvailable(desc.getTableName())) {
790 return true;
791 }
792 } catch (TableNotFoundException tnfe) {
793 LOG.debug("Table "+ desc.getTableName() +" was not enabled, sleeping. tries="+ tries);
794 }
795 return false;
796 }
797
798 @Override
799 public void throwInterruptedException() throws InterruptedIOException {
800 throw new InterruptedIOException("Interrupted when waiting for table " +
801 desc.getTableName() + " to be enabled");
802 }
803
804 @Override
805 public void throwTimeoutException(long elapsedTime) throws TimeoutException {
806 throw new TimeoutException("Table " + desc.getTableName() +
807 " not enabled after " + elapsedTime + "msec");
808 }
809 });
810 }
811
812 private void waitForAllRegionsOnline(final long deadlineTs)
813 throws IOException, TimeoutException {
814 final AtomicInteger actualRegCount = new AtomicInteger(0);
815 final MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
816 @Override
817 public boolean processRow(Result rowResult) throws IOException {
818 RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
819 if (list == null) {
820 LOG.warn("No serialized HRegionInfo in " + rowResult);
821 return true;
822 }
823 HRegionLocation l = list.getRegionLocation();
824 if (l == null) {
825 return true;
826 }
827 if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
828 return false;
829 }
830 if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
831 HRegionLocation[] locations = list.getRegionLocations();
832 for (HRegionLocation location : locations) {
833 if (location == null) continue;
834 ServerName serverName = location.getServerName();
835
836 if (serverName != null && serverName.getHostAndPort() != null) {
837 actualRegCount.incrementAndGet();
838 }
839 }
840 return true;
841 }
842 };
843
844 int tries = 0;
845 IOException serverEx = null;
846 int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
847 while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
848 actualRegCount.set(0);
849 MetaScanner.metaScan(getAdmin().getConnection(), visitor, desc.getTableName());
850 if (actualRegCount.get() == numRegs) {
851
852 return;
853 }
854
855 try {
856 Thread.sleep(getPauseTime(tries++, getAdmin().pause));
857 } catch (InterruptedException e) {
858 throw new InterruptedIOException("Interrupted when opening" +
859 " regions; " + actualRegCount.get() + " of " + numRegs +
860 " regions processed so far");
861 }
862 }
863 throw new TimeoutException("Only " + actualRegCount.get() +
864 " of " + numRegs + " regions are online; retries exhausted.");
865 }
866 }
867
868 public void deleteTable(final String tableName) throws IOException {
869 deleteTable(TableName.valueOf(tableName));
870 }
871
872 public void deleteTable(final byte[] tableName) throws IOException {
873 deleteTable(TableName.valueOf(tableName));
874 }
875
876
877
878
879
880
881
882
883 @Override
884 public void deleteTable(final TableName tableName) throws IOException {
885 Future<Void> future = deleteTableAsyncV2(tableName);
886 try {
887 future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
888 } catch (InterruptedException e) {
889 throw new InterruptedIOException("Interrupted when waiting for table to be deleted");
890 } catch (TimeoutException e) {
891 throw new TimeoutIOException(e);
892 } catch (ExecutionException e) {
893 if (e.getCause() instanceof IOException) {
894 throw (IOException)e.getCause();
895 } else {
896 throw new IOException(e.getCause());
897 }
898 }
899 }
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915 private Future<Void> deleteTableAsyncV2(final TableName tableName) throws IOException {
916 DeleteTableResponse response = executeCallable(
917 new MasterCallable<DeleteTableResponse>(getConnection()) {
918 @Override
919 public DeleteTableResponse call(int callTimeout) throws ServiceException {
920 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
921 controller.setCallTimeout(callTimeout);
922 controller.setPriority(tableName);
923 DeleteTableRequest req =
924 RequestConverter.buildDeleteTableRequest(tableName);
925 return master.deleteTable(controller,req);
926 }
927 });
928 return new DeleteTableFuture(this, tableName, response);
929 }
930
931 private static class DeleteTableFuture extends ProcedureFuture<Void> {
932 private final TableName tableName;
933
934 public DeleteTableFuture(final HBaseAdmin admin, final TableName tableName,
935 final DeleteTableResponse response) {
936 super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
937 this.tableName = tableName;
938 }
939
940 @Override
941 protected Void waitOperationResult(final long deadlineTs)
942 throws IOException, TimeoutException {
943 waitTableNotFound(deadlineTs);
944 return null;
945 }
946
947 @Override
948 protected Void postOperationResult(final Void result, final long deadlineTs)
949 throws IOException, TimeoutException {
950
951 getAdmin().getConnection().clearRegionCache(tableName);
952 LOG.info("Deleted " + tableName);
953 return result;
954 }
955
956 private void waitTableNotFound(final long deadlineTs)
957 throws IOException, TimeoutException {
958 waitForState(deadlineTs, new WaitForStateCallable() {
959 @Override
960 public boolean checkState(int tries) throws IOException {
961 return !getAdmin().tableExists(tableName);
962 }
963
964 @Override
965 public void throwInterruptedException() throws InterruptedIOException {
966 throw new InterruptedIOException("Interrupted when waiting for table to be deleted");
967 }
968
969 @Override
970 public void throwTimeoutException(long elapsedTime) throws TimeoutException {
971 throw new TimeoutException("Table " + tableName + " not yet deleted after " +
972 elapsedTime + "msec");
973 }
974 });
975 }
976 }
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991 @Override
992 public HTableDescriptor[] deleteTables(String regex) throws IOException {
993 return deleteTables(Pattern.compile(regex));
994 }
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007 @Override
1008 public HTableDescriptor[] deleteTables(Pattern pattern) throws IOException {
1009 List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
1010 for (HTableDescriptor table : listTables(pattern)) {
1011 try {
1012 deleteTable(table.getTableName());
1013 } catch (IOException ex) {
1014 LOG.info("Failed to delete table " + table.getTableName(), ex);
1015 failed.add(table);
1016 }
1017 }
1018 return failed.toArray(new HTableDescriptor[failed.size()]);
1019 }
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029 @Override
1030 public void truncateTable(final TableName tableName, final boolean preserveSplits)
1031 throws IOException {
1032 executeCallable(new MasterCallable<Void>(getConnection()) {
1033 @Override
1034 public Void call(int callTimeout) throws ServiceException {
1035 TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
1036 tableName, preserveSplits);
1037 master.truncateTable(null, req);
1038 return null;
1039 }
1040 });
1041 }
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056 @Override
1057 public void enableTable(final TableName tableName)
1058 throws IOException {
1059 Future<Void> future = enableTableAsyncV2(tableName);
1060 try {
1061 future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
1062 } catch (InterruptedException e) {
1063 throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
1064 } catch (TimeoutException e) {
1065 throw new TimeoutIOException(e);
1066 } catch (ExecutionException e) {
1067 if (e.getCause() instanceof IOException) {
1068 throw (IOException)e.getCause();
1069 } else {
1070 throw new IOException(e.getCause());
1071 }
1072 }
1073 }
1074
1075 public void enableTable(final byte[] tableName)
1076 throws IOException {
1077 enableTable(TableName.valueOf(tableName));
1078 }
1079
1080 public void enableTable(final String tableName)
1081 throws IOException {
1082 enableTable(TableName.valueOf(tableName));
1083 }
1084
1085
1086
1087
1088
1089
1090
1091
1092 private void waitUntilTableIsEnabled(final TableName tableName) throws IOException {
1093 boolean enabled = false;
1094 long start = EnvironmentEdgeManager.currentTime();
1095 for (int tries = 0; tries < (this.numRetries * this.retryLongerMultiplier); tries++) {
1096 try {
1097 enabled = isTableEnabled(tableName);
1098 } catch (TableNotFoundException tnfe) {
1099
1100 enabled = false;
1101 }
1102 enabled = enabled && isTableAvailable(tableName);
1103 if (enabled) {
1104 break;
1105 }
1106 long sleep = getPauseTime(tries, pause);
1107 if (LOG.isDebugEnabled()) {
1108 LOG.debug("Sleeping= " + sleep + "ms, waiting for all regions to be " +
1109 "enabled in " + tableName);
1110 }
1111 try {
1112 Thread.sleep(sleep);
1113 } catch (InterruptedException e) {
1114
1115
1116 throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
1117 }
1118 }
1119 if (!enabled) {
1120 long msec = EnvironmentEdgeManager.currentTime() - start;
1121 throw new IOException("Table '" + tableName +
1122 "' not yet enabled, after " + msec + "ms.");
1123 }
1124 }
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136 @Override
1137 public void enableTableAsync(final TableName tableName)
1138 throws IOException {
1139 enableTableAsyncV2(tableName);
1140 }
1141
1142 public void enableTableAsync(final byte[] tableName)
1143 throws IOException {
1144 enableTable(TableName.valueOf(tableName));
1145 }
1146
1147 public void enableTableAsync(final String tableName)
1148 throws IOException {
1149 enableTableAsync(TableName.valueOf(tableName));
1150 }
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165 private Future<Void> enableTableAsyncV2(final TableName tableName) throws IOException {
1166 TableName.isLegalFullyQualifiedTableName(tableName.getName());
1167 EnableTableResponse response = executeCallable(
1168 new MasterCallable<EnableTableResponse>(getConnection()) {
1169 @Override
1170 public EnableTableResponse call(int callTimeout) throws ServiceException {
1171 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1172 controller.setCallTimeout(callTimeout);
1173 controller.setPriority(tableName);
1174
1175 LOG.info("Started enable of " + tableName);
1176 EnableTableRequest req =
1177 RequestConverter.buildEnableTableRequest(tableName);
1178 return master.enableTable(controller, req);
1179 }
1180 });
1181 return new EnableTableFuture(this, tableName, response);
1182 }
1183
1184 private static class EnableTableFuture extends ProcedureFuture<Void> {
1185 private final TableName tableName;
1186
1187 public EnableTableFuture(final HBaseAdmin admin, final TableName tableName,
1188 final EnableTableResponse response) {
1189 super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
1190 this.tableName = tableName;
1191 }
1192
1193 @Override
1194 protected Void waitOperationResult(final long deadlineTs)
1195 throws IOException, TimeoutException {
1196 waitTableEnabled(deadlineTs);
1197 return null;
1198 }
1199
1200 @Override
1201 protected Void postOperationResult(final Void result, final long deadlineTs)
1202 throws IOException, TimeoutException {
1203 LOG.info("Enabled " + tableName);
1204 return result;
1205 }
1206
1207 private void waitTableEnabled(final long deadlineTs)
1208 throws IOException, TimeoutException {
1209 waitForState(deadlineTs, new WaitForStateCallable() {
1210 @Override
1211 public boolean checkState(int tries) throws IOException {
1212 boolean enabled;
1213 try {
1214 enabled = getAdmin().isTableEnabled(tableName);
1215 } catch (TableNotFoundException tnfe) {
1216 return false;
1217 }
1218 return enabled && getAdmin().isTableAvailable(tableName);
1219 }
1220
1221 @Override
1222 public void throwInterruptedException() throws InterruptedIOException {
1223 throw new InterruptedIOException("Interrupted when waiting for table to be enabled");
1224 }
1225
1226 @Override
1227 public void throwTimeoutException(long elapsedTime) throws TimeoutException {
1228 throw new TimeoutException("Table " + tableName + " not yet enabled after " +
1229 elapsedTime + "msec");
1230 }
1231 });
1232 }
1233 }
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247 @Override
1248 public HTableDescriptor[] enableTables(String regex) throws IOException {
1249 return enableTables(Pattern.compile(regex));
1250 }
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262 @Override
1263 public HTableDescriptor[] enableTables(Pattern pattern) throws IOException {
1264 List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
1265 for (HTableDescriptor table : listTables(pattern)) {
1266 if (isTableDisabled(table.getTableName())) {
1267 try {
1268 enableTable(table.getTableName());
1269 } catch (IOException ex) {
1270 LOG.info("Failed to enable table " + table.getTableName(), ex);
1271 failed.add(table);
1272 }
1273 }
1274 }
1275 return failed.toArray(new HTableDescriptor[failed.size()]);
1276 }
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291 @Override
1292 public void disableTableAsync(final TableName tableName) throws IOException {
1293 disableTableAsyncV2(tableName);
1294 }
1295
1296 public void disableTableAsync(final byte[] tableName) throws IOException {
1297 disableTableAsync(TableName.valueOf(tableName));
1298 }
1299
1300 public void disableTableAsync(final String tableName) throws IOException {
1301 disableTableAsync(TableName.valueOf(tableName));
1302 }
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315 @Override
1316 public void disableTable(final TableName tableName)
1317 throws IOException {
1318 Future<Void> future = disableTableAsyncV2(tableName);
1319 try {
1320 future.get(syncWaitTimeout, TimeUnit.MILLISECONDS);
1321 } catch (InterruptedException e) {
1322 throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
1323 } catch (TimeoutException e) {
1324 throw new TimeoutIOException(e);
1325 } catch (ExecutionException e) {
1326 if (e.getCause() instanceof IOException) {
1327 throw (IOException)e.getCause();
1328 } else {
1329 throw new IOException(e.getCause());
1330 }
1331 }
1332 }
1333
1334 public void disableTable(final byte[] tableName)
1335 throws IOException {
1336 disableTable(TableName.valueOf(tableName));
1337 }
1338
1339 public void disableTable(final String tableName)
1340 throws IOException {
1341 disableTable(TableName.valueOf(tableName));
1342 }
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357 private Future<Void> disableTableAsyncV2(final TableName tableName) throws IOException {
1358 TableName.isLegalFullyQualifiedTableName(tableName.getName());
1359 DisableTableResponse response = executeCallable(
1360 new MasterCallable<DisableTableResponse>(getConnection()) {
1361 @Override
1362 public DisableTableResponse call(int callTimeout) throws ServiceException {
1363 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1364 controller.setCallTimeout(callTimeout);
1365 controller.setPriority(tableName);
1366
1367 LOG.info("Started disable of " + tableName);
1368 DisableTableRequest req =
1369 RequestConverter.buildDisableTableRequest(
1370 tableName);
1371 return master.disableTable(controller, req);
1372 }
1373 });
1374 return new DisableTableFuture(this, tableName, response);
1375 }
1376
1377 private static class DisableTableFuture extends ProcedureFuture<Void> {
1378 private final TableName tableName;
1379
1380 public DisableTableFuture(final HBaseAdmin admin, final TableName tableName,
1381 final DisableTableResponse response) {
1382 super(admin, (response != null && response.hasProcId()) ? response.getProcId() : null);
1383 this.tableName = tableName;
1384 }
1385
1386 @Override
1387 protected Void waitOperationResult(final long deadlineTs)
1388 throws IOException, TimeoutException {
1389 waitTableDisabled(deadlineTs);
1390 return null;
1391 }
1392
1393 @Override
1394 protected Void postOperationResult(final Void result, final long deadlineTs)
1395 throws IOException, TimeoutException {
1396 LOG.info("Disabled " + tableName);
1397 return result;
1398 }
1399
1400 private void waitTableDisabled(final long deadlineTs)
1401 throws IOException, TimeoutException {
1402 waitForState(deadlineTs, new WaitForStateCallable() {
1403 @Override
1404 public boolean checkState(int tries) throws IOException {
1405 return getAdmin().isTableDisabled(tableName);
1406 }
1407
1408 @Override
1409 public void throwInterruptedException() throws InterruptedIOException {
1410 throw new InterruptedIOException("Interrupted when waiting for table to be disabled");
1411 }
1412
1413 @Override
1414 public void throwTimeoutException(long elapsedTime) throws TimeoutException {
1415 throw new TimeoutException("Table " + tableName + " not yet disabled after " +
1416 elapsedTime + "msec");
1417 }
1418 });
1419 }
1420 }
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435 @Override
1436 public HTableDescriptor[] disableTables(String regex) throws IOException {
1437 return disableTables(Pattern.compile(regex));
1438 }
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451 @Override
1452 public HTableDescriptor[] disableTables(Pattern pattern) throws IOException {
1453 List<HTableDescriptor> failed = new LinkedList<HTableDescriptor>();
1454 for (HTableDescriptor table : listTables(pattern)) {
1455 if (isTableEnabled(table.getTableName())) {
1456 try {
1457 disableTable(table.getTableName());
1458 } catch (IOException ex) {
1459 LOG.info("Failed to disable table " + table.getTableName(), ex);
1460 failed.add(table);
1461 }
1462 }
1463 }
1464 return failed.toArray(new HTableDescriptor[failed.size()]);
1465 }
1466
1467
1468
1469
1470
1471 private void checkTableExistence(TableName tableName) throws IOException {
1472 if (!tableExists(tableName)) {
1473 throw new TableNotFoundException(tableName);
1474 }
1475 }
1476
1477
1478
1479
1480
1481
1482 @Override
1483 public boolean isTableEnabled(TableName tableName) throws IOException {
1484 checkTableExistence(tableName);
1485 return connection.isTableEnabled(tableName);
1486 }
1487
1488 public boolean isTableEnabled(byte[] tableName) throws IOException {
1489 return isTableEnabled(TableName.valueOf(tableName));
1490 }
1491
1492 public boolean isTableEnabled(String tableName) throws IOException {
1493 return isTableEnabled(TableName.valueOf(tableName));
1494 }
1495
1496
1497
1498
1499
1500
1501
1502
1503 @Override
1504 public boolean isTableDisabled(TableName tableName) throws IOException {
1505 checkTableExistence(tableName);
1506 return connection.isTableDisabled(tableName);
1507 }
1508
1509 public boolean isTableDisabled(byte[] tableName) throws IOException {
1510 return isTableDisabled(TableName.valueOf(tableName));
1511 }
1512
1513 public boolean isTableDisabled(String tableName) throws IOException {
1514 return isTableDisabled(TableName.valueOf(tableName));
1515 }
1516
1517
1518
1519
1520
1521
1522 @Override
1523 public boolean isTableAvailable(TableName tableName) throws IOException {
1524 return connection.isTableAvailable(tableName);
1525 }
1526
1527 public boolean isTableAvailable(byte[] tableName) throws IOException {
1528 return isTableAvailable(TableName.valueOf(tableName));
1529 }
1530
1531 public boolean isTableAvailable(String tableName) throws IOException {
1532 return isTableAvailable(TableName.valueOf(tableName));
1533 }
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547 @Override
1548 public boolean isTableAvailable(TableName tableName,
1549 byte[][] splitKeys) throws IOException {
1550 return connection.isTableAvailable(tableName, splitKeys);
1551 }
1552
1553 public boolean isTableAvailable(byte[] tableName,
1554 byte[][] splitKeys) throws IOException {
1555 return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1556 }
1557
1558 public boolean isTableAvailable(String tableName,
1559 byte[][] splitKeys) throws IOException {
1560 return isTableAvailable(TableName.valueOf(tableName), splitKeys);
1561 }
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574 @Override
1575 public Pair<Integer, Integer> getAlterStatus(final TableName tableName)
1576 throws IOException {
1577 return executeCallable(new MasterCallable<Pair<Integer, Integer>>(getConnection()) {
1578 @Override
1579 public Pair<Integer, Integer> call(int callTimeout) throws ServiceException {
1580 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1581 controller.setCallTimeout(callTimeout);
1582 controller.setPriority(tableName);
1583
1584 GetSchemaAlterStatusRequest req = RequestConverter
1585 .buildGetSchemaAlterStatusRequest(tableName);
1586 GetSchemaAlterStatusResponse ret = master.getSchemaAlterStatus(controller, req);
1587 Pair<Integer, Integer> pair = new Pair<Integer, Integer>(Integer.valueOf(ret
1588 .getYetToUpdateRegions()), Integer.valueOf(ret.getTotalRegions()));
1589 return pair;
1590 }
1591 });
1592 }
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606 @Override
1607 public Pair<Integer, Integer> getAlterStatus(final byte[] tableName)
1608 throws IOException {
1609 return getAlterStatus(TableName.valueOf(tableName));
1610 }
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620 public void addColumn(final byte[] tableName, HColumnDescriptor column)
1621 throws IOException {
1622 addColumn(TableName.valueOf(tableName), column);
1623 }
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633 public void addColumn(final String tableName, HColumnDescriptor column)
1634 throws IOException {
1635 addColumn(TableName.valueOf(tableName), column);
1636 }
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646 @Override
1647 public void addColumn(final TableName tableName, final HColumnDescriptor column)
1648 throws IOException {
1649 executeCallable(new MasterCallable<Void>(getConnection()) {
1650 @Override
1651 public Void call(int callTimeout) throws ServiceException {
1652 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1653 controller.setCallTimeout(callTimeout);
1654 controller.setPriority(tableName);
1655 AddColumnRequest req = RequestConverter.buildAddColumnRequest(tableName, column);
1656 master.addColumn(controller, req);
1657 return null;
1658 }
1659 });
1660 }
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670 public void deleteColumn(final byte[] tableName, final String columnName)
1671 throws IOException {
1672 deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
1673 }
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683 public void deleteColumn(final String tableName, final String columnName)
1684 throws IOException {
1685 deleteColumn(TableName.valueOf(tableName), Bytes.toBytes(columnName));
1686 }
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696 @Override
1697 public void deleteColumn(final TableName tableName, final byte [] columnName)
1698 throws IOException {
1699 executeCallable(new MasterCallable<Void>(getConnection()) {
1700 @Override
1701 public Void call(int callTimeout) throws ServiceException {
1702 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1703 controller.setCallTimeout(callTimeout);
1704 controller.setPriority(tableName);
1705 DeleteColumnRequest req = RequestConverter.buildDeleteColumnRequest(tableName, columnName);
1706 master.deleteColumn(controller, req);
1707 return null;
1708 }
1709 });
1710 }
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720 public void modifyColumn(final String tableName, HColumnDescriptor descriptor)
1721 throws IOException {
1722 modifyColumn(TableName.valueOf(tableName), descriptor);
1723 }
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733 public void modifyColumn(final byte[] tableName, HColumnDescriptor descriptor)
1734 throws IOException {
1735 modifyColumn(TableName.valueOf(tableName), descriptor);
1736 }
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746 @Override
1747 public void modifyColumn(final TableName tableName, final HColumnDescriptor descriptor)
1748 throws IOException {
1749 executeCallable(new MasterCallable<Void>(getConnection()) {
1750 @Override
1751 public Void call(int callTimeout) throws ServiceException {
1752 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1753 controller.setCallTimeout(callTimeout);
1754 controller.setPriority(tableName);
1755 ModifyColumnRequest req = RequestConverter.buildModifyColumnRequest(tableName, descriptor);
1756 master.modifyColumn(controller, req);
1757 return null;
1758 }
1759 });
1760 }
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770 @Override
1771 public void closeRegion(final String regionname, final String serverName)
1772 throws IOException {
1773 closeRegion(Bytes.toBytes(regionname), serverName);
1774 }
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786 @Override
1787 public void closeRegion(final byte [] regionname, final String serverName)
1788 throws IOException {
1789 if (serverName != null) {
1790 Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
1791 if (pair == null || pair.getFirst() == null) {
1792 throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1793 } else {
1794 closeRegion(ServerName.valueOf(serverName), pair.getFirst());
1795 }
1796 } else {
1797 Pair<HRegionInfo, ServerName> pair = MetaTableAccessor.getRegion(connection, regionname);
1798 if (pair == null) {
1799 throw new UnknownRegionException(Bytes.toStringBinary(regionname));
1800 } else if (pair.getSecond() == null) {
1801 throw new NoServerForRegionException(Bytes.toStringBinary(regionname));
1802 } else {
1803 closeRegion(pair.getSecond(), pair.getFirst());
1804 }
1805 }
1806 }
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829 @Override
1830 public boolean closeRegionWithEncodedRegionName(final String encodedRegionName,
1831 final String serverName) throws IOException {
1832 if (null == serverName || ("").equals(serverName.trim())) {
1833 throw new IllegalArgumentException(
1834 "The servername cannot be null or empty.");
1835 }
1836 ServerName sn = ServerName.valueOf(serverName);
1837 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1838
1839 CloseRegionRequest request =
1840 RequestConverter.buildCloseRegionRequest(sn, encodedRegionName, false);
1841 try {
1842 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1843
1844
1845 CloseRegionResponse response = admin.closeRegion(controller, request);
1846 boolean isRegionClosed = response.getClosed();
1847 if (false == isRegionClosed) {
1848 LOG.error("Not able to close the region " + encodedRegionName + ".");
1849 }
1850 return isRegionClosed;
1851 } catch (ServiceException se) {
1852 throw ProtobufUtil.getRemoteException(se);
1853 }
1854 }
1855
1856
1857
1858
1859
1860
1861
1862
1863 @Override
1864 public void closeRegion(final ServerName sn, final HRegionInfo hri)
1865 throws IOException {
1866 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1867 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1868
1869
1870 ProtobufUtil.closeRegion(controller, admin, sn, hri.getRegionName(), false);
1871 }
1872
1873
1874
1875
1876 @Override
1877 public List<HRegionInfo> getOnlineRegions(final ServerName sn) throws IOException {
1878 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1879 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1880 return ProtobufUtil.getOnlineRegions(controller, admin);
1881 }
1882
1883
1884
1885
1886 @Override
1887 public void flush(final TableName tableName) throws IOException {
1888 checkTableExists(tableName);
1889 if (isTableDisabled(tableName)) {
1890 LOG.info("Table is disabled: " + tableName.getNameAsString());
1891 return;
1892 }
1893 execProcedure("flush-table-proc", tableName.getNameAsString(),
1894 new HashMap<String, String>());
1895 }
1896
1897
1898
1899
1900 @Override
1901 public void flushRegion(final byte[] regionName) throws IOException {
1902 Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
1903 if (regionServerPair == null) {
1904 throw new IllegalArgumentException("Unknown regionname: " + Bytes.toStringBinary(regionName));
1905 }
1906 if (regionServerPair.getSecond() == null) {
1907 throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
1908 }
1909 flush(regionServerPair.getSecond(), regionServerPair.getFirst());
1910 }
1911
1912
1913
1914
1915
1916 @Deprecated
1917 public void flush(final String tableNameOrRegionName)
1918 throws IOException, InterruptedException {
1919 flush(Bytes.toBytes(tableNameOrRegionName));
1920 }
1921
1922
1923
1924
1925
1926 @Deprecated
1927 public void flush(final byte[] tableNameOrRegionName)
1928 throws IOException, InterruptedException {
1929 try {
1930 flushRegion(tableNameOrRegionName);
1931 } catch (IllegalArgumentException e) {
1932
1933 flush(TableName.valueOf(tableNameOrRegionName));
1934 }
1935 }
1936
1937 private void flush(final ServerName sn, final HRegionInfo hri)
1938 throws IOException {
1939 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
1940 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
1941 FlushRegionRequest request =
1942 RequestConverter.buildFlushRegionRequest(hri.getRegionName());
1943 try {
1944 admin.flushRegion(controller, request);
1945 } catch (ServiceException se) {
1946 throw ProtobufUtil.getRemoteException(se);
1947 }
1948 }
1949
1950
1951
1952
1953 @Override
1954 public void compact(final TableName tableName)
1955 throws IOException {
1956 compact(tableName, null, false, CompactType.NORMAL);
1957 }
1958
1959
1960
1961
1962 @Override
1963 public void compactRegion(final byte[] regionName)
1964 throws IOException {
1965 compactRegion(regionName, null, false);
1966 }
1967
1968
1969
1970
1971
1972 @Deprecated
1973 public void compact(final String tableNameOrRegionName)
1974 throws IOException {
1975 compact(Bytes.toBytes(tableNameOrRegionName));
1976 }
1977
1978
1979
1980
1981
1982 @Deprecated
1983 public void compact(final byte[] tableNameOrRegionName)
1984 throws IOException {
1985 try {
1986 compactRegion(tableNameOrRegionName, null, false);
1987 } catch (IllegalArgumentException e) {
1988 compact(TableName.valueOf(tableNameOrRegionName), null, false, CompactType.NORMAL);
1989 }
1990 }
1991
1992
1993
1994
1995 @Override
1996 public void compact(final TableName tableName, final byte[] columnFamily)
1997 throws IOException {
1998 compact(tableName, columnFamily, false, CompactType.NORMAL);
1999 }
2000
2001
2002
2003
2004 @Override
2005 public void compactRegion(final byte[] regionName, final byte[] columnFamily)
2006 throws IOException {
2007 compactRegion(regionName, columnFamily, false);
2008 }
2009
2010
2011
2012
2013
2014 @Deprecated
2015 public void compact(String tableOrRegionName, String columnFamily)
2016 throws IOException {
2017 compact(Bytes.toBytes(tableOrRegionName), Bytes.toBytes(columnFamily));
2018 }
2019
2020
2021
2022
2023
2024 @Deprecated
2025 public void compact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
2026 throws IOException {
2027 try {
2028 compactRegion(tableNameOrRegionName, columnFamily, false);
2029 } catch (IllegalArgumentException e) {
2030
2031 compact(TableName.valueOf(tableNameOrRegionName), columnFamily, false, CompactType.NORMAL);
2032 }
2033 }
2034
2035
2036
2037
2038 @Override
2039 public void compactRegionServer(final ServerName sn, boolean major)
2040 throws IOException, InterruptedException {
2041 for (HRegionInfo region : getOnlineRegions(sn)) {
2042 compact(sn, region, major, null);
2043 }
2044 }
2045
2046
2047
2048
2049 @Override
2050 public void majorCompact(final TableName tableName)
2051 throws IOException {
2052 compact(tableName, null, true, CompactType.NORMAL);
2053 }
2054
2055
2056
2057
2058 @Override
2059 public void majorCompactRegion(final byte[] regionName)
2060 throws IOException {
2061 compactRegion(regionName, null, true);
2062 }
2063
2064
2065
2066
2067
2068 @Deprecated
2069 public void majorCompact(final String tableNameOrRegionName)
2070 throws IOException {
2071 majorCompact(Bytes.toBytes(tableNameOrRegionName));
2072 }
2073
2074
2075
2076
2077
2078 @Deprecated
2079 public void majorCompact(final byte[] tableNameOrRegionName)
2080 throws IOException {
2081 try {
2082 compactRegion(tableNameOrRegionName, null, true);
2083 } catch (IllegalArgumentException e) {
2084
2085 compact(TableName.valueOf(tableNameOrRegionName), null, true, CompactType.NORMAL);
2086 }
2087 }
2088
2089
2090
2091
2092 @Override
2093 public void majorCompact(final TableName tableName, final byte[] columnFamily)
2094 throws IOException {
2095 compact(tableName, columnFamily, true, CompactType.NORMAL);
2096 }
2097
2098
2099
2100
2101 @Override
2102 public void majorCompactRegion(final byte[] regionName, final byte[] columnFamily)
2103 throws IOException {
2104 compactRegion(regionName, columnFamily, true);
2105 }
2106
2107
2108
2109
2110
2111 @Deprecated
2112 public void majorCompact(final String tableNameOrRegionName, final String columnFamily)
2113 throws IOException {
2114 majorCompact(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(columnFamily));
2115 }
2116
2117
2118
2119
2120
2121 @Deprecated
2122 public void majorCompact(final byte[] tableNameOrRegionName, final byte[] columnFamily)
2123 throws IOException {
2124 try {
2125 compactRegion(tableNameOrRegionName, columnFamily, true);
2126 } catch (IllegalArgumentException e) {
2127
2128 compact(TableName.valueOf(tableNameOrRegionName), columnFamily, true, CompactType.NORMAL);
2129 }
2130 }
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142 private void compact(final TableName tableName, final byte[] columnFamily,final boolean major,
2143 CompactType compactType) throws IOException {
2144 switch (compactType) {
2145 case MOB:
2146 ServerName master = getMasterAddress();
2147 compact(master, getMobRegionInfo(tableName), major, columnFamily);
2148 break;
2149 case NORMAL:
2150 default:
2151 ZooKeeperWatcher zookeeper = null;
2152 try {
2153 checkTableExists(tableName);
2154 zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2155 new ThrowableAbortable());
2156 List<Pair<HRegionInfo, ServerName>> pairs =
2157 MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection, tableName);
2158 for (Pair<HRegionInfo, ServerName> pair: pairs) {
2159 if (pair.getFirst().isOffline()) continue;
2160 if (pair.getSecond() == null) continue;
2161 try {
2162 compact(pair.getSecond(), pair.getFirst(), major, columnFamily);
2163 } catch (NotServingRegionException e) {
2164 if (LOG.isDebugEnabled()) {
2165 LOG.debug("Trying to" + (major ? " major" : "") + " compact " +
2166 pair.getFirst() + ": " +
2167 StringUtils.stringifyException(e));
2168 }
2169 }
2170 }
2171 } finally {
2172 if (zookeeper != null) {
2173 zookeeper.close();
2174 }
2175 }
2176 break;
2177 }
2178 }
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190 private void compactRegion(final byte[] regionName, final byte[] columnFamily,final boolean major)
2191 throws IOException {
2192 Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
2193 if (regionServerPair == null) {
2194 throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
2195 }
2196 if (regionServerPair.getSecond() == null) {
2197 throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
2198 }
2199 compact(regionServerPair.getSecond(), regionServerPair.getFirst(), major, columnFamily);
2200 }
2201
2202 private void compact(final ServerName sn, final HRegionInfo hri,
2203 final boolean major, final byte [] family)
2204 throws IOException {
2205 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2206 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2207 CompactRegionRequest request =
2208 RequestConverter.buildCompactRegionRequest(hri.getRegionName(), major, family);
2209 try {
2210
2211 admin.compactRegion(controller, request);
2212 } catch (ServiceException se) {
2213 throw ProtobufUtil.getRemoteException(se);
2214 }
2215 }
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230 @Override
2231 public void move(final byte [] encodedRegionName, final byte [] destServerName)
2232 throws IOException {
2233
2234 executeCallable(new MasterCallable<Void>(getConnection()) {
2235 @Override
2236 public Void call(int callTimeout) throws ServiceException {
2237 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2238 controller.setCallTimeout(callTimeout);
2239
2240 if (isMetaRegion(encodedRegionName)) {
2241 controller.setPriority(TableName.META_TABLE_NAME);
2242 }
2243
2244 try {
2245 MoveRegionRequest request =
2246 RequestConverter.buildMoveRegionRequest(encodedRegionName, destServerName);
2247 master.moveRegion(controller, request);
2248 } catch (DeserializationException de) {
2249 LOG.error("Could not parse destination server name: " + de);
2250 throw new ServiceException(new DoNotRetryIOException(de));
2251 }
2252 return null;
2253 }
2254 });
2255 }
2256
2257 private boolean isMetaRegion(final byte[] regionName) {
2258 return Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
2259 || Bytes.equals(regionName, HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
2260 }
2261
2262
2263
2264
2265
2266
2267
2268
2269 @Override
2270 public void assign(final byte[] regionName) throws MasterNotRunningException,
2271 ZooKeeperConnectionException, IOException {
2272 final byte[] toBeAssigned = getRegionName(regionName);
2273 executeCallable(new MasterCallable<Void>(getConnection()) {
2274 @Override
2275 public Void call(int callTimeout) throws ServiceException {
2276 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2277 controller.setCallTimeout(callTimeout);
2278
2279 if (isMetaRegion(regionName)) {
2280 controller.setPriority(TableName.META_TABLE_NAME);
2281 }
2282
2283 AssignRegionRequest request =
2284 RequestConverter.buildAssignRegionRequest(toBeAssigned);
2285 master.assignRegion(controller,request);
2286 return null;
2287 }
2288 });
2289 }
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305 @Override
2306 public void unassign(final byte [] regionName, final boolean force)
2307 throws MasterNotRunningException, ZooKeeperConnectionException, IOException {
2308 final byte[] toBeUnassigned = getRegionName(regionName);
2309 executeCallable(new MasterCallable<Void>(getConnection()) {
2310 @Override
2311 public Void call(int callTimeout) throws ServiceException {
2312 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2313 controller.setCallTimeout(callTimeout);
2314
2315 if (isMetaRegion(regionName)) {
2316 controller.setPriority(TableName.META_TABLE_NAME);
2317 }
2318 UnassignRegionRequest request =
2319 RequestConverter.buildUnassignRegionRequest(toBeUnassigned, force);
2320 master.unassignRegion(controller, request);
2321 return null;
2322 }
2323 });
2324 }
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338 @Override
2339 public void offline(final byte [] regionName)
2340 throws IOException {
2341 executeCallable(new MasterCallable<Void>(getConnection()) {
2342 @Override
2343 public Void call(int callTimeout) throws ServiceException {
2344 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2345 controller.setCallTimeout(callTimeout);
2346
2347 if (isMetaRegion(regionName)) {
2348 controller.setPriority(TableName.META_TABLE_NAME);
2349 }
2350 master.offlineRegion(controller, RequestConverter.buildOfflineRegionRequest(regionName));
2351 return null;
2352 }
2353 });
2354 }
2355
2356
2357
2358
2359
2360
2361
2362 @Override
2363 public boolean setBalancerRunning(final boolean on, final boolean synchronous)
2364 throws IOException {
2365 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2366 @Override
2367 public Boolean call(int callTimeout) throws ServiceException {
2368 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2369 controller.setCallTimeout(callTimeout);
2370
2371 SetBalancerRunningRequest req =
2372 RequestConverter.buildSetBalancerRunningRequest(on, synchronous);
2373 return master.setBalancerRunning(controller, req).getPrevBalanceValue();
2374 }
2375 });
2376 }
2377
2378
2379
2380
2381
2382
2383
2384 @Override
2385 public boolean balancer() throws IOException {
2386 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2387 @Override
2388 public Boolean call(int callTimeout) throws ServiceException {
2389 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2390 controller.setCallTimeout(callTimeout);
2391 return master.balance(controller, RequestConverter.buildBalanceRequest(false))
2392 .getBalancerRan();
2393 }
2394 });
2395 }
2396
2397 @Override
2398 public boolean balancer(final boolean force) throws IOException {
2399 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2400 @Override
2401 public Boolean call(int callTimeout) throws ServiceException {
2402 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2403 controller.setCallTimeout(callTimeout);
2404 return master.balance(controller, RequestConverter.buildBalanceRequest(force))
2405 .getBalancerRan();
2406 }
2407 });
2408 }
2409
2410
2411
2412
2413
2414
2415
2416 @Override
2417 public boolean isBalancerEnabled() throws IOException {
2418 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2419 @Override
2420 public Boolean call(int callTimeout) throws ServiceException {
2421 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2422 controller.setCallTimeout(callTimeout);
2423
2424 return master.isBalancerEnabled(controller,
2425 RequestConverter.buildIsBalancerEnabledRequest()).getEnabled();
2426 }
2427 });
2428 }
2429
2430
2431
2432
2433
2434
2435 @Override
2436 public boolean normalize() throws IOException {
2437 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2438 @Override
2439 public Boolean call(int callTimeout) throws ServiceException {
2440 return master.normalize(null,
2441 RequestConverter.buildNormalizeRequest()).getNormalizerRan();
2442 }
2443 });
2444 }
2445
2446
2447
2448
2449
2450
2451 public boolean isNormalizerEnabled() throws IOException {
2452 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2453 @Override
2454 public Boolean call(int callTimeout) throws ServiceException {
2455 return master.isNormalizerEnabled(null,
2456 RequestConverter.buildIsNormalizerEnabledRequest()).getEnabled();
2457 }
2458 });
2459 }
2460
2461
2462
2463
2464
2465
2466 public boolean setNormalizerRunning(final boolean on) throws IOException {
2467 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2468 @Override
2469 public Boolean call(int callTimeout) throws ServiceException {
2470 SetNormalizerRunningRequest req =
2471 RequestConverter.buildSetNormalizerRunningRequest(on);
2472 return master.setNormalizerRunning(null, req).getPrevNormalizerValue();
2473 }
2474 });
2475 }
2476
2477
2478
2479
2480
2481
2482
2483 @Override
2484 public boolean enableCatalogJanitor(final boolean enable)
2485 throws IOException {
2486 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2487 @Override
2488 public Boolean call(int callTimeout) throws ServiceException {
2489 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2490 controller.setCallTimeout(callTimeout);
2491
2492 return master.enableCatalogJanitor(controller,
2493 RequestConverter.buildEnableCatalogJanitorRequest(enable)).getPrevValue();
2494 }
2495 });
2496 }
2497
2498
2499
2500
2501
2502
2503 @Override
2504 public int runCatalogScan() throws IOException {
2505 return executeCallable(new MasterCallable<Integer>(getConnection()) {
2506 @Override
2507 public Integer call(int callTimeout) throws ServiceException {
2508 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2509 controller.setCallTimeout(callTimeout);
2510
2511 return master.runCatalogScan(controller,
2512 RequestConverter.buildCatalogScanRequest()).getScanResult();
2513 }
2514 });
2515 }
2516
2517
2518
2519
2520
2521 @Override
2522 public boolean isCatalogJanitorEnabled() throws IOException {
2523 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
2524 @Override
2525 public Boolean call(int callTimeout) throws ServiceException {
2526 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2527 controller.setCallTimeout(callTimeout);
2528
2529 return master.isCatalogJanitorEnabled(controller,
2530 RequestConverter.buildIsCatalogJanitorEnabledRequest()).getValue();
2531 }
2532 });
2533 }
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543 @Override
2544 public void mergeRegions(final byte[] encodedNameOfRegionA,
2545 final byte[] encodedNameOfRegionB, final boolean forcible)
2546 throws IOException {
2547 Pair<HRegionInfo, ServerName> pair = getRegion(encodedNameOfRegionA);
2548 if (pair != null && pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
2549 throw new IllegalArgumentException("Can't invoke merge on non-default regions directly");
2550 pair = getRegion(encodedNameOfRegionB);
2551 if (pair != null && pair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID)
2552 throw new IllegalArgumentException("Can't invoke merge on non-default regions directly");
2553 executeCallable(new MasterCallable<Void>(getConnection()) {
2554 @Override
2555 public Void call(int callTimeout) throws ServiceException {
2556 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2557 controller.setCallTimeout(callTimeout);
2558
2559 try {
2560 DispatchMergingRegionsRequest request = RequestConverter
2561 .buildDispatchMergingRegionsRequest(encodedNameOfRegionA,
2562 encodedNameOfRegionB, forcible);
2563 master.dispatchMergingRegions(controller, request);
2564 } catch (DeserializationException de) {
2565 LOG.error("Could not parse destination server name: " + de);
2566 }
2567 return null;
2568 }
2569 });
2570 }
2571
2572
2573
2574
2575 @Override
2576 public void split(final TableName tableName)
2577 throws IOException {
2578 split(tableName, null);
2579 }
2580
2581
2582
2583
2584 @Override
2585 public void splitRegion(final byte[] regionName)
2586 throws IOException {
2587 splitRegion(regionName, null);
2588 }
2589
2590
2591
2592
2593
2594 @Deprecated
2595 public void split(final String tableNameOrRegionName)
2596 throws IOException, InterruptedException {
2597 split(Bytes.toBytes(tableNameOrRegionName));
2598 }
2599
2600
2601
2602
2603
2604 @Deprecated
2605 public void split(final byte[] tableNameOrRegionName)
2606 throws IOException, InterruptedException {
2607 split(tableNameOrRegionName, null);
2608 }
2609
2610
2611
2612
2613 @Override
2614 public void split(final TableName tableName, final byte [] splitPoint)
2615 throws IOException {
2616 ZooKeeperWatcher zookeeper = null;
2617 try {
2618 checkTableExists(tableName);
2619 zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
2620 new ThrowableAbortable());
2621 List<Pair<HRegionInfo, ServerName>> pairs =
2622 MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection, tableName);
2623 for (Pair<HRegionInfo, ServerName> pair: pairs) {
2624
2625 if (pair.getSecond() == null) continue;
2626 HRegionInfo r = pair.getFirst();
2627
2628 if (r.isSplitParent()) continue;
2629
2630 if (r.getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID ||
2631 (splitPoint != null && !r.containsRow(splitPoint))) continue;
2632
2633 split(pair.getSecond(), pair.getFirst(), splitPoint);
2634 }
2635 } finally {
2636 if (zookeeper != null) {
2637 zookeeper.close();
2638 }
2639 }
2640 }
2641
2642
2643
2644
2645 @Override
2646 public void splitRegion(final byte[] regionName, final byte [] splitPoint)
2647 throws IOException {
2648 Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
2649 if (regionServerPair == null) {
2650 throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
2651 }
2652 if (regionServerPair.getFirst() != null &&
2653 regionServerPair.getFirst().getReplicaId() != HRegionInfo.DEFAULT_REPLICA_ID) {
2654 throw new IllegalArgumentException("Can't split replicas directly. "
2655 + "Replicas are auto-split when their primary is split.");
2656 }
2657 if (regionServerPair.getSecond() == null) {
2658 throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
2659 }
2660 split(regionServerPair.getSecond(), regionServerPair.getFirst(), splitPoint);
2661 }
2662
2663
2664
2665
2666
2667 @Deprecated
2668 public void split(final String tableNameOrRegionName,
2669 final String splitPoint) throws IOException {
2670 split(Bytes.toBytes(tableNameOrRegionName), Bytes.toBytes(splitPoint));
2671 }
2672
2673
2674
2675
2676
2677 @Deprecated
2678 public void split(final byte[] tableNameOrRegionName,
2679 final byte [] splitPoint) throws IOException {
2680 try {
2681 splitRegion(tableNameOrRegionName, splitPoint);
2682 } catch (IllegalArgumentException e) {
2683
2684 split(TableName.valueOf(tableNameOrRegionName), splitPoint);
2685 }
2686 }
2687
2688 @VisibleForTesting
2689 public void split(final ServerName sn, final HRegionInfo hri,
2690 byte[] splitPoint) throws IOException {
2691 if (hri.getStartKey() != null && splitPoint != null &&
2692 Bytes.compareTo(hri.getStartKey(), splitPoint) == 0) {
2693 throw new IOException("should not give a splitkey which equals to startkey!");
2694 }
2695 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2696 controller.setPriority(hri.getTable());
2697
2698
2699 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
2700 ProtobufUtil.split(controller, admin, hri, splitPoint);
2701 }
2702
2703
2704 Future<String> backupTablesAsync(final BackupRequest userRequest) throws IOException {
2705 BackupClientUtil.checkTargetDir(userRequest.getTargetRootDir(), conf);
2706 if (userRequest.getTableList() != null) {
2707 for (TableName table : userRequest.getTableList()) {
2708 if (!tableExists(table)) {
2709 throw new DoNotRetryIOException(table + "does not exist");
2710 }
2711 }
2712 }
2713 BackupTablesResponse response = executeCallable(
2714 new MasterCallable<BackupTablesResponse>(getConnection()) {
2715 @Override
2716 public BackupTablesResponse call(int callTimeout) throws ServiceException {
2717 BackupTablesRequest request = RequestConverter.buildBackupTablesRequest(
2718 userRequest.getBackupType(), userRequest.getTableList(), userRequest.getTargetRootDir(),
2719 userRequest.getWorkers(), userRequest.getBandwidth());
2720 return master.backupTables(null, request);
2721 }
2722 }, backupWaitTimeout);
2723 return new TableBackupFuture(this, TableName.BACKUP_TABLE_NAME, response);
2724 }
2725
2726
2727
2728
2729 private static <T> T get(final Future<T> future, final long timeout, final TimeUnit units)
2730 throws IOException {
2731 try {
2732
2733 return future.get(timeout, units);
2734 } catch (InterruptedException e) {
2735 throw new InterruptedIOException("Interrupt while waiting on " + future);
2736 } catch (TimeoutException e) {
2737 throw new TimeoutIOException(e);
2738 } catch (ExecutionException e) {
2739 if (e.getCause() instanceof IOException) {
2740 throw (IOException)e.getCause();
2741 } else {
2742 throw new IOException(e.getCause());
2743 }
2744 }
2745 }
2746
2747
2748 public String backupTables(final BackupRequest userRequest) throws IOException {
2749 return get(
2750 backupTablesAsync(userRequest),
2751 backupWaitTimeout,
2752 TimeUnit.SECONDS);
2753 }
2754
2755 public static class TableBackupFuture extends TableFuture<String> {
2756 String backupId;
2757 public TableBackupFuture(final HBaseAdmin admin, final TableName tableName,
2758 final BackupTablesResponse response) {
2759 super(admin, tableName,
2760 (response != null && response.hasProcId()) ? response.getProcId() : null);
2761 backupId = response.getBackupId();
2762 }
2763
2764 String getBackupId() {
2765 return backupId;
2766 }
2767
2768 @Override
2769 public String getOperationType() {
2770 return "BACKUP";
2771 }
2772
2773 @Override
2774 protected String convertResult(final GetProcedureResultResponse response) throws IOException {
2775 if (response.hasException()) {
2776 throw ForeignExceptionUtil.toIOException(response.getException());
2777 }
2778 ByteString result = response.getResult();
2779 if (result == null) return null;
2780 return Bytes.toStringBinary(result.toByteArray());
2781 }
2782
2783 @Override
2784 protected String postOperationResult(final String result,
2785 final long deadlineTs) throws IOException, TimeoutException {
2786 return result;
2787 }
2788 }
2789
2790
2791
2792
2793
2794
2795
2796
2797
2798
2799 @Override
2800 public void modifyTable(final TableName tableName, final HTableDescriptor htd)
2801 throws IOException {
2802 if (!tableName.equals(htd.getTableName())) {
2803 throw new IllegalArgumentException("the specified table name '" + tableName +
2804 "' doesn't match with the HTD one: " + htd.getTableName());
2805 }
2806
2807 executeCallable(new MasterCallable<Void>(getConnection()) {
2808 @Override
2809 public Void call(int callTimeout) throws ServiceException {
2810 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2811 controller.setCallTimeout(callTimeout);
2812 controller.setPriority(tableName);
2813 ModifyTableRequest request = RequestConverter.buildModifyTableRequest(tableName, htd);
2814 master.modifyTable(controller, request);
2815 return null;
2816 }
2817 });
2818 }
2819
2820 public void modifyTable(final byte[] tableName, final HTableDescriptor htd)
2821 throws IOException {
2822 modifyTable(TableName.valueOf(tableName), htd);
2823 }
2824
2825 public void modifyTable(final String tableName, final HTableDescriptor htd)
2826 throws IOException {
2827 modifyTable(TableName.valueOf(tableName), htd);
2828 }
2829
2830
2831
2832
2833
2834
2835
2836
2837
2838
2839 Pair<HRegionInfo, ServerName> getRegion(final byte[] regionName) throws IOException {
2840 if (regionName == null) {
2841 throw new IllegalArgumentException("Pass a table name or region name");
2842 }
2843 Pair<HRegionInfo, ServerName> pair =
2844 MetaTableAccessor.getRegion(connection, regionName);
2845 if (pair == null) {
2846 final AtomicReference<Pair<HRegionInfo, ServerName>> result =
2847 new AtomicReference<Pair<HRegionInfo, ServerName>>(null);
2848 final String encodedName = Bytes.toString(regionName);
2849 MetaScannerVisitor visitor = new MetaScannerVisitorBase() {
2850 @Override
2851 public boolean processRow(Result data) throws IOException {
2852 HRegionInfo info = HRegionInfo.getHRegionInfo(data);
2853 if (info == null) {
2854 LOG.warn("No serialized HRegionInfo in " + data);
2855 return true;
2856 }
2857 RegionLocations rl = MetaTableAccessor.getRegionLocations(data);
2858 boolean matched = false;
2859 ServerName sn = null;
2860 for (HRegionLocation h : rl.getRegionLocations()) {
2861 if (h != null && encodedName.equals(h.getRegionInfo().getEncodedName())) {
2862 sn = h.getServerName();
2863 info = h.getRegionInfo();
2864 matched = true;
2865 }
2866 }
2867 if (!matched) return true;
2868 result.set(new Pair<HRegionInfo, ServerName>(info, sn));
2869 return false;
2870 }
2871 };
2872
2873 MetaScanner.metaScan(connection, visitor, null);
2874 pair = result.get();
2875 }
2876 return pair;
2877 }
2878
2879
2880
2881
2882
2883
2884
2885
2886
2887 private byte[] getRegionName(
2888 final byte[] regionNameOrEncodedRegionName) throws IOException {
2889 if (Bytes.equals(regionNameOrEncodedRegionName,
2890 HRegionInfo.FIRST_META_REGIONINFO.getRegionName())
2891 || Bytes.equals(regionNameOrEncodedRegionName,
2892 HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
2893 return HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
2894 }
2895 byte[] tmp = regionNameOrEncodedRegionName;
2896 Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionNameOrEncodedRegionName);
2897 if (regionServerPair != null && regionServerPair.getFirst() != null) {
2898 tmp = regionServerPair.getFirst().getRegionName();
2899 }
2900 return tmp;
2901 }
2902
2903
2904
2905
2906
2907
2908
2909
2910 private TableName checkTableExists(final TableName tableName)
2911 throws IOException {
2912 if (!MetaTableAccessor.tableExists(connection, tableName)) {
2913 throw new TableNotFoundException(tableName);
2914 }
2915 return tableName;
2916 }
2917
2918
2919
2920
2921
2922 @Override
2923 public synchronized void shutdown() throws IOException {
2924 executeCallable(new MasterCallable<Void>(getConnection()) {
2925 @Override
2926 public Void call(int callTimeout) throws ServiceException {
2927 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2928 controller.setCallTimeout(callTimeout);
2929 controller.setPriority(HConstants.HIGH_QOS);
2930 master.shutdown(controller, ShutdownRequest.newBuilder().build());
2931 return null;
2932 }
2933 });
2934 }
2935
2936
2937
2938
2939
2940
2941
2942 @Override
2943 public synchronized void stopMaster() throws IOException {
2944 executeCallable(new MasterCallable<Void>(getConnection()) {
2945 @Override
2946 public Void call(int callTimeout) throws ServiceException {
2947 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2948 controller.setCallTimeout(callTimeout);
2949 controller.setPriority(HConstants.HIGH_QOS);
2950 master.stopMaster(controller, StopMasterRequest.newBuilder().build());
2951 return null;
2952 }
2953 });
2954 }
2955
2956
2957
2958
2959
2960
2961
2962 @Override
2963 public synchronized void stopRegionServer(final String hostnamePort)
2964 throws IOException {
2965 String hostname = Addressing.parseHostname(hostnamePort);
2966 int port = Addressing.parsePort(hostnamePort);
2967 AdminService.BlockingInterface admin =
2968 this.connection.getAdmin(ServerName.valueOf(hostname, port, 0));
2969 StopServerRequest request = RequestConverter.buildStopServerRequest(
2970 "Called by admin client " + this.connection.toString());
2971 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2972
2973 controller.setPriority(HConstants.HIGH_QOS);
2974 try {
2975
2976 admin.stopServer(controller, request);
2977 } catch (ServiceException se) {
2978 throw ProtobufUtil.getRemoteException(se);
2979 }
2980 }
2981
2982
2983
2984
2985
2986
2987 @Override
2988 public ClusterStatus getClusterStatus() throws IOException {
2989 return executeCallable(new MasterCallable<ClusterStatus>(getConnection()) {
2990 @Override
2991 public ClusterStatus call(int callTimeout) throws ServiceException {
2992 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
2993 controller.setCallTimeout(callTimeout);
2994 GetClusterStatusRequest req = RequestConverter.buildGetClusterStatusRequest();
2995 return ClusterStatus.convert(master.getClusterStatus(controller, req).getClusterStatus());
2996 }
2997 });
2998 }
2999
3000
3001
3002
3003 @Override
3004 public Configuration getConfiguration() {
3005 return this.conf;
3006 }
3007
3008
3009
3010
3011
3012
3013 @Override
3014 public void createNamespace(final NamespaceDescriptor descriptor) throws IOException {
3015 executeCallable(new MasterCallable<Void>(getConnection()) {
3016 @Override
3017 public Void call(int callTimeout) throws Exception {
3018 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3019 controller.setCallTimeout(callTimeout);
3020
3021 master.createNamespace(controller,
3022 CreateNamespaceRequest.newBuilder()
3023 .setNamespaceDescriptor(ProtobufUtil
3024 .toProtoNamespaceDescriptor(descriptor)).build()
3025 );
3026 return null;
3027 }
3028 });
3029 }
3030
3031
3032
3033
3034
3035
3036 @Override
3037 public void modifyNamespace(final NamespaceDescriptor descriptor) throws IOException {
3038 executeCallable(new MasterCallable<Void>(getConnection()) {
3039 @Override
3040 public Void call(int callTimeout) throws Exception {
3041 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3042 controller.setCallTimeout(callTimeout);
3043 master.modifyNamespace(controller, ModifyNamespaceRequest.newBuilder().
3044 setNamespaceDescriptor(ProtobufUtil.toProtoNamespaceDescriptor(descriptor)).build());
3045 return null;
3046 }
3047 });
3048 }
3049
3050
3051
3052
3053
3054
3055 @Override
3056 public void deleteNamespace(final String name) throws IOException {
3057 executeCallable(new MasterCallable<Void>(getConnection()) {
3058 @Override
3059 public Void call(int callTimeout) throws Exception {
3060 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3061 controller.setCallTimeout(callTimeout);
3062 master.deleteNamespace(controller, DeleteNamespaceRequest.newBuilder().
3063 setNamespaceName(name).build());
3064 return null;
3065 }
3066 });
3067 }
3068
3069
3070
3071
3072
3073
3074
3075 @Override
3076 public NamespaceDescriptor getNamespaceDescriptor(final String name) throws IOException {
3077 return
3078 executeCallable(new MasterCallable<NamespaceDescriptor>(getConnection()) {
3079 @Override
3080 public NamespaceDescriptor call(int callTimeout) throws Exception {
3081 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3082 controller.setCallTimeout(callTimeout);
3083 return ProtobufUtil.toNamespaceDescriptor(
3084 master.getNamespaceDescriptor(controller, GetNamespaceDescriptorRequest.newBuilder().
3085 setNamespaceName(name).build()).getNamespaceDescriptor());
3086 }
3087 });
3088 }
3089
3090
3091
3092
3093
3094
3095 @Override
3096 public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
3097 return
3098 executeCallable(new MasterCallable<NamespaceDescriptor[]>(getConnection()) {
3099 @Override
3100 public NamespaceDescriptor[] call(int callTimeout) throws Exception {
3101 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3102 controller.setCallTimeout(callTimeout);
3103 List<HBaseProtos.NamespaceDescriptor> list =
3104 master.listNamespaceDescriptors(controller,
3105 ListNamespaceDescriptorsRequest.newBuilder().build())
3106 .getNamespaceDescriptorList();
3107 NamespaceDescriptor[] res = new NamespaceDescriptor[list.size()];
3108 for(int i = 0; i < list.size(); i++) {
3109 res[i] = ProtobufUtil.toNamespaceDescriptor(list.get(i));
3110 }
3111 return res;
3112 }
3113 });
3114 }
3115
3116
3117
3118
3119
3120
3121 @Override
3122 public ProcedureInfo[] listProcedures() throws IOException {
3123 return
3124 executeCallable(new MasterCallable<ProcedureInfo[]>(getConnection()) {
3125 @Override
3126 public ProcedureInfo[] call(int callTimeout) throws Exception {
3127 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3128 controller.setCallTimeout(callTimeout);
3129 List<ProcedureProtos.Procedure> procList = master.listProcedures(
3130 controller, ListProceduresRequest.newBuilder().build()).getProcedureList();
3131 ProcedureInfo[] procInfoList = new ProcedureInfo[procList.size()];
3132 for (int i = 0; i < procList.size(); i++) {
3133 procInfoList[i] = ProcedureInfo.convert(procList.get(i));
3134 }
3135 return procInfoList;
3136 }
3137 });
3138 }
3139
3140
3141
3142
3143
3144
3145
3146 @Override
3147 public HTableDescriptor[] listTableDescriptorsByNamespace(final String name) throws IOException {
3148 return
3149 executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
3150 @Override
3151 public HTableDescriptor[] call(int callTimeout) throws Exception {
3152 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3153 controller.setCallTimeout(callTimeout);
3154 List<TableSchema> list =
3155 master.listTableDescriptorsByNamespace(controller,
3156 ListTableDescriptorsByNamespaceRequest.newBuilder().setNamespaceName(name)
3157 .build()).getTableSchemaList();
3158 HTableDescriptor[] res = new HTableDescriptor[list.size()];
3159 for(int i=0; i < list.size(); i++) {
3160
3161 res[i] = HTableDescriptor.convert(list.get(i));
3162 }
3163 return res;
3164 }
3165 });
3166 }
3167
3168
3169
3170
3171
3172
3173
3174 @Override
3175 public TableName[] listTableNamesByNamespace(final String name) throws IOException {
3176 return
3177 executeCallable(new MasterCallable<TableName[]>(getConnection()) {
3178 @Override
3179 public TableName[] call(int callTimeout) throws Exception {
3180 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3181 controller.setCallTimeout(callTimeout);
3182 List<HBaseProtos.TableName> tableNames =
3183 master.listTableNamesByNamespace(controller, ListTableNamesByNamespaceRequest.
3184 newBuilder().setNamespaceName(name).build())
3185 .getTableNameList();
3186 TableName[] result = new TableName[tableNames.size()];
3187 for (int i = 0; i < tableNames.size(); i++) {
3188 result[i] = ProtobufUtil.toTableName(tableNames.get(i));
3189 }
3190 return result;
3191 }
3192 });
3193 }
3194
3195
3196
3197
3198
3199
3200
3201
3202 public static void checkHBaseAvailable(Configuration conf)
3203 throws MasterNotRunningException, ZooKeeperConnectionException, ServiceException, IOException {
3204 Configuration copyOfConf = HBaseConfiguration.create(conf);
3205
3206 copyOfConf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
3207 copyOfConf.setInt("zookeeper.recovery.retry", 0);
3208 try (ClusterConnection connection =
3209 (ClusterConnection)ConnectionFactory.createConnection(copyOfConf)) {
3210
3211
3212 ZooKeeperKeepAliveConnection zkw = null;
3213 try {
3214
3215 zkw = ((ConnectionManager.HConnectionImplementation)connection).
3216 getKeepAliveZooKeeperWatcher();
3217 zkw.getRecoverableZooKeeper().getZooKeeper().exists(zkw.baseZNode, false);
3218 } catch (IOException e) {
3219 throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
3220 } catch (InterruptedException e) {
3221 throw (InterruptedIOException)
3222 new InterruptedIOException("Can't connect to ZooKeeper").initCause(e);
3223 } catch (KeeperException e) {
3224 throw new ZooKeeperConnectionException("Can't connect to ZooKeeper", e);
3225 } finally {
3226 if (zkw != null) {
3227 zkw.close();
3228 }
3229 }
3230 connection.isMasterRunning();
3231 }
3232 }
3233
3234
3235
3236
3237
3238
3239
3240
3241 @Override
3242 public List<HRegionInfo> getTableRegions(final TableName tableName)
3243 throws IOException {
3244 ZooKeeperWatcher zookeeper =
3245 new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
3246 new ThrowableAbortable());
3247 List<HRegionInfo> Regions = null;
3248 try {
3249 Regions = MetaTableAccessor.getTableRegions(zookeeper, connection, tableName, true);
3250 } finally {
3251 zookeeper.close();
3252 }
3253 return Regions;
3254 }
3255
3256 public List<HRegionInfo> getTableRegions(final byte[] tableName)
3257 throws IOException {
3258 return getTableRegions(TableName.valueOf(tableName));
3259 }
3260
3261 @Override
3262 public synchronized void close() throws IOException {
3263 if (cleanupConnectionOnClose && this.connection != null && !this.closed) {
3264 this.connection.close();
3265 this.closed = true;
3266 }
3267 }
3268
3269
3270
3271
3272
3273
3274
3275 @Override
3276 public HTableDescriptor[] getTableDescriptorsByTableName(final List<TableName> tableNames)
3277 throws IOException {
3278 return executeCallable(new MasterCallable<HTableDescriptor[]>(getConnection()) {
3279 @Override
3280 public HTableDescriptor[] call(int callTimeout) throws Exception {
3281 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3282 controller.setCallTimeout(callTimeout);
3283 GetTableDescriptorsRequest req =
3284 RequestConverter.buildGetTableDescriptorsRequest(tableNames);
3285 return ProtobufUtil.getHTableDescriptorArray(master.getTableDescriptors(controller, req));
3286 }
3287 });
3288 }
3289
3290
3291
3292
3293
3294
3295
3296 private HTableDescriptor getTableDescriptorByTableName(TableName tableName)
3297 throws IOException {
3298 List<TableName> tableNames = new ArrayList<TableName>(1);
3299 tableNames.add(tableName);
3300
3301 HTableDescriptor[] htdl = getTableDescriptorsByTableName(tableNames);
3302
3303 if (htdl == null || htdl.length == 0) {
3304 return null;
3305 }
3306 else {
3307 return htdl[0];
3308 }
3309 }
3310
3311
3312
3313
3314
3315
3316
3317 @Override
3318 public HTableDescriptor[] getTableDescriptors(List<String> names)
3319 throws IOException {
3320 List<TableName> tableNames = new ArrayList<TableName>(names.size());
3321 for(String name : names) {
3322 tableNames.add(TableName.valueOf(name));
3323 }
3324 return getTableDescriptorsByTableName(tableNames);
3325 }
3326
3327 private RollWALWriterResponse rollWALWriterImpl(final ServerName sn) throws IOException,
3328 FailedLogCloseException {
3329 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
3330 RollWALWriterRequest request = RequestConverter.buildRollWALWriterRequest();
3331 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3332
3333 try {
3334
3335 return admin.rollWALWriter(controller, request);
3336 } catch (ServiceException se) {
3337 throw ProtobufUtil.getRemoteException(se);
3338 }
3339 }
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355
3356
3357
3358
3359
3360
3361
3362
3363 @Deprecated
3364 public synchronized byte[][] rollHLogWriter(String serverName)
3365 throws IOException, FailedLogCloseException {
3366 ServerName sn = ServerName.valueOf(serverName);
3367 final RollWALWriterResponse response = rollWALWriterImpl(sn);
3368 int regionCount = response.getRegionToFlushCount();
3369 if (0 == regionCount) {
3370 return null;
3371 }
3372 byte[][] regionsToFlush = new byte[regionCount][];
3373 for (int i = 0; i < regionCount; i++) {
3374 ByteString region = response.getRegionToFlush(i);
3375 regionsToFlush[i] = region.toByteArray();
3376 }
3377 return regionsToFlush;
3378 }
3379
3380 @Override
3381 public synchronized void rollWALWriter(ServerName serverName)
3382 throws IOException, FailedLogCloseException {
3383 rollWALWriterImpl(serverName);
3384 }
3385
3386 @Override
3387 public String[] getMasterCoprocessors() {
3388 try {
3389 return getClusterStatus().getMasterCoprocessors();
3390 } catch (IOException e) {
3391 LOG.error("Could not getClusterStatus()",e);
3392 return null;
3393 }
3394 }
3395
3396
3397
3398
3399 @Override
3400 public CompactionState getCompactionState(final TableName tableName, CompactType compactType)
3401 throws IOException {
3402 CompactionState state = CompactionState.NONE;
3403 checkTableExists(tableName);
3404 switch (compactType) {
3405 case MOB:
3406 try {
3407 ServerName master = getMasterAddress();
3408 HRegionInfo info = getMobRegionInfo(tableName);
3409 GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
3410 info.getRegionName(), true);
3411 GetRegionInfoResponse response = this.connection.getAdmin(master)
3412 .getRegionInfo(null, request);
3413 state = response.getCompactionState();
3414 } catch (ServiceException se) {
3415 throw ProtobufUtil.getRemoteException(se);
3416 }
3417 break;
3418 case NORMAL:
3419 ZooKeeperWatcher zookeeper =
3420 new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(),
3421 new ThrowableAbortable());
3422 try {
3423 checkTableExists(tableName);
3424 List<Pair<HRegionInfo, ServerName>> pairs =
3425 MetaTableAccessor.getTableRegionsAndLocations(zookeeper, connection, tableName);
3426 for (Pair<HRegionInfo, ServerName> pair: pairs) {
3427 if (pair.getFirst().isOffline()) continue;
3428 if (pair.getSecond() == null) continue;
3429 try {
3430 ServerName sn = pair.getSecond();
3431 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
3432 GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
3433 pair.getFirst().getRegionName(), true);
3434 GetRegionInfoResponse response = admin.getRegionInfo(null, request);
3435 switch (response.getCompactionState()) {
3436 case MAJOR_AND_MINOR:
3437 return CompactionState.MAJOR_AND_MINOR;
3438 case MAJOR:
3439 if (state == CompactionState.MINOR) {
3440 return CompactionState.MAJOR_AND_MINOR;
3441 }
3442 state = CompactionState.MAJOR;
3443 break;
3444 case MINOR:
3445 if (state == CompactionState.MAJOR) {
3446 return CompactionState.MAJOR_AND_MINOR;
3447 }
3448 state = CompactionState.MINOR;
3449 break;
3450 case NONE:
3451 default:
3452 }
3453 } catch (NotServingRegionException e) {
3454 if (LOG.isDebugEnabled()) {
3455 LOG.debug("Trying to get compaction state of " +
3456 pair.getFirst() + ": " +
3457 StringUtils.stringifyException(e));
3458 }
3459 } catch (RemoteException e) {
3460 if (e.getMessage().indexOf(NotServingRegionException.class.getName()) >= 0) {
3461 if (LOG.isDebugEnabled()) {
3462 LOG.debug("Trying to get compaction state of " + pair.getFirst() + ": "
3463 + StringUtils.stringifyException(e));
3464 }
3465 } else {
3466 throw e;
3467 }
3468 }
3469 }
3470 } catch (ServiceException se) {
3471 throw ProtobufUtil.getRemoteException(se);
3472 } finally {
3473 zookeeper.close();
3474 }
3475 break;
3476 }
3477 return state;
3478 }
3479
3480
3481
3482
3483 @Override
3484 public CompactionState getCompactionStateForRegion(final byte[] regionName)
3485 throws IOException {
3486 try {
3487 Pair<HRegionInfo, ServerName> regionServerPair = getRegion(regionName);
3488 if (regionServerPair == null) {
3489 throw new IllegalArgumentException("Invalid region: " + Bytes.toStringBinary(regionName));
3490 }
3491 if (regionServerPair.getSecond() == null) {
3492 throw new NoServerForRegionException(Bytes.toStringBinary(regionName));
3493 }
3494 ServerName sn = regionServerPair.getSecond();
3495 AdminService.BlockingInterface admin = this.connection.getAdmin(sn);
3496 GetRegionInfoRequest request = RequestConverter.buildGetRegionInfoRequest(
3497 regionServerPair.getFirst().getRegionName(), true);
3498 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3499
3500 GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
3501 return response.getCompactionState();
3502 } catch (ServiceException se) {
3503 throw ProtobufUtil.getRemoteException(se);
3504 }
3505 }
3506
3507
3508
3509
3510
3511 @Deprecated
3512 public CompactionState getCompactionState(final String tableNameOrRegionName)
3513 throws IOException, InterruptedException {
3514 return getCompactionState(Bytes.toBytes(tableNameOrRegionName));
3515 }
3516
3517
3518
3519
3520
3521 @Deprecated
3522 public CompactionState getCompactionState(final byte[] tableNameOrRegionName)
3523 throws IOException, InterruptedException {
3524 try {
3525 return getCompactionStateForRegion(tableNameOrRegionName);
3526 } catch (IllegalArgumentException e) {
3527
3528 return getCompactionState(TableName.valueOf(tableNameOrRegionName));
3529 }
3530 }
3531
3532
3533
3534
3535
3536
3537
3538
3539
3540
3541
3542
3543
3544
3545
3546
3547
3548 @Override
3549 public void snapshot(final String snapshotName,
3550 final TableName tableName) throws IOException,
3551 SnapshotCreationException, IllegalArgumentException {
3552 snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH);
3553 }
3554
3555 public void snapshot(final String snapshotName,
3556 final String tableName) throws IOException,
3557 SnapshotCreationException, IllegalArgumentException {
3558 snapshot(snapshotName, TableName.valueOf(tableName),
3559 SnapshotDescription.Type.FLUSH);
3560 }
3561
3562
3563
3564
3565
3566
3567
3568
3569
3570
3571
3572
3573
3574
3575
3576
3577 public void snapshot(final byte[] snapshotName, final byte[] tableName,
3578 final SnapshotDescription.Type flushType) throws
3579 IOException, SnapshotCreationException, IllegalArgumentException {
3580 snapshot(Bytes.toString(snapshotName), Bytes.toString(tableName), flushType);
3581 }
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595
3596
3597
3598 @Override
3599 public void snapshot(final byte[] snapshotName,
3600 final TableName tableName) throws IOException,
3601 SnapshotCreationException, IllegalArgumentException {
3602 snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH);
3603 }
3604
3605 public void snapshot(final byte[] snapshotName,
3606 final byte[] tableName) throws IOException,
3607 SnapshotCreationException, IllegalArgumentException {
3608 snapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName),
3609 SnapshotDescription.Type.FLUSH);
3610 }
3611
3612
3613
3614
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624
3625
3626
3627
3628
3629
3630 @Override
3631 public void snapshot(final String snapshotName,
3632 final TableName tableName,
3633 SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
3634 IllegalArgumentException {
3635 SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
3636 builder.setTable(tableName.getNameAsString());
3637 builder.setName(snapshotName);
3638 builder.setType(type);
3639 snapshot(builder.build());
3640 }
3641
3642 public void snapshot(final String snapshotName,
3643 final String tableName,
3644 SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
3645 IllegalArgumentException {
3646 snapshot(snapshotName, TableName.valueOf(tableName), type);
3647 }
3648
3649 public void snapshot(final String snapshotName,
3650 final byte[] tableName,
3651 SnapshotDescription.Type type) throws IOException, SnapshotCreationException,
3652 IllegalArgumentException {
3653 snapshot(snapshotName, TableName.valueOf(tableName), type);
3654 }
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665
3666
3667
3668
3669
3670
3671
3672
3673
3674
3675
3676
3677 @Override
3678 public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException,
3679 IllegalArgumentException {
3680
3681 SnapshotResponse response = takeSnapshotAsync(snapshot);
3682 waitForSnapshot(snapshot, response.getExpectedTimeout(), getConnection());
3683 }
3684
3685 public void waitForSnapshot(SnapshotDescription snapshot, long max,
3686 HConnection conn) throws IOException {
3687 final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot)
3688 .build();
3689 IsSnapshotDoneResponse done = null;
3690 long start = EnvironmentEdgeManager.currentTime();
3691 long maxPauseTime = max / numRetries;
3692 int tries = 0;
3693 LOG.debug("Waiting a max of " + max + " ms for snapshot '" +
3694 ClientSnapshotDescriptionUtils.toString(snapshot) + "'' to complete. (max " +
3695 maxPauseTime + " ms per retry)");
3696 while (tries == 0
3697 || ((EnvironmentEdgeManager.currentTime() - start) < max && !done.getDone())) {
3698 try {
3699
3700 long sleep = getPauseTime(tries++, pause);
3701 sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
3702 LOG.debug("(#" + tries + ") Sleeping: " + sleep +
3703 "ms while waiting for snapshot completion.");
3704 Thread.sleep(sleep);
3705 } catch (InterruptedException e) {
3706 throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
3707 }
3708 LOG.debug("Getting current status of snapshot from master...");
3709 done = executeCallable(new MasterCallable<IsSnapshotDoneResponse>(conn) {
3710 @Override
3711 public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
3712 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3713 controller.setCallTimeout(callTimeout);
3714 return master.isSnapshotDone(controller, request);
3715 }
3716 });
3717 }
3718 if (!done.getDone()) {
3719 throw new SnapshotCreationException("Snapshot '" + snapshot.getName()
3720 + "' wasn't completed in expectedTime:" + max + " ms", snapshot);
3721 }
3722 }
3723
3724
3725
3726
3727
3728
3729
3730
3731
3732
3733
3734 @Override
3735 public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException,
3736 SnapshotCreationException {
3737 ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
3738 final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
3739 .build();
3740
3741 return executeCallable(new MasterCallable<SnapshotResponse>(getConnection()) {
3742 @Override
3743 public SnapshotResponse call(int callTimeout) throws ServiceException {
3744 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3745 controller.setCallTimeout(callTimeout);
3746 return master.snapshot(controller, request);
3747 }
3748 });
3749 }
3750
3751
3752
3753
3754
3755
3756
3757
3758
3759
3760
3761
3762
3763
3764
3765
3766
3767
3768
3769
3770
3771 @Override
3772 public boolean isSnapshotFinished(final SnapshotDescription snapshot)
3773 throws IOException, HBaseSnapshotException, UnknownSnapshotException {
3774
3775 return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
3776 @Override
3777 public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
3778 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
3779 controller.setCallTimeout(callTimeout);
3780 return master.isSnapshotDone(controller,
3781 IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build());
3782 }
3783 }).getDone();
3784 }
3785
3786
3787
3788
3789
3790
3791
3792
3793
3794
3795
3796
3797
3798
3799 @Override
3800 public void restoreSnapshot(final byte[] snapshotName)
3801 throws IOException, RestoreSnapshotException {
3802 restoreSnapshot(Bytes.toString(snapshotName));
3803 }
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818 @Override
3819 public void restoreSnapshot(final String snapshotName)
3820 throws IOException, RestoreSnapshotException {
3821 boolean takeFailSafeSnapshot =
3822 conf.getBoolean("hbase.snapshot.restore.take.failsafe.snapshot", false);
3823 restoreSnapshot(snapshotName, takeFailSafeSnapshot);
3824 }
3825
3826
3827
3828
3829
3830
3831
3832
3833
3834
3835
3836
3837
3838
3839
3840
3841
3842 @Override
3843 public void restoreSnapshot(final byte[] snapshotName, final boolean takeFailSafeSnapshot)
3844 throws IOException, RestoreSnapshotException {
3845 restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot);
3846 }
3847
3848
3849
3850
3851
3852
3853
3854
3855
3856
3857
3858
3859
3860
3861
3862
3863
3864 @Override
3865 public void restoreSnapshot(final String snapshotName, boolean takeFailSafeSnapshot)
3866 throws IOException, RestoreSnapshotException {
3867 TableName tableName = null;
3868 for (SnapshotDescription snapshotInfo: listSnapshots()) {
3869 if (snapshotInfo.getName().equals(snapshotName)) {
3870 tableName = TableName.valueOf(snapshotInfo.getTable());
3871 break;
3872 }
3873 }
3874
3875 if (tableName == null) {
3876 throw new RestoreSnapshotException(
3877 "Unable to find the table name for snapshot=" + snapshotName);
3878 }
3879
3880
3881 if (!tableExists(tableName)) {
3882 cloneSnapshot(snapshotName, tableName);
3883 return;
3884 }
3885
3886
3887 if (!isTableDisabled(tableName)) {
3888 throw new TableNotDisabledException(tableName);
3889 }
3890
3891
3892 String failSafeSnapshotSnapshotName = null;
3893 if (takeFailSafeSnapshot) {
3894 failSafeSnapshotSnapshotName = conf.get("hbase.snapshot.restore.failsafe.name",
3895 "hbase-failsafe-{snapshot.name}-{restore.timestamp}");
3896 failSafeSnapshotSnapshotName = failSafeSnapshotSnapshotName
3897 .replace("{snapshot.name}", snapshotName)
3898 .replace("{table.name}", tableName.toString().replace(TableName.NAMESPACE_DELIM, '.'))
3899 .replace("{restore.timestamp}", String.valueOf(EnvironmentEdgeManager.currentTime()));
3900 LOG.info("Taking restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
3901 snapshot(failSafeSnapshotSnapshotName, tableName);
3902 }
3903
3904 try {
3905
3906 internalRestoreSnapshot(snapshotName, tableName);
3907 } catch (IOException e) {
3908
3909
3910 if (takeFailSafeSnapshot) {
3911 try {
3912 internalRestoreSnapshot(failSafeSnapshotSnapshotName, tableName);
3913 String msg = "Restore snapshot=" + snapshotName +
3914 " failed. Rollback to snapshot=" + failSafeSnapshotSnapshotName + " succeeded.";
3915 LOG.error(msg, e);
3916 throw new RestoreSnapshotException(msg, e);
3917 } catch (IOException ex) {
3918 String msg = "Failed to restore and rollback to snapshot=" + failSafeSnapshotSnapshotName;
3919 LOG.error(msg, ex);
3920 throw new RestoreSnapshotException(msg, e);
3921 }
3922 } else {
3923 throw new RestoreSnapshotException("Failed to restore snapshot=" + snapshotName, e);
3924 }
3925 }
3926
3927
3928 if (takeFailSafeSnapshot) {
3929 try {
3930 LOG.info("Deleting restore-failsafe snapshot: " + failSafeSnapshotSnapshotName);
3931 deleteSnapshot(failSafeSnapshotSnapshotName);
3932 } catch (IOException e) {
3933 LOG.error("Unable to remove the failsafe snapshot: " + failSafeSnapshotSnapshotName, e);
3934 }
3935 }
3936 }
3937
3938
3939
3940
3941
3942
3943
3944
3945
3946
3947
3948 public void cloneSnapshot(final byte[] snapshotName, final byte[] tableName)
3949 throws IOException, TableExistsException, RestoreSnapshotException {
3950 cloneSnapshot(Bytes.toString(snapshotName), TableName.valueOf(tableName));
3951 }
3952
3953
3954
3955
3956
3957
3958
3959
3960
3961
3962
3963 @Override
3964 public void cloneSnapshot(final byte[] snapshotName, final TableName tableName)
3965 throws IOException, TableExistsException, RestoreSnapshotException {
3966 cloneSnapshot(Bytes.toString(snapshotName), tableName);
3967 }
3968
3969
3970
3971
3972
3973
3974
3975
3976
3977
3978
3979
3980
3981 public void cloneSnapshot(final String snapshotName, final String tableName)
3982 throws IOException, TableExistsException, RestoreSnapshotException, InterruptedException {
3983 cloneSnapshot(snapshotName, TableName.valueOf(tableName));
3984 }
3985
3986
3987
3988
3989
3990
3991
3992
3993
3994
3995
3996 @Override
3997 public void cloneSnapshot(final String snapshotName, final TableName tableName)
3998 throws IOException, TableExistsException, RestoreSnapshotException {
3999 if (tableExists(tableName)) {
4000 throw new TableExistsException(tableName);
4001 }
4002 internalRestoreSnapshot(snapshotName, tableName);
4003 waitUntilTableIsEnabled(tableName);
4004 }
4005
4006
4007
4008
4009
4010
4011
4012
4013
4014
4015
4016
4017 @Override
4018 public byte[] execProcedureWithRet(String signature, String instance,
4019 Map<String, String> props) throws IOException {
4020 ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
4021 builder.setSignature(signature).setInstance(instance);
4022 for (Entry<String, String> entry : props.entrySet()) {
4023 NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
4024 .setValue(entry.getValue()).build();
4025 builder.addConfiguration(pair);
4026 }
4027
4028 final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
4029 .setProcedure(builder.build()).build();
4030
4031 ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
4032 getConnection()) {
4033 @Override
4034 public ExecProcedureResponse call(int callTimeout) throws ServiceException {
4035 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4036 controller.setCallTimeout(callTimeout);
4037 return master.execProcedureWithRet(controller, request);
4038 }
4039 });
4040
4041 return response.hasReturnData() ? response.getReturnData().toByteArray() : null;
4042 }
4043
4044
4045
4046
4047
4048
4049
4050
4051
4052
4053 @Override
4054 public void execProcedure(String signature, String instance,
4055 Map<String, String> props) throws IOException {
4056 ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
4057 builder.setSignature(signature).setInstance(instance);
4058 for (Entry<String, String> entry : props.entrySet()) {
4059 NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
4060 .setValue(entry.getValue()).build();
4061 builder.addConfiguration(pair);
4062 }
4063
4064 final ExecProcedureRequest request = ExecProcedureRequest.newBuilder()
4065 .setProcedure(builder.build()).build();
4066
4067 ExecProcedureResponse response = executeCallable(new MasterCallable<ExecProcedureResponse>(
4068 getConnection()) {
4069 @Override
4070 public ExecProcedureResponse call(int callTimeout) throws ServiceException {
4071 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4072 controller.setCallTimeout(callTimeout);
4073 return master.execProcedure(controller, request);
4074 }
4075 });
4076
4077 long start = EnvironmentEdgeManager.currentTime();
4078 long max = response.getExpectedTimeout();
4079 long maxPauseTime = max / this.numRetries;
4080 int tries = 0;
4081 LOG.debug("Waiting a max of " + max + " ms for procedure '" +
4082 signature + " : " + instance + "'' to complete. (max " + maxPauseTime + " ms per retry)");
4083 boolean done = false;
4084 while (tries == 0
4085 || ((EnvironmentEdgeManager.currentTime() - start) < max && !done)) {
4086 try {
4087
4088 long sleep = getPauseTime(tries++, pause);
4089 sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
4090 LOG.debug("(#" + tries + ") Sleeping: " + sleep +
4091 "ms while waiting for procedure completion.");
4092 Thread.sleep(sleep);
4093 } catch (InterruptedException e) {
4094 throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
4095 }
4096 LOG.debug("Getting current status of procedure from master...");
4097 done = isProcedureFinished(signature, instance, props);
4098 }
4099 if (!done) {
4100 throw new IOException("Procedure '" + signature + " : " + instance
4101 + "' wasn't completed in expectedTime:" + max + " ms");
4102 }
4103 }
4104
4105
4106
4107
4108
4109
4110
4111
4112
4113
4114
4115
4116
4117
4118
4119
4120
4121
4122 @Override
4123 public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
4124 throws IOException {
4125 final ProcedureDescription.Builder builder = ProcedureDescription.newBuilder();
4126 builder.setSignature(signature).setInstance(instance);
4127 for (Entry<String, String> entry : props.entrySet()) {
4128 NameStringPair pair = NameStringPair.newBuilder().setName(entry.getKey())
4129 .setValue(entry.getValue()).build();
4130 builder.addConfiguration(pair);
4131 }
4132 final ProcedureDescription desc = builder.build();
4133 return executeCallable(
4134 new MasterCallable<IsProcedureDoneResponse>(getConnection()) {
4135 @Override
4136 public IsProcedureDoneResponse call(int callTimeout) throws ServiceException {
4137 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4138 controller.setCallTimeout(callTimeout);
4139 return master.isProcedureDone(controller, IsProcedureDoneRequest
4140 .newBuilder().setProcedure(desc).build());
4141 }
4142 }).getDone();
4143 }
4144
4145
4146
4147
4148
4149
4150
4151
4152
4153
4154
4155 private void internalRestoreSnapshot(final String snapshotName, final TableName
4156 tableName)
4157 throws IOException, RestoreSnapshotException {
4158 SnapshotDescription snapshot = SnapshotDescription.newBuilder()
4159 .setName(snapshotName).setTable(tableName.getNameAsString()).build();
4160
4161
4162 internalRestoreSnapshotAsync(snapshot);
4163
4164 final IsRestoreSnapshotDoneRequest request = IsRestoreSnapshotDoneRequest.newBuilder()
4165 .setSnapshot(snapshot).build();
4166 IsRestoreSnapshotDoneResponse done = IsRestoreSnapshotDoneResponse.newBuilder()
4167 .setDone(false).buildPartial();
4168 final long maxPauseTime = 5000;
4169 int tries = 0;
4170 while (!done.getDone()) {
4171 try {
4172
4173 long sleep = getPauseTime(tries++, pause);
4174 sleep = sleep > maxPauseTime ? maxPauseTime : sleep;
4175 LOG.debug(tries + ") Sleeping: " + sleep + " ms while we wait for snapshot restore to complete.");
4176 Thread.sleep(sleep);
4177 } catch (InterruptedException e) {
4178 throw (InterruptedIOException)new InterruptedIOException("Interrupted").initCause(e);
4179 }
4180 LOG.debug("Getting current status of snapshot restore from master...");
4181 done = executeCallable(new MasterCallable<IsRestoreSnapshotDoneResponse>(
4182 getConnection()) {
4183 @Override
4184 public IsRestoreSnapshotDoneResponse call(int callTimeout) throws ServiceException {
4185 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4186 controller.setCallTimeout(callTimeout);
4187 return master.isRestoreSnapshotDone(controller, request);
4188 }
4189 });
4190 }
4191 if (!done.getDone()) {
4192 throw new RestoreSnapshotException("Snapshot '" + snapshot.getName() + "' wasn't restored.");
4193 }
4194 }
4195
4196
4197
4198
4199
4200
4201
4202
4203
4204
4205
4206 private RestoreSnapshotResponse internalRestoreSnapshotAsync(final SnapshotDescription snapshot)
4207 throws IOException, RestoreSnapshotException {
4208 ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
4209
4210 final RestoreSnapshotRequest request = RestoreSnapshotRequest.newBuilder().setSnapshot(snapshot)
4211 .build();
4212
4213
4214 return executeCallable(new MasterCallable<RestoreSnapshotResponse>(getConnection()) {
4215 @Override
4216 public RestoreSnapshotResponse call(int callTimeout) throws ServiceException {
4217 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4218 controller.setCallTimeout(callTimeout);
4219 return master.restoreSnapshot(controller, request);
4220 }
4221 });
4222 }
4223
4224
4225
4226
4227
4228
4229 @Override
4230 public List<SnapshotDescription> listSnapshots() throws IOException {
4231 return executeCallable(new MasterCallable<List<SnapshotDescription>>(getConnection()) {
4232 @Override
4233 public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
4234 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4235 controller.setCallTimeout(callTimeout);
4236 return master.getCompletedSnapshots(controller,
4237 GetCompletedSnapshotsRequest.newBuilder().build()).getSnapshotsList();
4238 }
4239 });
4240 }
4241
4242
4243
4244
4245
4246
4247
4248
4249 @Override
4250 public List<SnapshotDescription> listSnapshots(String regex) throws IOException {
4251 return listSnapshots(Pattern.compile(regex));
4252 }
4253
4254
4255
4256
4257
4258
4259
4260
4261 @Override
4262 public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
4263 List<SnapshotDescription> matched = new LinkedList<SnapshotDescription>();
4264 List<SnapshotDescription> snapshots = listSnapshots();
4265 for (SnapshotDescription snapshot : snapshots) {
4266 if (pattern.matcher(snapshot.getName()).matches()) {
4267 matched.add(snapshot);
4268 }
4269 }
4270 return matched;
4271 }
4272
4273
4274
4275
4276
4277
4278 @Override
4279 public void deleteSnapshot(final byte[] snapshotName) throws IOException {
4280 deleteSnapshot(Bytes.toString(snapshotName));
4281 }
4282
4283
4284
4285
4286
4287
4288 @Override
4289 public void deleteSnapshot(final String snapshotName) throws IOException {
4290
4291 TableName.isLegalFullyQualifiedTableName(Bytes.toBytes(snapshotName));
4292
4293 executeCallable(new MasterCallable<Void>(getConnection()) {
4294 @Override
4295 public Void call(int callTimeout) throws ServiceException {
4296 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4297 controller.setCallTimeout(callTimeout);
4298 master.deleteSnapshot(controller,
4299 DeleteSnapshotRequest.newBuilder().
4300 setSnapshot(SnapshotDescription.newBuilder().setName(snapshotName).build()).build()
4301 );
4302 return null;
4303 }
4304 });
4305 }
4306
4307
4308
4309
4310
4311
4312 @Override
4313 public void deleteSnapshots(final String regex) throws IOException {
4314 deleteSnapshots(Pattern.compile(regex));
4315 }
4316
4317
4318
4319
4320
4321
4322 @Override
4323 public void deleteSnapshots(final Pattern pattern) throws IOException {
4324 List<SnapshotDescription> snapshots = listSnapshots(pattern);
4325 for (final SnapshotDescription snapshot : snapshots) {
4326 try {
4327 internalDeleteSnapshot(snapshot);
4328 } catch (IOException ex) {
4329 LOG.info(
4330 "Failed to delete snapshot " + snapshot.getName() + " for table " + snapshot.getTable(),
4331 ex);
4332 }
4333 }
4334 }
4335
4336 private void internalDeleteSnapshot(final SnapshotDescription snapshot) throws IOException {
4337 executeCallable(new MasterCallable<Void>(getConnection()) {
4338 @Override
4339 public Void call(int callTimeout) throws ServiceException {
4340 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4341 controller.setCallTimeout(callTimeout);
4342 this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder()
4343 .setSnapshot(snapshot).build());
4344 return null;
4345 }
4346 });
4347 }
4348
4349
4350
4351
4352
4353
4354 @Override
4355 public void setQuota(final QuotaSettings quota) throws IOException {
4356 executeCallable(new MasterCallable<Void>(getConnection()) {
4357 @Override
4358 public Void call(int callTimeout) throws ServiceException {
4359 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4360 controller.setCallTimeout(callTimeout);
4361 this.master.setQuota(controller, QuotaSettings.buildSetQuotaRequestProto(quota));
4362 return null;
4363 }
4364 });
4365 }
4366
4367
4368
4369
4370
4371
4372
4373 @Override
4374 public QuotaRetriever getQuotaRetriever(final QuotaFilter filter) throws IOException {
4375 return QuotaRetriever.open(conf, filter);
4376 }
4377
4378 private <V> V executeCallable(MasterCallable<V> callable) throws IOException {
4379 RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
4380 try {
4381 return caller.callWithRetries(callable, operationTimeout);
4382 } finally {
4383 callable.close();
4384 }
4385 }
4386
4387 private <V> V executeCallable(MasterCallable<V> callable, long timeout) throws IOException {
4388 RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
4389 try {
4390 return caller.callWithRetries(callable, (int) timeout);
4391 } finally {
4392 callable.close();
4393 }
4394 }
4395
4396
4397
4398
4399
4400
4401
4402
4403
4404
4405
4406
4407
4408
4409
4410
4411
4412
4413
4414
4415
4416 @Override
4417 public CoprocessorRpcChannel coprocessorService() {
4418 return new MasterCoprocessorRpcChannel(connection);
4419 }
4420
4421
4422
4423
4424 private static class ThrowableAbortable implements Abortable {
4425
4426 @Override
4427 public void abort(String why, Throwable e) {
4428 throw new RuntimeException(why, e);
4429 }
4430
4431 @Override
4432 public boolean isAborted() {
4433 return true;
4434 }
4435 }
4436
4437 private static <C extends RetryingCallable<V> & Closeable, V> V executeCallable(C callable,
4438 RpcRetryingCallerFactory rpcCallerFactory, int operationTimeout) throws IOException {
4439 RpcRetryingCaller<V> caller = rpcCallerFactory.newCaller();
4440 try {
4441 return caller.callWithRetries(callable, operationTimeout);
4442 } finally {
4443 callable.close();
4444 }
4445 }
4446
4447
4448
4449
4450
4451
4452
4453
4454
4455
4456
4457
4458
4459
4460
4461
4462
4463
4464
4465
4466
4467
4468
4469 @Override
4470 public CoprocessorRpcChannel coprocessorService(ServerName sn) {
4471 return new RegionServerCoprocessorRpcChannel(connection, sn);
4472 }
4473
4474 public static long getPauseTime(int tries, long pause) {
4475 int triesCount = tries;
4476 if (triesCount >= HConstants.RETRY_BACKOFF.length) {
4477 triesCount = HConstants.RETRY_BACKOFF.length - 1;
4478 }
4479 return pause * HConstants.RETRY_BACKOFF[triesCount];
4480 }
4481
4482 @Override
4483 public void updateConfiguration(ServerName server) throws IOException {
4484 try {
4485 this.connection.getAdmin(server).updateConfiguration(null,
4486 UpdateConfigurationRequest.getDefaultInstance());
4487 } catch (ServiceException e) {
4488 throw ProtobufUtil.getRemoteException(e);
4489 }
4490 }
4491
4492 @Override
4493 public void updateConfiguration() throws IOException {
4494 for (ServerName server : this.getClusterStatus().getServers()) {
4495 updateConfiguration(server);
4496 }
4497 }
4498
4499 @Override
4500 public int getMasterInfoPort() throws IOException {
4501
4502 ConnectionManager.HConnectionImplementation connection =
4503 (ConnectionManager.HConnectionImplementation)this.connection;
4504 ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
4505 try {
4506 return MasterAddressTracker.getMasterInfoPort(zkw);
4507 } catch (KeeperException e) {
4508 throw new IOException("Failed to get master info port from MasterAddressTracker", e);
4509 }
4510 }
4511
4512 private ServerName getMasterAddress() throws IOException {
4513
4514 HConnectionImplementation connection =
4515 (HConnectionImplementation)this.connection;
4516 ZooKeeperKeepAliveConnection zkw = connection.getKeepAliveZooKeeperWatcher();
4517 try {
4518 return MasterAddressTracker.getMasterAddress(zkw);
4519 } catch (KeeperException e) {
4520 throw new IOException("Failed to get master server name from MasterAddressTracker", e);
4521 }
4522 }
4523
4524 @Override
4525 public long getLastMajorCompactionTimestamp(final TableName tableName) throws IOException {
4526 return executeCallable(new MasterCallable<Long>(getConnection()) {
4527 @Override
4528 public Long call(int callTimeout) throws ServiceException {
4529 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4530 controller.setCallTimeout(callTimeout);
4531 MajorCompactionTimestampRequest req =
4532 MajorCompactionTimestampRequest.newBuilder()
4533 .setTableName(ProtobufUtil.toProtoTableName(tableName)).build();
4534 return master.getLastMajorCompactionTimestamp(controller, req).getCompactionTimestamp();
4535 }
4536 });
4537 }
4538
4539 @Override
4540 public long getLastMajorCompactionTimestampForRegion(final byte[] regionName) throws IOException {
4541 return executeCallable(new MasterCallable<Long>(getConnection()) {
4542 @Override
4543 public Long call(int callTimeout) throws ServiceException {
4544 PayloadCarryingRpcController controller = rpcControllerFactory.newController();
4545 controller.setCallTimeout(callTimeout);
4546 MajorCompactionTimestampForRegionRequest req =
4547 MajorCompactionTimestampForRegionRequest
4548 .newBuilder()
4549 .setRegion(
4550 RequestConverter
4551 .buildRegionSpecifier(RegionSpecifierType.REGION_NAME, regionName)).build();
4552 return master.getLastMajorCompactionTimestampForRegion(controller, req)
4553 .getCompactionTimestamp();
4554 }
4555 });
4556 }
4557
4558
4559
4560
4561 @Override
4562 public void compact(final TableName tableName, final byte[] columnFamily, CompactType compactType)
4563 throws IOException, InterruptedException {
4564 compact(tableName, columnFamily, false, compactType);
4565 }
4566
4567
4568
4569
4570 @Override
4571 public void compact(final TableName tableName, CompactType compactType)
4572 throws IOException, InterruptedException {
4573 compact(tableName, null, false, compactType);
4574 }
4575
4576
4577
4578
4579 @Override
4580 public void majorCompact(final TableName tableName, final byte[] columnFamily,
4581 CompactType compactType) throws IOException, InterruptedException {
4582 compact(tableName, columnFamily, true, compactType);
4583 }
4584
4585
4586
4587
4588 @Override
4589 public void majorCompact(final TableName tableName, CompactType compactType)
4590 throws IOException, InterruptedException {
4591 compact(tableName, null, true, compactType);
4592 }
4593
4594 @Override
4595 public CompactionState getCompactionState(final TableName tableName)
4596 throws IOException {
4597 return getCompactionState(tableName, CompactType.NORMAL);
4598 }
4599
4600
4601
4602
4603
4604
4605 @InterfaceAudience.Private
4606 @InterfaceStability.Evolving
4607 protected static class ProcedureFuture<V> implements Future<V> {
4608 private ExecutionException exception = null;
4609 private boolean procResultFound = false;
4610 private boolean done = false;
4611 private boolean cancelled = false;
4612 private V result = null;
4613
4614 private final HBaseAdmin admin;
4615 private final Long procId;
4616
4617 public ProcedureFuture(final HBaseAdmin admin, final Long procId) {
4618 this.admin = admin;
4619 this.procId = procId;
4620 }
4621
4622 @Override
4623 public boolean cancel(boolean mayInterruptIfRunning) {
4624 AbortProcedureRequest abortProcRequest = AbortProcedureRequest.newBuilder()
4625 .setProcId(procId).setMayInterruptIfRunning(mayInterruptIfRunning).build();
4626 try {
4627 cancelled = abortProcedureResult(abortProcRequest).getIsProcedureAborted();
4628 if (cancelled) {
4629 done = true;
4630 }
4631 } catch (IOException e) {
4632
4633
4634
4635 LOG.warn(
4636 "Cancelling the procedure with procId=" + procId + " throws exception " + e.getMessage(),
4637 e);
4638 cancelled = false;
4639 }
4640 return cancelled;
4641 }
4642
4643 @Override
4644 public boolean isCancelled() {
4645 return cancelled;
4646 }
4647
4648 protected AbortProcedureResponse abortProcedureResult(
4649 final AbortProcedureRequest request) throws IOException {
4650 return admin.executeCallable(new MasterCallable<AbortProcedureResponse>(
4651 admin.getConnection()) {
4652 @Override
4653 public AbortProcedureResponse call(int callTimeout) throws ServiceException {
4654 PayloadCarryingRpcController controller = admin.getRpcControllerFactory().newController();
4655 controller.setCallTimeout(callTimeout);
4656 return master.abortProcedure(controller, request);
4657 }
4658 });
4659 }
4660
4661 @Override
4662 public V get() throws InterruptedException, ExecutionException {
4663
4664 throw new UnsupportedOperationException();
4665 }
4666
4667 @Override
4668 public V get(long timeout, TimeUnit unit)
4669 throws InterruptedException, ExecutionException, TimeoutException {
4670 if (!done) {
4671 long deadlineTs = EnvironmentEdgeManager.currentTime() + unit.toMillis(timeout);
4672 try {
4673 try {
4674
4675 if (procId != null) {
4676 result = waitProcedureResult(procId, deadlineTs);
4677 }
4678
4679 if (!procResultFound) {
4680 result = waitOperationResult(deadlineTs);
4681 }
4682 result = postOperationResult(result, deadlineTs);
4683 done = true;
4684 } catch (IOException e) {
4685 result = postOpeartionFailure(e, deadlineTs);
4686 done = true;
4687 }
4688 } catch (IOException e) {
4689 exception = new ExecutionException(e);
4690 done = true;
4691 }
4692 }
4693 if (exception != null) {
4694 throw exception;
4695 }
4696 return result;
4697 }
4698
4699 @Override
4700 public boolean isDone() {
4701 return done;
4702 }
4703
4704 protected HBaseAdmin getAdmin() {
4705 return admin;
4706 }
4707
4708 private V waitProcedureResult(long procId, long deadlineTs)
4709 throws IOException, TimeoutException, InterruptedException {
4710 GetProcedureResultRequest request = GetProcedureResultRequest.newBuilder()
4711 .setProcId(procId)
4712 .build();
4713
4714 int tries = 0;
4715 IOException serviceEx = null;
4716 while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
4717 GetProcedureResultResponse response = null;
4718 try {
4719
4720 response = getProcedureResult(request);
4721 } catch (IOException e) {
4722 serviceEx = unwrapException(e);
4723
4724
4725 LOG.warn("failed to get the procedure result procId=" + procId, serviceEx);
4726
4727
4728 if (serviceEx instanceof DoNotRetryIOException) {
4729
4730
4731
4732
4733 LOG.warn("Proc-v2 is unsupported on this master: " + serviceEx.getMessage(), serviceEx);
4734 procResultFound = false;
4735 return null;
4736 }
4737 }
4738
4739
4740 if (response != null && response.getState() != GetProcedureResultResponse.State.RUNNING) {
4741 procResultFound = response.getState() != GetProcedureResultResponse.State.NOT_FOUND;
4742 return convertResult(response);
4743 }
4744
4745 try {
4746 Thread.sleep(getPauseTime(tries++, getAdmin().pause));
4747 } catch (InterruptedException e) {
4748 throw new InterruptedException(
4749 "Interrupted while waiting for the result of proc " + procId);
4750 }
4751 }
4752 if (serviceEx != null) {
4753 throw serviceEx;
4754 } else {
4755 throw new TimeoutException("The procedure " + procId + " is still running");
4756 }
4757 }
4758
4759 private static IOException unwrapException(IOException e) {
4760 if (e instanceof RemoteException) {
4761 return ((RemoteException)e).unwrapRemoteException();
4762 }
4763 return e;
4764 }
4765
4766 protected GetProcedureResultResponse getProcedureResult(final GetProcedureResultRequest request)
4767 throws IOException {
4768 return admin.executeCallable(new MasterCallable<GetProcedureResultResponse>(
4769 admin.getConnection()) {
4770 @Override
4771 public GetProcedureResultResponse call(int callTimeout) throws ServiceException {
4772 return master.getProcedureResult(null, request);
4773 }
4774 });
4775 }
4776
4777
4778
4779
4780
4781
4782 protected V convertResult(final GetProcedureResultResponse response) throws IOException {
4783 if (response.hasException()) {
4784 throw ForeignExceptionUtil.toIOException(response.getException());
4785 }
4786 return null;
4787 }
4788
4789
4790
4791
4792
4793
4794
4795 protected V waitOperationResult(final long deadlineTs)
4796 throws IOException, TimeoutException {
4797 return null;
4798 }
4799
4800
4801
4802
4803
4804
4805
4806
4807
4808 protected V postOperationResult(final V result, final long deadlineTs)
4809 throws IOException, TimeoutException {
4810 return result;
4811 }
4812
4813
4814
4815
4816
4817
4818
4819
4820
4821 protected V postOperationFailure(final IOException exception, final long deadlineTs)
4822 throws IOException, TimeoutException {
4823 throw exception;
4824 }
4825
4826
4827
4828
4829
4830
4831
4832
4833
4834
4835 protected V postOpeartionFailure(final IOException exception, final long deadlineTs)
4836 throws IOException, TimeoutException {
4837 throw exception;
4838 }
4839
4840 protected interface WaitForStateCallable {
4841 boolean checkState(int tries) throws IOException;
4842 void throwInterruptedException() throws InterruptedIOException;
4843 void throwTimeoutException(long elapsed) throws TimeoutException;
4844 }
4845
4846 protected void waitForState(final long deadlineTs, final WaitForStateCallable callable)
4847 throws IOException, TimeoutException {
4848 int tries = 0;
4849 IOException serverEx = null;
4850 long startTime = EnvironmentEdgeManager.currentTime();
4851 while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
4852 serverEx = null;
4853 try {
4854 if (callable.checkState(tries)) {
4855 return;
4856 }
4857 } catch (IOException e) {
4858 serverEx = e;
4859 }
4860 try {
4861 Thread.sleep(getPauseTime(tries++, getAdmin().pause));
4862 } catch (InterruptedException e) {
4863 callable.throwInterruptedException();
4864 }
4865 }
4866 if (serverEx != null) {
4867 throw unwrapException(serverEx);
4868 } else {
4869 callable.throwTimeoutException(EnvironmentEdgeManager.currentTime() - startTime);
4870 }
4871 }
4872 }
4873
4874 @InterfaceAudience.Private
4875 @InterfaceStability.Evolving
4876 protected static abstract class TableFuture<V> extends ProcedureFuture<V> {
4877 private final TableName tableName;
4878
4879 public TableFuture(final HBaseAdmin admin, final TableName tableName, final Long procId) {
4880 super(admin, procId);
4881 this.tableName = tableName;
4882 }
4883
4884 @Override
4885 public String toString() {
4886 return getDescription();
4887 }
4888
4889
4890
4891
4892 protected TableName getTableName() {
4893 return tableName;
4894 }
4895
4896
4897
4898
4899 protected HTableDescriptor getTableDescriptor() throws IOException {
4900 return getAdmin().getTableDescriptorByTableName(getTableName());
4901 }
4902
4903
4904
4905 public abstract String getOperationType();
4906
4907
4908
4909
4910 protected String getDescription() {
4911 return "Operation: " + getOperationType() + ", "
4912 + "Table Name: " + tableName.getNameWithNamespaceInclAsString();
4913
4914 };
4915
4916 protected abstract class TableWaitForStateCallable implements WaitForStateCallable {
4917 @Override
4918 public void throwInterruptedException() throws InterruptedIOException {
4919 throw new InterruptedIOException("Interrupted while waiting for operation: "
4920 + getOperationType() + " on table: " + tableName.getNameWithNamespaceInclAsString());
4921 }
4922
4923 @Override
4924 public void throwTimeoutException(long elapsedTime) throws TimeoutException {
4925 throw new TimeoutException("The operation: " + getOperationType() + " on table: " +
4926 tableName.getNameAsString() + " has not completed after " + elapsedTime + "ms");
4927 }
4928 }
4929
4930 @Override
4931 protected V postOperationResult(final V result, final long deadlineTs)
4932 throws IOException, TimeoutException {
4933 LOG.info(getDescription() + " completed");
4934 return super.postOperationResult(result, deadlineTs);
4935 }
4936
4937 @Override
4938 protected V postOperationFailure(final IOException exception, final long deadlineTs)
4939 throws IOException, TimeoutException {
4940 LOG.info(getDescription() + " failed with " + exception.getMessage());
4941 return super.postOperationFailure(exception, deadlineTs);
4942 }
4943
4944 protected void waitForTableEnabled(final long deadlineTs)
4945 throws IOException, TimeoutException {
4946 waitForState(deadlineTs, new TableWaitForStateCallable() {
4947 @Override
4948 public boolean checkState(int tries) throws IOException {
4949 try {
4950 if (getAdmin().isTableAvailable(tableName)) {
4951 return true;
4952 }
4953 } catch (TableNotFoundException tnfe) {
4954 LOG.debug("Table " + tableName.getNameWithNamespaceInclAsString()
4955 + " was not enabled, sleeping. tries=" + tries);
4956 }
4957 return false;
4958 }
4959 });
4960 }
4961
4962 protected void waitForTableDisabled(final long deadlineTs)
4963 throws IOException, TimeoutException {
4964 waitForState(deadlineTs, new TableWaitForStateCallable() {
4965 @Override
4966 public boolean checkState(int tries) throws IOException {
4967 return getAdmin().isTableDisabled(tableName);
4968 }
4969 });
4970 }
4971
4972 protected void waitTableNotFound(final long deadlineTs)
4973 throws IOException, TimeoutException {
4974 waitForState(deadlineTs, new TableWaitForStateCallable() {
4975 @Override
4976 public boolean checkState(int tries) throws IOException {
4977 return !getAdmin().tableExists(tableName);
4978 }
4979 });
4980 }
4981
4982 protected void waitForSchemaUpdate(final long deadlineTs)
4983 throws IOException, TimeoutException {
4984 waitForState(deadlineTs, new TableWaitForStateCallable() {
4985 @Override
4986 public boolean checkState(int tries) throws IOException {
4987 return getAdmin().getAlterStatus(tableName).getFirst() == 0;
4988 }
4989 });
4990 }
4991
4992 protected void waitForAllRegionsOnline(final long deadlineTs, final byte[][] splitKeys)
4993 throws IOException, TimeoutException {
4994 final HTableDescriptor desc = getTableDescriptor();
4995 final AtomicInteger actualRegCount = new AtomicInteger(0);
4996 final MetaTableAccessor.Visitor visitor = new MetaTableAccessor.Visitor() {
4997 @Override
4998 public boolean visit(Result rowResult) throws IOException {
4999 RegionLocations list = MetaTableAccessor.getRegionLocations(rowResult);
5000 if (list == null) {
5001 LOG.warn("No serialized HRegionInfo in " + rowResult);
5002 return true;
5003 }
5004 HRegionLocation l = list.getRegionLocation();
5005 if (l == null) {
5006 return true;
5007 }
5008 if (!l.getRegionInfo().getTable().equals(desc.getTableName())) {
5009 return false;
5010 }
5011 if (l.getRegionInfo().isOffline() || l.getRegionInfo().isSplit()) return true;
5012 HRegionLocation[] locations = list.getRegionLocations();
5013 for (HRegionLocation location : locations) {
5014 if (location == null) continue;
5015 ServerName serverName = location.getServerName();
5016
5017 if (serverName != null && serverName.getHostAndPort() != null) {
5018 actualRegCount.incrementAndGet();
5019 }
5020 }
5021 return true;
5022 }
5023 };
5024
5025 int tries = 0;
5026 int numRegs = (splitKeys == null ? 1 : splitKeys.length + 1) * desc.getRegionReplication();
5027 while (EnvironmentEdgeManager.currentTime() < deadlineTs) {
5028 actualRegCount.set(0);
5029
5030
5031 if (actualRegCount.get() == numRegs) {
5032
5033 return;
5034 }
5035
5036 try {
5037 Thread.sleep(getPauseTime(tries++, getAdmin().pause));
5038 } catch (InterruptedException e) {
5039 throw new InterruptedIOException("Interrupted when opening" + " regions; "
5040 + actualRegCount.get() + " of " + numRegs + " regions processed so far");
5041 }
5042 }
5043 throw new TimeoutException("Only " + actualRegCount.get() + " of " + numRegs
5044 + " regions are online; retries exhausted.");
5045 }
5046 }
5047
5048 @Override
5049 public boolean[] setSplitOrMergeEnabled(final boolean enabled, final boolean synchronous,
5050 final MasterSwitchType... switchTypes)
5051 throws IOException {
5052 return executeCallable(new MasterCallable<boolean[]>(getConnection()) {
5053 @Override
5054 public boolean[] call(int callTimeout) throws ServiceException {
5055 MasterProtos.SetSplitOrMergeEnabledResponse response = master.setSplitOrMergeEnabled(null,
5056 RequestConverter.buildSetSplitOrMergeEnabledRequest(enabled, synchronous, switchTypes));
5057 boolean[] result = new boolean[switchTypes.length];
5058 int i = 0;
5059 for (Boolean prevValue : response.getPrevValueList()) {
5060 result[i++] = prevValue;
5061 }
5062 return result;
5063 }
5064 });
5065 }
5066
5067 @Override
5068 public boolean isSplitOrMergeEnabled(final MasterSwitchType switchType) throws IOException {
5069 return executeCallable(new MasterCallable<Boolean>(getConnection()) {
5070 @Override
5071 public Boolean call(int callTimeout) throws ServiceException {
5072 return master.isSplitOrMergeEnabled(null,
5073 RequestConverter.buildIsSplitOrMergeEnabledRequest(switchType)).getEnabled();
5074 }
5075 });
5076 }
5077
5078 private HRegionInfo getMobRegionInfo(TableName tableName) {
5079 return new HRegionInfo(tableName, Bytes.toBytes(".mob"),
5080 HConstants.EMPTY_END_ROW, false, 0);
5081 }
5082
5083 @Override
5084 public BackupAdmin getBackupAdmin() throws IOException {
5085 return new HBaseBackupAdmin(this);
5086 }
5087
5088 private RpcControllerFactory getRpcControllerFactory() {
5089 return rpcControllerFactory;
5090 }
5091 }