2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org
.apache
.hadoop
.hbase
.rsgroup
;
20 import static org
.junit
.Assert
.assertEquals
;
21 import static org
.junit
.Assert
.assertTrue
;
23 import java
.io
.ByteArrayInputStream
;
24 import java
.io
.Closeable
;
25 import java
.io
.IOException
;
26 import java
.util
.ArrayList
;
27 import java
.util
.EnumSet
;
28 import java
.util
.List
;
31 import java
.util
.SortedSet
;
32 import java
.util
.concurrent
.Future
;
33 import java
.util
.regex
.Pattern
;
34 import org
.apache
.hadoop
.conf
.Configuration
;
35 import org
.apache
.hadoop
.hbase
.CacheEvictionStats
;
36 import org
.apache
.hadoop
.hbase
.ClusterMetrics
;
37 import org
.apache
.hadoop
.hbase
.ClusterMetrics
.Option
;
38 import org
.apache
.hadoop
.hbase
.NamespaceDescriptor
;
39 import org
.apache
.hadoop
.hbase
.NamespaceNotFoundException
;
40 import org
.apache
.hadoop
.hbase
.RegionMetrics
;
41 import org
.apache
.hadoop
.hbase
.ServerName
;
42 import org
.apache
.hadoop
.hbase
.TableExistsException
;
43 import org
.apache
.hadoop
.hbase
.TableName
;
44 import org
.apache
.hadoop
.hbase
.TableNotFoundException
;
45 import org
.apache
.hadoop
.hbase
.client
.Admin
;
46 import org
.apache
.hadoop
.hbase
.client
.ColumnFamilyDescriptor
;
47 import org
.apache
.hadoop
.hbase
.client
.CompactType
;
48 import org
.apache
.hadoop
.hbase
.client
.CompactionState
;
49 import org
.apache
.hadoop
.hbase
.client
.Connection
;
50 import org
.apache
.hadoop
.hbase
.client
.ConnectionFactory
;
51 import org
.apache
.hadoop
.hbase
.client
.RegionInfo
;
52 import org
.apache
.hadoop
.hbase
.client
.Result
;
53 import org
.apache
.hadoop
.hbase
.client
.ResultScanner
;
54 import org
.apache
.hadoop
.hbase
.client
.Scan
;
55 import org
.apache
.hadoop
.hbase
.client
.SlowLogQueryFilter
;
56 import org
.apache
.hadoop
.hbase
.client
.SlowLogRecord
;
57 import org
.apache
.hadoop
.hbase
.client
.SnapshotDescription
;
58 import org
.apache
.hadoop
.hbase
.client
.Table
;
59 import org
.apache
.hadoop
.hbase
.client
.TableDescriptor
;
60 import org
.apache
.hadoop
.hbase
.client
.replication
.TableCFs
;
61 import org
.apache
.hadoop
.hbase
.client
.security
.SecurityCapability
;
62 import org
.apache
.hadoop
.hbase
.exceptions
.DeserializationException
;
63 import org
.apache
.hadoop
.hbase
.ipc
.CoprocessorRpcChannel
;
64 import org
.apache
.hadoop
.hbase
.net
.Address
;
65 import org
.apache
.hadoop
.hbase
.protobuf
.ProtobufUtil
;
66 import org
.apache
.hadoop
.hbase
.protobuf
.generated
.RSGroupProtos
;
67 import org
.apache
.hadoop
.hbase
.quotas
.QuotaFilter
;
68 import org
.apache
.hadoop
.hbase
.quotas
.QuotaSettings
;
69 import org
.apache
.hadoop
.hbase
.quotas
.SpaceQuotaSnapshotView
;
70 import org
.apache
.hadoop
.hbase
.regionserver
.wal
.FailedLogCloseException
;
71 import org
.apache
.hadoop
.hbase
.replication
.ReplicationPeerConfig
;
72 import org
.apache
.hadoop
.hbase
.replication
.ReplicationPeerDescription
;
73 import org
.apache
.hadoop
.hbase
.replication
.SyncReplicationState
;
74 import org
.apache
.hadoop
.hbase
.security
.access
.GetUserPermissionsRequest
;
75 import org
.apache
.hadoop
.hbase
.security
.access
.Permission
;
76 import org
.apache
.hadoop
.hbase
.security
.access
.UserPermission
;
77 import org
.apache
.hadoop
.hbase
.snapshot
.HBaseSnapshotException
;
78 import org
.apache
.hadoop
.hbase
.snapshot
.RestoreSnapshotException
;
79 import org
.apache
.hadoop
.hbase
.snapshot
.SnapshotCreationException
;
80 import org
.apache
.hadoop
.hbase
.snapshot
.UnknownSnapshotException
;
81 import org
.apache
.hadoop
.hbase
.util
.Pair
;
82 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKUtil
;
83 import org
.apache
.hadoop
.hbase
.zookeeper
.ZKWatcher
;
84 import org
.apache
.hadoop
.hbase
.zookeeper
.ZNodePaths
;
85 import org
.apache
.yetus
.audience
.InterfaceAudience
;
86 import org
.apache
.zookeeper
.KeeperException
;
88 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Maps
;
89 import org
.apache
.hbase
.thirdparty
.com
.google
.common
.collect
.Sets
;
91 @InterfaceAudience.Private
92 public class VerifyingRSGroupAdmin
implements Admin
, Closeable
{
94 private final Connection conn
;
96 private final Admin admin
;
98 private final ZKWatcher zkw
;
100 public VerifyingRSGroupAdmin(Configuration conf
) throws IOException
{
101 conn
= ConnectionFactory
.createConnection(conf
);
102 admin
= conn
.getAdmin();
103 zkw
= new ZKWatcher(conf
, this.getClass().getSimpleName(), null);
106 public int getOperationTimeout() {
107 return admin
.getOperationTimeout();
110 public int getSyncWaitTimeout() {
111 return admin
.getSyncWaitTimeout();
114 public void abort(String why
, Throwable e
) {
118 public boolean isAborted() {
119 return admin
.isAborted();
122 public Connection
getConnection() {
123 return admin
.getConnection();
126 public boolean tableExists(TableName tableName
) throws IOException
{
127 return admin
.tableExists(tableName
);
130 public List
<TableDescriptor
> listTableDescriptors() throws IOException
{
131 return admin
.listTableDescriptors();
134 public List
<TableDescriptor
> listTableDescriptors(boolean includeSysTables
) throws IOException
{
135 return admin
.listTableDescriptors(includeSysTables
);
138 public List
<TableDescriptor
> listTableDescriptors(Pattern pattern
, boolean includeSysTables
)
140 return admin
.listTableDescriptors(pattern
, includeSysTables
);
143 public TableName
[] listTableNames() throws IOException
{
144 return admin
.listTableNames();
147 public TableName
[] listTableNames(Pattern pattern
, boolean includeSysTables
) throws IOException
{
148 return admin
.listTableNames(pattern
, includeSysTables
);
151 public TableDescriptor
getDescriptor(TableName tableName
)
152 throws TableNotFoundException
, IOException
{
153 return admin
.getDescriptor(tableName
);
156 public void createTable(TableDescriptor desc
, byte[] startKey
, byte[] endKey
, int numRegions
)
158 admin
.createTable(desc
, startKey
, endKey
, numRegions
);
161 public Future
<Void
> createTableAsync(TableDescriptor desc
) throws IOException
{
162 return admin
.createTableAsync(desc
);
165 public Future
<Void
> createTableAsync(TableDescriptor desc
, byte[][] splitKeys
)
167 return admin
.createTableAsync(desc
, splitKeys
);
170 public Future
<Void
> deleteTableAsync(TableName tableName
) throws IOException
{
171 return admin
.deleteTableAsync(tableName
);
174 public Future
<Void
> truncateTableAsync(TableName tableName
, boolean preserveSplits
)
176 return admin
.truncateTableAsync(tableName
, preserveSplits
);
179 public Future
<Void
> enableTableAsync(TableName tableName
) throws IOException
{
180 return admin
.enableTableAsync(tableName
);
183 public Future
<Void
> disableTableAsync(TableName tableName
) throws IOException
{
184 return admin
.disableTableAsync(tableName
);
187 public boolean isTableEnabled(TableName tableName
) throws IOException
{
188 return admin
.isTableEnabled(tableName
);
191 public boolean isTableDisabled(TableName tableName
) throws IOException
{
192 return admin
.isTableDisabled(tableName
);
195 public boolean isTableAvailable(TableName tableName
) throws IOException
{
196 return admin
.isTableAvailable(tableName
);
199 public Future
<Void
> addColumnFamilyAsync(TableName tableName
, ColumnFamilyDescriptor columnFamily
)
201 return admin
.addColumnFamilyAsync(tableName
, columnFamily
);
204 public Future
<Void
> deleteColumnFamilyAsync(TableName tableName
, byte[] columnFamily
)
206 return admin
.deleteColumnFamilyAsync(tableName
, columnFamily
);
209 public Future
<Void
> modifyColumnFamilyAsync(TableName tableName
,
210 ColumnFamilyDescriptor columnFamily
) throws IOException
{
211 return admin
.modifyColumnFamilyAsync(tableName
, columnFamily
);
214 public List
<RegionInfo
> getRegions(ServerName serverName
) throws IOException
{
215 return admin
.getRegions(serverName
);
218 public void flush(TableName tableName
) throws IOException
{
219 admin
.flush(tableName
);
222 public void flushRegion(byte[] regionName
) throws IOException
{
223 admin
.flushRegion(regionName
);
226 public void flushRegionServer(ServerName serverName
) throws IOException
{
227 admin
.flushRegionServer(serverName
);
230 public void compact(TableName tableName
) throws IOException
{
231 admin
.compact(tableName
);
234 public void compactRegion(byte[] regionName
) throws IOException
{
235 admin
.compactRegion(regionName
);
238 public void compact(TableName tableName
, byte[] columnFamily
) throws IOException
{
239 admin
.compact(tableName
, columnFamily
);
242 public void compactRegion(byte[] regionName
, byte[] columnFamily
) throws IOException
{
243 admin
.compactRegion(regionName
, columnFamily
);
246 public void compact(TableName tableName
, CompactType compactType
)
247 throws IOException
, InterruptedException
{
248 admin
.compact(tableName
, compactType
);
251 public void compact(TableName tableName
, byte[] columnFamily
, CompactType compactType
)
252 throws IOException
, InterruptedException
{
253 admin
.compact(tableName
, columnFamily
, compactType
);
256 public void majorCompact(TableName tableName
) throws IOException
{
257 admin
.majorCompact(tableName
);
260 public void majorCompactRegion(byte[] regionName
) throws IOException
{
261 admin
.majorCompactRegion(regionName
);
264 public void majorCompact(TableName tableName
, byte[] columnFamily
) throws IOException
{
265 admin
.majorCompact(tableName
, columnFamily
);
268 public void majorCompactRegion(byte[] regionName
, byte[] columnFamily
) throws IOException
{
269 admin
.majorCompactRegion(regionName
, columnFamily
);
272 public void majorCompact(TableName tableName
, CompactType compactType
)
273 throws IOException
, InterruptedException
{
274 admin
.majorCompact(tableName
, compactType
);
277 public void majorCompact(TableName tableName
, byte[] columnFamily
, CompactType compactType
)
278 throws IOException
, InterruptedException
{
279 admin
.majorCompact(tableName
, columnFamily
, compactType
);
282 public Map
<ServerName
, Boolean
> compactionSwitch(boolean switchState
,
283 List
<String
> serverNamesList
) throws IOException
{
284 return admin
.compactionSwitch(switchState
, serverNamesList
);
287 public void compactRegionServer(ServerName serverName
) throws IOException
{
288 admin
.compactRegionServer(serverName
);
291 public void majorCompactRegionServer(ServerName serverName
) throws IOException
{
292 admin
.majorCompactRegionServer(serverName
);
295 public void move(byte[] encodedRegionName
) throws IOException
{
296 admin
.move(encodedRegionName
);
299 public void move(byte[] encodedRegionName
, ServerName destServerName
) throws IOException
{
300 admin
.move(encodedRegionName
, destServerName
);
303 public void assign(byte[] regionName
) throws IOException
{
304 admin
.assign(regionName
);
307 public void unassign(byte[] regionName
, boolean force
) throws IOException
{
308 admin
.unassign(regionName
, force
);
311 public void offline(byte[] regionName
) throws IOException
{
312 admin
.offline(regionName
);
315 public boolean balancerSwitch(boolean onOrOff
, boolean synchronous
) throws IOException
{
316 return admin
.balancerSwitch(onOrOff
, synchronous
);
319 public boolean balance() throws IOException
{
320 return admin
.balance();
323 public boolean balance(boolean force
) throws IOException
{
324 return admin
.balance(force
);
327 public boolean isBalancerEnabled() throws IOException
{
328 return admin
.isBalancerEnabled();
331 public CacheEvictionStats
clearBlockCache(TableName tableName
) throws IOException
{
332 return admin
.clearBlockCache(tableName
);
335 public boolean normalize() throws IOException
{
336 return admin
.normalize();
339 public boolean isNormalizerEnabled() throws IOException
{
340 return admin
.isNormalizerEnabled();
343 public boolean normalizerSwitch(boolean on
) throws IOException
{
344 return admin
.normalizerSwitch(on
);
347 public boolean catalogJanitorSwitch(boolean onOrOff
) throws IOException
{
348 return admin
.catalogJanitorSwitch(onOrOff
);
351 public int runCatalogJanitor() throws IOException
{
352 return admin
.runCatalogJanitor();
355 public boolean isCatalogJanitorEnabled() throws IOException
{
356 return admin
.isCatalogJanitorEnabled();
359 public boolean cleanerChoreSwitch(boolean onOrOff
) throws IOException
{
360 return admin
.cleanerChoreSwitch(onOrOff
);
363 public boolean runCleanerChore() throws IOException
{
364 return admin
.runCleanerChore();
367 public boolean isCleanerChoreEnabled() throws IOException
{
368 return admin
.isCleanerChoreEnabled();
371 public Future
<Void
> mergeRegionsAsync(byte[][] nameofRegionsToMerge
, boolean forcible
)
373 return admin
.mergeRegionsAsync(nameofRegionsToMerge
, forcible
);
376 public void split(TableName tableName
) throws IOException
{
377 admin
.split(tableName
);
380 public void split(TableName tableName
, byte[] splitPoint
) throws IOException
{
381 admin
.split(tableName
, splitPoint
);
384 public Future
<Void
> splitRegionAsync(byte[] regionName
) throws IOException
{
385 return admin
.splitRegionAsync(regionName
);
388 public Future
<Void
> splitRegionAsync(byte[] regionName
, byte[] splitPoint
) throws IOException
{
389 return admin
.splitRegionAsync(regionName
, splitPoint
);
392 public Future
<Void
> modifyTableAsync(TableDescriptor td
) throws IOException
{
393 return admin
.modifyTableAsync(td
);
396 public void shutdown() throws IOException
{
400 public void stopMaster() throws IOException
{
404 public boolean isMasterInMaintenanceMode() throws IOException
{
405 return admin
.isMasterInMaintenanceMode();
408 public void stopRegionServer(String hostnamePort
) throws IOException
{
409 admin
.stopRegionServer(hostnamePort
);
412 public ClusterMetrics
getClusterMetrics(EnumSet
<Option
> options
) throws IOException
{
413 return admin
.getClusterMetrics(options
);
416 public List
<RegionMetrics
> getRegionMetrics(ServerName serverName
) throws IOException
{
417 return admin
.getRegionMetrics(serverName
);
420 public List
<RegionMetrics
> getRegionMetrics(ServerName serverName
, TableName tableName
)
422 return admin
.getRegionMetrics(serverName
, tableName
);
425 public Configuration
getConfiguration() {
426 return admin
.getConfiguration();
429 public Future
<Void
> createNamespaceAsync(NamespaceDescriptor descriptor
) throws IOException
{
430 return admin
.createNamespaceAsync(descriptor
);
433 public Future
<Void
> modifyNamespaceAsync(NamespaceDescriptor descriptor
) throws IOException
{
434 return admin
.modifyNamespaceAsync(descriptor
);
437 public Future
<Void
> deleteNamespaceAsync(String name
) throws IOException
{
438 return admin
.deleteNamespaceAsync(name
);
441 public NamespaceDescriptor
getNamespaceDescriptor(String name
)
442 throws NamespaceNotFoundException
, IOException
{
443 return admin
.getNamespaceDescriptor(name
);
446 public String
[] listNamespaces() throws IOException
{
447 return admin
.listNamespaces();
450 public NamespaceDescriptor
[] listNamespaceDescriptors() throws IOException
{
451 return admin
.listNamespaceDescriptors();
454 public List
<TableDescriptor
> listTableDescriptorsByNamespace(byte[] name
) throws IOException
{
455 return admin
.listTableDescriptorsByNamespace(name
);
458 public TableName
[] listTableNamesByNamespace(String name
) throws IOException
{
459 return admin
.listTableNamesByNamespace(name
);
462 public List
<RegionInfo
> getRegions(TableName tableName
) throws IOException
{
463 return admin
.getRegions(tableName
);
466 public void close() {
470 public List
<TableDescriptor
> listTableDescriptors(List
<TableName
> tableNames
) throws IOException
{
471 return admin
.listTableDescriptors(tableNames
);
474 public Future
<Boolean
> abortProcedureAsync(long procId
, boolean mayInterruptIfRunning
)
476 return admin
.abortProcedureAsync(procId
, mayInterruptIfRunning
);
479 public String
getProcedures() throws IOException
{
480 return admin
.getProcedures();
483 public String
getLocks() throws IOException
{
484 return admin
.getLocks();
487 public void rollWALWriter(ServerName serverName
) throws IOException
, FailedLogCloseException
{
488 admin
.rollWALWriter(serverName
);
491 public CompactionState
getCompactionState(TableName tableName
) throws IOException
{
492 return admin
.getCompactionState(tableName
);
495 public CompactionState
getCompactionState(TableName tableName
, CompactType compactType
)
497 return admin
.getCompactionState(tableName
, compactType
);
500 public CompactionState
getCompactionStateForRegion(byte[] regionName
) throws IOException
{
501 return admin
.getCompactionStateForRegion(regionName
);
504 public long getLastMajorCompactionTimestamp(TableName tableName
) throws IOException
{
505 return admin
.getLastMajorCompactionTimestamp(tableName
);
508 public long getLastMajorCompactionTimestampForRegion(byte[] regionName
) throws IOException
{
509 return admin
.getLastMajorCompactionTimestampForRegion(regionName
);
512 public void snapshot(SnapshotDescription snapshot
)
513 throws IOException
, SnapshotCreationException
, IllegalArgumentException
{
514 admin
.snapshot(snapshot
);
517 public Future
<Void
> snapshotAsync(SnapshotDescription snapshot
)
518 throws IOException
, SnapshotCreationException
{
519 return admin
.snapshotAsync(snapshot
);
522 public boolean isSnapshotFinished(SnapshotDescription snapshot
)
523 throws IOException
, HBaseSnapshotException
, UnknownSnapshotException
{
524 return admin
.isSnapshotFinished(snapshot
);
527 public void restoreSnapshot(String snapshotName
) throws IOException
, RestoreSnapshotException
{
528 admin
.restoreSnapshot(snapshotName
);
531 public void restoreSnapshot(String snapshotName
, boolean takeFailSafeSnapshot
, boolean restoreAcl
)
532 throws IOException
, RestoreSnapshotException
{
533 admin
.restoreSnapshot(snapshotName
, takeFailSafeSnapshot
, restoreAcl
);
536 public Future
<Void
> cloneSnapshotAsync(String snapshotName
, TableName tableName
,
537 boolean restoreAcl
) throws IOException
, TableExistsException
, RestoreSnapshotException
{
538 return admin
.cloneSnapshotAsync(snapshotName
, tableName
, restoreAcl
);
541 public void execProcedure(String signature
, String instance
, Map
<String
, String
> props
)
543 admin
.execProcedure(signature
, instance
, props
);
546 public byte[] execProcedureWithReturn(String signature
, String instance
,
547 Map
<String
, String
> props
) throws IOException
{
548 return admin
.execProcedureWithReturn(signature
, instance
, props
);
551 public boolean isProcedureFinished(String signature
, String instance
, Map
<String
, String
> props
)
553 return admin
.isProcedureFinished(signature
, instance
, props
);
556 public List
<SnapshotDescription
> listSnapshots() throws IOException
{
557 return admin
.listSnapshots();
560 public List
<SnapshotDescription
> listSnapshots(Pattern pattern
) throws IOException
{
561 return admin
.listSnapshots(pattern
);
564 public List
<SnapshotDescription
> listTableSnapshots(Pattern tableNamePattern
,
565 Pattern snapshotNamePattern
) throws IOException
{
566 return admin
.listTableSnapshots(tableNamePattern
, snapshotNamePattern
);
569 public void deleteSnapshot(String snapshotName
) throws IOException
{
570 admin
.deleteSnapshot(snapshotName
);
573 public void deleteSnapshots(Pattern pattern
) throws IOException
{
574 admin
.deleteSnapshots(pattern
);
577 public void deleteTableSnapshots(Pattern tableNamePattern
, Pattern snapshotNamePattern
)
579 admin
.deleteTableSnapshots(tableNamePattern
, snapshotNamePattern
);
582 public void setQuota(QuotaSettings quota
) throws IOException
{
583 admin
.setQuota(quota
);
586 public List
<QuotaSettings
> getQuota(QuotaFilter filter
) throws IOException
{
587 return admin
.getQuota(filter
);
590 public CoprocessorRpcChannel
coprocessorService() {
591 return admin
.coprocessorService();
594 public CoprocessorRpcChannel
coprocessorService(ServerName serverName
) {
595 return admin
.coprocessorService(serverName
);
598 public void updateConfiguration(ServerName server
) throws IOException
{
599 admin
.updateConfiguration(server
);
602 public void updateConfiguration() throws IOException
{
603 admin
.updateConfiguration();
606 public List
<SecurityCapability
> getSecurityCapabilities() throws IOException
{
607 return admin
.getSecurityCapabilities();
610 public boolean splitSwitch(boolean enabled
, boolean synchronous
) throws IOException
{
611 return admin
.splitSwitch(enabled
, synchronous
);
614 public boolean mergeSwitch(boolean enabled
, boolean synchronous
) throws IOException
{
615 return admin
.mergeSwitch(enabled
, synchronous
);
618 public boolean isSplitEnabled() throws IOException
{
619 return admin
.isSplitEnabled();
622 public boolean isMergeEnabled() throws IOException
{
623 return admin
.isMergeEnabled();
626 public Future
<Void
> addReplicationPeerAsync(String peerId
, ReplicationPeerConfig peerConfig
,
627 boolean enabled
) throws IOException
{
628 return admin
.addReplicationPeerAsync(peerId
, peerConfig
, enabled
);
631 public Future
<Void
> removeReplicationPeerAsync(String peerId
) throws IOException
{
632 return admin
.removeReplicationPeerAsync(peerId
);
635 public Future
<Void
> enableReplicationPeerAsync(String peerId
) throws IOException
{
636 return admin
.enableReplicationPeerAsync(peerId
);
639 public Future
<Void
> disableReplicationPeerAsync(String peerId
) throws IOException
{
640 return admin
.disableReplicationPeerAsync(peerId
);
643 public ReplicationPeerConfig
getReplicationPeerConfig(String peerId
) throws IOException
{
644 return admin
.getReplicationPeerConfig(peerId
);
647 public Future
<Void
> updateReplicationPeerConfigAsync(String peerId
,
648 ReplicationPeerConfig peerConfig
) throws IOException
{
649 return admin
.updateReplicationPeerConfigAsync(peerId
, peerConfig
);
652 public List
<ReplicationPeerDescription
> listReplicationPeers() throws IOException
{
653 return admin
.listReplicationPeers();
656 public List
<ReplicationPeerDescription
> listReplicationPeers(Pattern pattern
) throws IOException
{
657 return admin
.listReplicationPeers(pattern
);
660 public Future
<Void
> transitReplicationPeerSyncReplicationStateAsync(String peerId
,
661 SyncReplicationState state
) throws IOException
{
662 return admin
.transitReplicationPeerSyncReplicationStateAsync(peerId
, state
);
665 public void decommissionRegionServers(List
<ServerName
> servers
, boolean offload
)
667 admin
.decommissionRegionServers(servers
, offload
);
670 public List
<ServerName
> listDecommissionedRegionServers() throws IOException
{
671 return admin
.listDecommissionedRegionServers();
674 public void recommissionRegionServer(ServerName server
, List
<byte[]> encodedRegionNames
)
676 admin
.recommissionRegionServer(server
, encodedRegionNames
);
679 public List
<TableCFs
> listReplicatedTableCFs() throws IOException
{
680 return admin
.listReplicatedTableCFs();
683 public void enableTableReplication(TableName tableName
) throws IOException
{
684 admin
.enableTableReplication(tableName
);
687 public void disableTableReplication(TableName tableName
) throws IOException
{
688 admin
.disableTableReplication(tableName
);
691 public void clearCompactionQueues(ServerName serverName
, Set
<String
> queues
)
692 throws IOException
, InterruptedException
{
693 admin
.clearCompactionQueues(serverName
, queues
);
696 public List
<ServerName
> clearDeadServers(List
<ServerName
> servers
) throws IOException
{
697 return admin
.clearDeadServers(servers
);
700 public void cloneTableSchema(TableName tableName
, TableName newTableName
, boolean preserveSplits
)
702 admin
.cloneTableSchema(tableName
, newTableName
, preserveSplits
);
705 public boolean switchRpcThrottle(boolean enable
) throws IOException
{
706 return admin
.switchRpcThrottle(enable
);
709 public boolean isRpcThrottleEnabled() throws IOException
{
710 return admin
.isRpcThrottleEnabled();
713 public boolean exceedThrottleQuotaSwitch(boolean enable
) throws IOException
{
714 return admin
.exceedThrottleQuotaSwitch(enable
);
717 public Map
<TableName
, Long
> getSpaceQuotaTableSizes() throws IOException
{
718 return admin
.getSpaceQuotaTableSizes();
721 public Map
<TableName
, ?
extends SpaceQuotaSnapshotView
>
722 getRegionServerSpaceQuotaSnapshots(ServerName serverName
) throws IOException
{
723 return admin
.getRegionServerSpaceQuotaSnapshots(serverName
);
726 public SpaceQuotaSnapshotView
getCurrentSpaceQuotaSnapshot(String namespace
) throws IOException
{
727 return admin
.getCurrentSpaceQuotaSnapshot(namespace
);
730 public SpaceQuotaSnapshotView
getCurrentSpaceQuotaSnapshot(TableName tableName
)
732 return admin
.getCurrentSpaceQuotaSnapshot(tableName
);
735 public void grant(UserPermission userPermission
, boolean mergeExistingPermissions
)
737 admin
.grant(userPermission
, mergeExistingPermissions
);
740 public void revoke(UserPermission userPermission
) throws IOException
{
741 admin
.revoke(userPermission
);
744 public List
<UserPermission
>
745 getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest
) throws IOException
{
746 return admin
.getUserPermissions(getUserPermissionsRequest
);
749 public List
<Boolean
> hasUserPermissions(String userName
, List
<Permission
> permissions
)
751 return admin
.hasUserPermissions(userName
, permissions
);
754 public boolean snapshotCleanupSwitch(boolean on
, boolean synchronous
) throws IOException
{
755 return admin
.snapshotCleanupSwitch(on
, synchronous
);
758 public boolean isSnapshotCleanupEnabled() throws IOException
{
759 return admin
.isSnapshotCleanupEnabled();
762 public void addRSGroup(String groupName
) throws IOException
{
763 admin
.addRSGroup(groupName
);
767 public RSGroupInfo
getRSGroup(String groupName
) throws IOException
{
768 return admin
.getRSGroup(groupName
);
771 public RSGroupInfo
getRSGroup(Address hostPort
) throws IOException
{
772 return admin
.getRSGroup(hostPort
);
775 public RSGroupInfo
getRSGroup(TableName tableName
) throws IOException
{
776 return admin
.getRSGroup(tableName
);
779 public List
<RSGroupInfo
> listRSGroups() throws IOException
{
780 return admin
.listRSGroups();
784 public List
<TableName
> listTablesInRSGroup(String groupName
) throws IOException
{
785 return admin
.listTablesInRSGroup(groupName
);
789 public Pair
<List
<String
>, List
<TableName
>>
790 getConfiguredNamespacesAndTablesInRSGroup(String groupName
) throws IOException
{
791 return admin
.getConfiguredNamespacesAndTablesInRSGroup(groupName
);
794 public void removeRSGroup(String groupName
) throws IOException
{
795 admin
.removeRSGroup(groupName
);
799 public void removeServersFromRSGroup(Set
<Address
> servers
) throws IOException
{
800 admin
.removeServersFromRSGroup(servers
);
804 public void moveServersToRSGroup(Set
<Address
> servers
, String targetGroup
) throws IOException
{
805 admin
.moveServersToRSGroup(servers
, targetGroup
);
809 public void setRSGroup(Set
<TableName
> tables
, String groupName
) throws IOException
{
810 admin
.setRSGroup(tables
, groupName
);
814 public boolean balanceRSGroup(String groupName
) throws IOException
{
815 return admin
.balanceRSGroup(groupName
);
818 private void verify() throws IOException
{
819 Map
<String
, RSGroupInfo
> groupMap
= Maps
.newHashMap();
820 Set
<RSGroupInfo
> zList
= Sets
.newHashSet();
821 List
<TableDescriptor
> tds
= new ArrayList
<>();
822 try (Admin admin
= conn
.getAdmin()) {
823 tds
.addAll(admin
.listTableDescriptors());
824 tds
.addAll(admin
.listTableDescriptorsByNamespace(NamespaceDescriptor
.SYSTEM_NAMESPACE_NAME
));
826 SortedSet
<Address
> lives
= Sets
.newTreeSet();
827 for (ServerName sn
: conn
.getAdmin().getClusterMetrics().getLiveServerMetrics().keySet()) {
828 lives
.add(sn
.getAddress());
830 for (ServerName sn
: conn
.getAdmin().listDecommissionedRegionServers()) {
831 lives
.remove(sn
.getAddress());
833 try (Table table
= conn
.getTable(RSGroupInfoManagerImpl
.RSGROUP_TABLE_NAME
);
834 ResultScanner scanner
= table
.getScanner(new Scan())) {
836 Result result
= scanner
.next();
837 if (result
== null) {
840 RSGroupProtos
.RSGroupInfo proto
= RSGroupProtos
.RSGroupInfo
.parseFrom(result
.getValue(
841 RSGroupInfoManagerImpl
.META_FAMILY_BYTES
, RSGroupInfoManagerImpl
.META_QUALIFIER_BYTES
));
842 RSGroupInfo rsGroupInfo
= ProtobufUtil
.toGroupInfo(proto
);
843 groupMap
.put(proto
.getName(), RSGroupUtil
.fillTables(rsGroupInfo
, tds
));
844 for (Address address
: rsGroupInfo
.getServers()) {
845 lives
.remove(address
);
849 SortedSet
<TableName
> tables
= Sets
.newTreeSet();
850 for (TableDescriptor td
: conn
.getAdmin().listTableDescriptors(Pattern
.compile(".*"), true)) {
851 String groupName
= td
.getRegionServerGroup().orElse(RSGroupInfo
.DEFAULT_GROUP
);
852 if (groupName
.equals(RSGroupInfo
.DEFAULT_GROUP
)) {
853 tables
.add(td
.getTableName());
857 groupMap
.put(RSGroupInfo
.DEFAULT_GROUP
,
858 new RSGroupInfo(RSGroupInfo
.DEFAULT_GROUP
, lives
, tables
));
859 assertEquals(Sets
.newHashSet(groupMap
.values()), Sets
.newHashSet(admin
.listRSGroups()));
861 String groupBasePath
= ZNodePaths
.joinZNode(zkw
.getZNodePaths().baseZNode
, "rsgroup");
862 for (String znode
: ZKUtil
.listChildrenNoWatch(zkw
, groupBasePath
)) {
863 byte[] data
= ZKUtil
.getData(zkw
, ZNodePaths
.joinZNode(groupBasePath
, znode
));
864 if (data
.length
> 0) {
865 ProtobufUtil
.expectPBMagicPrefix(data
);
866 ByteArrayInputStream bis
=
867 new ByteArrayInputStream(data
, ProtobufUtil
.lengthOfPBMagic(), data
.length
);
868 RSGroupInfo rsGroupInfo
=
869 ProtobufUtil
.toGroupInfo(RSGroupProtos
.RSGroupInfo
.parseFrom(bis
));
870 zList
.add(RSGroupUtil
.fillTables(rsGroupInfo
, tds
));
873 groupMap
.remove(RSGroupInfo
.DEFAULT_GROUP
);
874 assertEquals(zList
.size(), groupMap
.size());
875 for (RSGroupInfo rsGroupInfo
: zList
) {
876 assertTrue(groupMap
.get(rsGroupInfo
.getName()).equals(rsGroupInfo
));
878 } catch (KeeperException e
) {
879 throw new IOException("ZK verification failed", e
);
880 } catch (DeserializationException e
) {
881 throw new IOException("ZK verification failed", e
);
882 } catch (InterruptedException e
) {
883 throw new IOException("ZK verification failed", e
);
888 public List
<SlowLogRecord
> getSlowLogResponses(Set
<ServerName
> serverNames
,
889 SlowLogQueryFilter slowLogQueryFilter
) throws IOException
{
894 public List
<Boolean
> clearSlowLogResponses(Set
<ServerName
> serverNames
) throws IOException
{