HBASE-26921 Rewrite the counting cells part in TestMultiVersions (#4316)
[hbase.git] / hbase-server / src / test / java / org / apache / hadoop / hbase / rsgroup / VerifyingRSGroupAdmin.java
blob44a42f15517657403338b27edf40029d6a5effd7
1 /*
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;
29 import java.util.Map;
30 import java.util.Set;
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.BalanceRequest;
47 import org.apache.hadoop.hbase.client.BalanceResponse;
48 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
49 import org.apache.hadoop.hbase.client.CompactType;
50 import org.apache.hadoop.hbase.client.CompactionState;
51 import org.apache.hadoop.hbase.client.Connection;
52 import org.apache.hadoop.hbase.client.ConnectionFactory;
53 import org.apache.hadoop.hbase.client.LogEntry;
54 import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
55 import org.apache.hadoop.hbase.client.RegionInfo;
56 import org.apache.hadoop.hbase.client.Result;
57 import org.apache.hadoop.hbase.client.ResultScanner;
58 import org.apache.hadoop.hbase.client.Scan;
59 import org.apache.hadoop.hbase.client.ServerType;
60 import org.apache.hadoop.hbase.client.SnapshotDescription;
61 import org.apache.hadoop.hbase.client.Table;
62 import org.apache.hadoop.hbase.client.TableDescriptor;
63 import org.apache.hadoop.hbase.client.replication.TableCFs;
64 import org.apache.hadoop.hbase.client.security.SecurityCapability;
65 import org.apache.hadoop.hbase.exceptions.DeserializationException;
66 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
67 import org.apache.hadoop.hbase.net.Address;
68 import org.apache.hadoop.hbase.quotas.QuotaFilter;
69 import org.apache.hadoop.hbase.quotas.QuotaSettings;
70 import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
71 import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
72 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
73 import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
74 import org.apache.hadoop.hbase.replication.SyncReplicationState;
75 import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
76 import org.apache.hadoop.hbase.security.access.Permission;
77 import org.apache.hadoop.hbase.security.access.UserPermission;
78 import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
79 import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException;
80 import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
81 import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException;
82 import org.apache.hadoop.hbase.util.Pair;
83 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
84 import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
85 import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
86 import org.apache.yetus.audience.InterfaceAudience;
87 import org.apache.zookeeper.KeeperException;
89 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
90 import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
92 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
93 import org.apache.hadoop.hbase.shaded.protobuf.generated.RSGroupProtos;
95 @InterfaceAudience.Private
96 public class VerifyingRSGroupAdmin implements Admin, Closeable {
98 private final Connection conn;
100 private final Admin admin;
102 private final ZKWatcher zkw;
104 public VerifyingRSGroupAdmin(Configuration conf) throws IOException {
105 conn = ConnectionFactory.createConnection(conf);
106 admin = conn.getAdmin();
107 zkw = new ZKWatcher(conf, this.getClass().getSimpleName(), null);
110 public int getOperationTimeout() {
111 return admin.getOperationTimeout();
114 public int getSyncWaitTimeout() {
115 return admin.getSyncWaitTimeout();
118 public void abort(String why, Throwable e) {
119 admin.abort(why, e);
122 public boolean isAborted() {
123 return admin.isAborted();
126 public Connection getConnection() {
127 return admin.getConnection();
130 public boolean tableExists(TableName tableName) throws IOException {
131 return admin.tableExists(tableName);
134 public List<TableDescriptor> listTableDescriptors() throws IOException {
135 return admin.listTableDescriptors();
138 public List<TableDescriptor> listTableDescriptors(boolean includeSysTables) throws IOException {
139 return admin.listTableDescriptors(includeSysTables);
142 public List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables)
143 throws IOException {
144 return admin.listTableDescriptors(pattern, includeSysTables);
147 public TableName[] listTableNames() throws IOException {
148 return admin.listTableNames();
151 public TableName[] listTableNames(Pattern pattern, boolean includeSysTables) throws IOException {
152 return admin.listTableNames(pattern, includeSysTables);
155 public TableDescriptor getDescriptor(TableName tableName)
156 throws TableNotFoundException, IOException {
157 return admin.getDescriptor(tableName);
160 public void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions)
161 throws IOException {
162 admin.createTable(desc, startKey, endKey, numRegions);
165 public Future<Void> createTableAsync(TableDescriptor desc) throws IOException {
166 return admin.createTableAsync(desc);
169 public Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys)
170 throws IOException {
171 return admin.createTableAsync(desc, splitKeys);
174 public Future<Void> deleteTableAsync(TableName tableName) throws IOException {
175 return admin.deleteTableAsync(tableName);
178 public Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits)
179 throws IOException {
180 return admin.truncateTableAsync(tableName, preserveSplits);
183 public Future<Void> enableTableAsync(TableName tableName) throws IOException {
184 return admin.enableTableAsync(tableName);
187 public Future<Void> disableTableAsync(TableName tableName) throws IOException {
188 return admin.disableTableAsync(tableName);
191 public boolean isTableEnabled(TableName tableName) throws IOException {
192 return admin.isTableEnabled(tableName);
195 public boolean isTableDisabled(TableName tableName) throws IOException {
196 return admin.isTableDisabled(tableName);
199 public boolean isTableAvailable(TableName tableName) throws IOException {
200 return admin.isTableAvailable(tableName);
203 public Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily)
204 throws IOException {
205 return admin.addColumnFamilyAsync(tableName, columnFamily);
208 public Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily)
209 throws IOException {
210 return admin.deleteColumnFamilyAsync(tableName, columnFamily);
213 public Future<Void> modifyColumnFamilyAsync(TableName tableName,
214 ColumnFamilyDescriptor columnFamily) throws IOException {
215 return admin.modifyColumnFamilyAsync(tableName, columnFamily);
218 public List<RegionInfo> getRegions(ServerName serverName) throws IOException {
219 return admin.getRegions(serverName);
222 public void flush(TableName tableName) throws IOException {
223 admin.flush(tableName);
226 public void flush(TableName tableName, byte[] columnFamily) throws IOException {
227 admin.flush(tableName, columnFamily);
230 public void flushRegion(byte[] regionName) throws IOException {
231 admin.flushRegion(regionName);
234 public void flushRegion(byte[] regionName, byte[] columnFamily) throws IOException {
235 admin.flushRegion(regionName, columnFamily);
238 public void flushRegionServer(ServerName serverName) throws IOException {
239 admin.flushRegionServer(serverName);
242 public void compact(TableName tableName) throws IOException {
243 admin.compact(tableName);
246 public void compactRegion(byte[] regionName) throws IOException {
247 admin.compactRegion(regionName);
250 public void compact(TableName tableName, byte[] columnFamily) throws IOException {
251 admin.compact(tableName, columnFamily);
254 public void compactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
255 admin.compactRegion(regionName, columnFamily);
258 public void compact(TableName tableName, CompactType compactType)
259 throws IOException, InterruptedException {
260 admin.compact(tableName, compactType);
263 public void compact(TableName tableName, byte[] columnFamily, CompactType compactType)
264 throws IOException, InterruptedException {
265 admin.compact(tableName, columnFamily, compactType);
268 public void majorCompact(TableName tableName) throws IOException {
269 admin.majorCompact(tableName);
272 public void majorCompactRegion(byte[] regionName) throws IOException {
273 admin.majorCompactRegion(regionName);
276 public void majorCompact(TableName tableName, byte[] columnFamily) throws IOException {
277 admin.majorCompact(tableName, columnFamily);
280 public void majorCompactRegion(byte[] regionName, byte[] columnFamily) throws IOException {
281 admin.majorCompactRegion(regionName, columnFamily);
284 public void majorCompact(TableName tableName, CompactType compactType)
285 throws IOException, InterruptedException {
286 admin.majorCompact(tableName, compactType);
289 public void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType)
290 throws IOException, InterruptedException {
291 admin.majorCompact(tableName, columnFamily, compactType);
294 public Map<ServerName, Boolean> compactionSwitch(boolean switchState,
295 List<String> serverNamesList) throws IOException {
296 return admin.compactionSwitch(switchState, serverNamesList);
299 public void compactRegionServer(ServerName serverName) throws IOException {
300 admin.compactRegionServer(serverName);
303 public void majorCompactRegionServer(ServerName serverName) throws IOException {
304 admin.majorCompactRegionServer(serverName);
307 public void move(byte[] encodedRegionName) throws IOException {
308 admin.move(encodedRegionName);
311 public void move(byte[] encodedRegionName, ServerName destServerName) throws IOException {
312 admin.move(encodedRegionName, destServerName);
315 public void assign(byte[] regionName) throws IOException {
316 admin.assign(regionName);
319 public void unassign(byte[] regionName) throws IOException {
320 admin.unassign(regionName);
323 public void offline(byte[] regionName) throws IOException {
324 admin.offline(regionName);
327 public boolean balancerSwitch(boolean onOrOff, boolean synchronous) throws IOException {
328 return admin.balancerSwitch(onOrOff, synchronous);
331 public BalanceResponse balance(BalanceRequest request) throws IOException {
332 return admin.balance(request);
335 public boolean isBalancerEnabled() throws IOException {
336 return admin.isBalancerEnabled();
339 public CacheEvictionStats clearBlockCache(TableName tableName) throws IOException {
340 return admin.clearBlockCache(tableName);
343 @Override
344 public boolean normalize(NormalizeTableFilterParams ntfp) throws IOException {
345 return admin.normalize(ntfp);
348 public boolean isNormalizerEnabled() throws IOException {
349 return admin.isNormalizerEnabled();
352 public boolean normalizerSwitch(boolean on) throws IOException {
353 return admin.normalizerSwitch(on);
356 public boolean catalogJanitorSwitch(boolean onOrOff) throws IOException {
357 return admin.catalogJanitorSwitch(onOrOff);
360 public int runCatalogJanitor() throws IOException {
361 return admin.runCatalogJanitor();
364 public boolean isCatalogJanitorEnabled() throws IOException {
365 return admin.isCatalogJanitorEnabled();
368 public boolean cleanerChoreSwitch(boolean onOrOff) throws IOException {
369 return admin.cleanerChoreSwitch(onOrOff);
372 public boolean runCleanerChore() throws IOException {
373 return admin.runCleanerChore();
376 public boolean isCleanerChoreEnabled() throws IOException {
377 return admin.isCleanerChoreEnabled();
380 public Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible)
381 throws IOException {
382 return admin.mergeRegionsAsync(nameofRegionsToMerge, forcible);
385 public void split(TableName tableName) throws IOException {
386 admin.split(tableName);
389 public void split(TableName tableName, byte[] splitPoint) throws IOException {
390 admin.split(tableName, splitPoint);
393 public Future<Void> splitRegionAsync(byte[] regionName) throws IOException {
394 return admin.splitRegionAsync(regionName);
397 public Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException {
398 return admin.splitRegionAsync(regionName, splitPoint);
401 public Future<Void> modifyTableAsync(TableDescriptor td) throws IOException {
402 return admin.modifyTableAsync(td);
405 public void shutdown() throws IOException {
406 admin.shutdown();
409 public void stopMaster() throws IOException {
410 admin.stopMaster();
413 public boolean isMasterInMaintenanceMode() throws IOException {
414 return admin.isMasterInMaintenanceMode();
417 public void stopRegionServer(String hostnamePort) throws IOException {
418 admin.stopRegionServer(hostnamePort);
421 public ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException {
422 return admin.getClusterMetrics(options);
425 public List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException {
426 return admin.getRegionMetrics(serverName);
429 public List<RegionMetrics> getRegionMetrics(ServerName serverName, TableName tableName)
430 throws IOException {
431 return admin.getRegionMetrics(serverName, tableName);
434 public Configuration getConfiguration() {
435 return admin.getConfiguration();
438 public Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
439 return admin.createNamespaceAsync(descriptor);
442 public Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException {
443 return admin.modifyNamespaceAsync(descriptor);
446 public Future<Void> deleteNamespaceAsync(String name) throws IOException {
447 return admin.deleteNamespaceAsync(name);
450 public NamespaceDescriptor getNamespaceDescriptor(String name)
451 throws NamespaceNotFoundException, IOException {
452 return admin.getNamespaceDescriptor(name);
455 public String[] listNamespaces() throws IOException {
456 return admin.listNamespaces();
459 public NamespaceDescriptor[] listNamespaceDescriptors() throws IOException {
460 return admin.listNamespaceDescriptors();
463 public List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException {
464 return admin.listTableDescriptorsByNamespace(name);
467 public TableName[] listTableNamesByNamespace(String name) throws IOException {
468 return admin.listTableNamesByNamespace(name);
471 public List<RegionInfo> getRegions(TableName tableName) throws IOException {
472 return admin.getRegions(tableName);
475 public void close() {
476 admin.close();
479 public List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) throws IOException {
480 return admin.listTableDescriptors(tableNames);
483 public Future<Boolean> abortProcedureAsync(long procId, boolean mayInterruptIfRunning)
484 throws IOException {
485 return admin.abortProcedureAsync(procId, mayInterruptIfRunning);
488 public String getProcedures() throws IOException {
489 return admin.getProcedures();
492 public String getLocks() throws IOException {
493 return admin.getLocks();
496 public void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException {
497 admin.rollWALWriter(serverName);
500 public CompactionState getCompactionState(TableName tableName) throws IOException {
501 return admin.getCompactionState(tableName);
504 public CompactionState getCompactionState(TableName tableName, CompactType compactType)
505 throws IOException {
506 return admin.getCompactionState(tableName, compactType);
509 public CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException {
510 return admin.getCompactionStateForRegion(regionName);
513 public long getLastMajorCompactionTimestamp(TableName tableName) throws IOException {
514 return admin.getLastMajorCompactionTimestamp(tableName);
517 public long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException {
518 return admin.getLastMajorCompactionTimestampForRegion(regionName);
521 public void snapshot(SnapshotDescription snapshot)
522 throws IOException, SnapshotCreationException, IllegalArgumentException {
523 admin.snapshot(snapshot);
526 public Future<Void> snapshotAsync(SnapshotDescription snapshot)
527 throws IOException, SnapshotCreationException {
528 return admin.snapshotAsync(snapshot);
531 public boolean isSnapshotFinished(SnapshotDescription snapshot)
532 throws IOException, HBaseSnapshotException, UnknownSnapshotException {
533 return admin.isSnapshotFinished(snapshot);
536 public void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException {
537 admin.restoreSnapshot(snapshotName);
540 public void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, boolean restoreAcl)
541 throws IOException, RestoreSnapshotException {
542 admin.restoreSnapshot(snapshotName, takeFailSafeSnapshot, restoreAcl);
545 public Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName,
546 boolean restoreAcl, String customSFT)
547 throws IOException, TableExistsException, RestoreSnapshotException {
548 return admin.cloneSnapshotAsync(snapshotName, tableName, restoreAcl, customSFT);
551 public void execProcedure(String signature, String instance, Map<String, String> props)
552 throws IOException {
553 admin.execProcedure(signature, instance, props);
556 public byte[] execProcedureWithReturn(String signature, String instance,
557 Map<String, String> props) throws IOException {
558 return admin.execProcedureWithReturn(signature, instance, props);
561 public boolean isProcedureFinished(String signature, String instance, Map<String, String> props)
562 throws IOException {
563 return admin.isProcedureFinished(signature, instance, props);
566 public List<SnapshotDescription> listSnapshots() throws IOException {
567 return admin.listSnapshots();
570 public List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException {
571 return admin.listSnapshots(pattern);
574 public List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
575 Pattern snapshotNamePattern) throws IOException {
576 return admin.listTableSnapshots(tableNamePattern, snapshotNamePattern);
579 public void deleteSnapshot(String snapshotName) throws IOException {
580 admin.deleteSnapshot(snapshotName);
583 public void deleteSnapshots(Pattern pattern) throws IOException {
584 admin.deleteSnapshots(pattern);
587 public void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern)
588 throws IOException {
589 admin.deleteTableSnapshots(tableNamePattern, snapshotNamePattern);
592 public void setQuota(QuotaSettings quota) throws IOException {
593 admin.setQuota(quota);
596 public List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException {
597 return admin.getQuota(filter);
600 public CoprocessorRpcChannel coprocessorService() {
601 return admin.coprocessorService();
604 public CoprocessorRpcChannel coprocessorService(ServerName serverName) {
605 return admin.coprocessorService(serverName);
608 public void updateConfiguration(ServerName server) throws IOException {
609 admin.updateConfiguration(server);
612 public void updateConfiguration() throws IOException {
613 admin.updateConfiguration();
616 public void updateConfiguration(String groupName) throws IOException {
617 admin.updateConfiguration(groupName);
620 public List<SecurityCapability> getSecurityCapabilities() throws IOException {
621 return admin.getSecurityCapabilities();
624 public boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException {
625 return admin.splitSwitch(enabled, synchronous);
628 public boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException {
629 return admin.mergeSwitch(enabled, synchronous);
632 public boolean isSplitEnabled() throws IOException {
633 return admin.isSplitEnabled();
636 public boolean isMergeEnabled() throws IOException {
637 return admin.isMergeEnabled();
640 public Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig,
641 boolean enabled) throws IOException {
642 return admin.addReplicationPeerAsync(peerId, peerConfig, enabled);
645 public Future<Void> removeReplicationPeerAsync(String peerId) throws IOException {
646 return admin.removeReplicationPeerAsync(peerId);
649 public Future<Void> enableReplicationPeerAsync(String peerId) throws IOException {
650 return admin.enableReplicationPeerAsync(peerId);
653 public Future<Void> disableReplicationPeerAsync(String peerId) throws IOException {
654 return admin.disableReplicationPeerAsync(peerId);
657 public ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException {
658 return admin.getReplicationPeerConfig(peerId);
661 public Future<Void> updateReplicationPeerConfigAsync(String peerId,
662 ReplicationPeerConfig peerConfig) throws IOException {
663 return admin.updateReplicationPeerConfigAsync(peerId, peerConfig);
666 public List<ReplicationPeerDescription> listReplicationPeers() throws IOException {
667 return admin.listReplicationPeers();
670 public List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException {
671 return admin.listReplicationPeers(pattern);
674 public Future<Void> transitReplicationPeerSyncReplicationStateAsync(String peerId,
675 SyncReplicationState state) throws IOException {
676 return admin.transitReplicationPeerSyncReplicationStateAsync(peerId, state);
679 public void decommissionRegionServers(List<ServerName> servers, boolean offload)
680 throws IOException {
681 admin.decommissionRegionServers(servers, offload);
684 public List<ServerName> listDecommissionedRegionServers() throws IOException {
685 return admin.listDecommissionedRegionServers();
688 public void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames)
689 throws IOException {
690 admin.recommissionRegionServer(server, encodedRegionNames);
693 public List<TableCFs> listReplicatedTableCFs() throws IOException {
694 return admin.listReplicatedTableCFs();
697 public void enableTableReplication(TableName tableName) throws IOException {
698 admin.enableTableReplication(tableName);
701 public void disableTableReplication(TableName tableName) throws IOException {
702 admin.disableTableReplication(tableName);
705 public void clearCompactionQueues(ServerName serverName, Set<String> queues)
706 throws IOException, InterruptedException {
707 admin.clearCompactionQueues(serverName, queues);
710 public List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException {
711 return admin.clearDeadServers(servers);
714 public void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits)
715 throws IOException {
716 admin.cloneTableSchema(tableName, newTableName, preserveSplits);
719 public boolean switchRpcThrottle(boolean enable) throws IOException {
720 return admin.switchRpcThrottle(enable);
723 public boolean isRpcThrottleEnabled() throws IOException {
724 return admin.isRpcThrottleEnabled();
727 public boolean exceedThrottleQuotaSwitch(boolean enable) throws IOException {
728 return admin.exceedThrottleQuotaSwitch(enable);
731 public Map<TableName, Long> getSpaceQuotaTableSizes() throws IOException {
732 return admin.getSpaceQuotaTableSizes();
735 public Map<TableName, ? extends SpaceQuotaSnapshotView>
736 getRegionServerSpaceQuotaSnapshots(ServerName serverName) throws IOException {
737 return admin.getRegionServerSpaceQuotaSnapshots(serverName);
740 public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(String namespace) throws IOException {
741 return admin.getCurrentSpaceQuotaSnapshot(namespace);
744 public SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(TableName tableName)
745 throws IOException {
746 return admin.getCurrentSpaceQuotaSnapshot(tableName);
749 public void grant(UserPermission userPermission, boolean mergeExistingPermissions)
750 throws IOException {
751 admin.grant(userPermission, mergeExistingPermissions);
754 public void revoke(UserPermission userPermission) throws IOException {
755 admin.revoke(userPermission);
758 public List<UserPermission>
759 getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) throws IOException {
760 return admin.getUserPermissions(getUserPermissionsRequest);
763 public List<Boolean> hasUserPermissions(String userName, List<Permission> permissions)
764 throws IOException {
765 return admin.hasUserPermissions(userName, permissions);
768 public boolean snapshotCleanupSwitch(boolean on, boolean synchronous) throws IOException {
769 return admin.snapshotCleanupSwitch(on, synchronous);
772 public boolean isSnapshotCleanupEnabled() throws IOException {
773 return admin.isSnapshotCleanupEnabled();
776 public void addRSGroup(String groupName) throws IOException {
777 admin.addRSGroup(groupName);
778 verify();
781 public RSGroupInfo getRSGroup(String groupName) throws IOException {
782 return admin.getRSGroup(groupName);
785 public RSGroupInfo getRSGroup(Address hostPort) throws IOException {
786 return admin.getRSGroup(hostPort);
789 public RSGroupInfo getRSGroup(TableName tableName) throws IOException {
790 return admin.getRSGroup(tableName);
793 public List<RSGroupInfo> listRSGroups() throws IOException {
794 return admin.listRSGroups();
797 @Override
798 public List<TableName> listTablesInRSGroup(String groupName) throws IOException {
799 return admin.listTablesInRSGroup(groupName);
802 @Override
803 public Pair<List<String>, List<TableName>>
804 getConfiguredNamespacesAndTablesInRSGroup(String groupName) throws IOException {
805 return admin.getConfiguredNamespacesAndTablesInRSGroup(groupName);
808 public void removeRSGroup(String groupName) throws IOException {
809 admin.removeRSGroup(groupName);
810 verify();
813 public void removeServersFromRSGroup(Set<Address> servers) throws IOException {
814 admin.removeServersFromRSGroup(servers);
815 verify();
818 public void moveServersToRSGroup(Set<Address> servers, String targetGroup) throws IOException {
819 admin.moveServersToRSGroup(servers, targetGroup);
820 verify();
823 public void setRSGroup(Set<TableName> tables, String groupName) throws IOException {
824 admin.setRSGroup(tables, groupName);
825 verify();
828 public BalanceResponse balanceRSGroup(String groupName, BalanceRequest request) throws IOException {
829 return admin.balanceRSGroup(groupName, request);
832 @Override
833 public void renameRSGroup(String oldName, String newName) throws IOException {
834 admin.renameRSGroup(oldName, newName);
835 verify();
838 @Override
839 public void updateRSGroupConfig(String groupName, Map<String, String> configuration)
840 throws IOException {
841 admin.updateRSGroupConfig(groupName, configuration);
842 verify();
845 @Override
846 public List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType,
847 ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException {
848 return admin.getLogEntries(serverNames, logType, serverType, limit, filterParams);
851 private void verify() throws IOException {
852 Map<String, RSGroupInfo> groupMap = Maps.newHashMap();
853 Set<RSGroupInfo> zList = Sets.newHashSet();
854 List<TableDescriptor> tds = new ArrayList<>();
855 try (Admin admin = conn.getAdmin()) {
856 tds.addAll(admin.listTableDescriptors());
857 tds.addAll(admin.listTableDescriptorsByNamespace(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME));
859 SortedSet<Address> lives = Sets.newTreeSet();
860 for (ServerName sn : conn.getAdmin().getClusterMetrics().getLiveServerMetrics().keySet()) {
861 lives.add(sn.getAddress());
863 for (ServerName sn : conn.getAdmin().listDecommissionedRegionServers()) {
864 lives.remove(sn.getAddress());
866 try (Table table = conn.getTable(RSGroupInfoManagerImpl.RSGROUP_TABLE_NAME);
867 ResultScanner scanner = table.getScanner(new Scan())) {
868 for (;;) {
869 Result result = scanner.next();
870 if (result == null) {
871 break;
873 RSGroupProtos.RSGroupInfo proto = RSGroupProtos.RSGroupInfo.parseFrom(result.getValue(
874 RSGroupInfoManagerImpl.META_FAMILY_BYTES, RSGroupInfoManagerImpl.META_QUALIFIER_BYTES));
875 RSGroupInfo rsGroupInfo = ProtobufUtil.toGroupInfo(proto);
876 groupMap.put(proto.getName(), RSGroupUtil.fillTables(rsGroupInfo, tds));
877 for (Address address : rsGroupInfo.getServers()) {
878 lives.remove(address);
882 SortedSet<TableName> tables = Sets.newTreeSet();
883 for (TableDescriptor td : conn.getAdmin().listTableDescriptors(Pattern.compile(".*"), true)) {
884 String groupName = td.getRegionServerGroup().orElse(RSGroupInfo.DEFAULT_GROUP);
885 if (groupName.equals(RSGroupInfo.DEFAULT_GROUP)) {
886 tables.add(td.getTableName());
890 groupMap.put(RSGroupInfo.DEFAULT_GROUP,
891 new RSGroupInfo(RSGroupInfo.DEFAULT_GROUP, lives, tables));
892 assertEquals(Sets.newHashSet(groupMap.values()), Sets.newHashSet(admin.listRSGroups()));
893 try {
894 String groupBasePath = ZNodePaths.joinZNode(zkw.getZNodePaths().baseZNode, "rsgroup");
895 for (String znode : ZKUtil.listChildrenNoWatch(zkw, groupBasePath)) {
896 byte[] data = ZKUtil.getData(zkw, ZNodePaths.joinZNode(groupBasePath, znode));
897 if (data.length > 0) {
898 ProtobufUtil.expectPBMagicPrefix(data);
899 ByteArrayInputStream bis =
900 new ByteArrayInputStream(data, ProtobufUtil.lengthOfPBMagic(), data.length);
901 RSGroupInfo rsGroupInfo =
902 ProtobufUtil.toGroupInfo(RSGroupProtos.RSGroupInfo.parseFrom(bis));
903 zList.add(RSGroupUtil.fillTables(rsGroupInfo, tds));
906 groupMap.remove(RSGroupInfo.DEFAULT_GROUP);
907 assertEquals(zList.size(), groupMap.size());
908 for (RSGroupInfo rsGroupInfo : zList) {
909 assertTrue(groupMap.get(rsGroupInfo.getName()).equals(rsGroupInfo));
911 } catch (KeeperException e) {
912 throw new IOException("ZK verification failed", e);
913 } catch (DeserializationException e) {
914 throw new IOException("ZK verification failed", e);
915 } catch (InterruptedException e) {
916 throw new IOException("ZK verification failed", e);
920 @Override
921 public List<Boolean> clearSlowLogResponses(Set<ServerName> serverNames) throws IOException {
922 return admin.clearSlowLogResponses(serverNames);
925 @Override
926 public Future<Void> modifyColumnFamilyStoreFileTrackerAsync(TableName tableName, byte[] family,
927 String dstSFT) throws IOException {
928 return admin.modifyColumnFamilyStoreFileTrackerAsync(tableName, family, dstSFT);
931 @Override
932 public Future<Void> modifyTableStoreFileTrackerAsync(TableName tableName, String dstSFT)
933 throws IOException {
934 return admin.modifyTableStoreFileTrackerAsync(tableName, dstSFT);