1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.rsgroup;
21
22 import com.google.common.collect.Maps;
23 import com.google.common.collect.Sets;
24
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.TableName;
27 import org.apache.hadoop.hbase.client.ConnectionFactory;
28 import org.apache.hadoop.hbase.client.Result;
29 import org.apache.hadoop.hbase.client.Scan;
30 import org.apache.hadoop.hbase.client.Table;
31 import org.apache.hadoop.hbase.exceptions.DeserializationException;
32 import org.apache.hadoop.hbase.net.Address;
33 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
34 import org.apache.hadoop.hbase.protobuf.generated.RSGroupProtos;
35 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
36 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
37 import org.apache.zookeeper.KeeperException;
38 import org.junit.Assert;
39
40 import java.io.ByteArrayInputStream;
41 import java.io.IOException;
42 import java.util.List;
43 import java.util.Map;
44 import java.util.Set;
45
46 public class VerifyingRSGroupAdminClient implements RSGroupAdmin {
47 private Table table;
48 private ZooKeeperWatcher zkw;
49 private RSGroupAdmin wrapped;
50
51 public VerifyingRSGroupAdminClient(RSGroupAdmin RSGroupAdmin, Configuration conf)
52 throws IOException {
53 wrapped = RSGroupAdmin;
54 table = ConnectionFactory.createConnection(conf).getTable(RSGroupInfoManager.RSGROUP_TABLE_NAME);
55 zkw = new ZooKeeperWatcher(conf, this.getClass().getSimpleName(), null);
56 }
57
58 @Override
59 public void addRSGroup(String groupName) throws IOException {
60 wrapped.addRSGroup(groupName);
61 verify();
62 }
63
64 @Override
65 public RSGroupInfo getRSGroupInfo(String groupName) throws IOException {
66 return wrapped.getRSGroupInfo(groupName);
67 }
68
69 @Override
70 public RSGroupInfo getRSGroupInfoOfTable(TableName tableName) throws IOException {
71 return wrapped.getRSGroupInfoOfTable(tableName);
72 }
73
74 @Override
75 public void moveServers(Set<Address> servers, String targetGroup) throws IOException {
76 wrapped.moveServers(servers, targetGroup);
77 verify();
78 }
79
80 @Override
81 public void moveTables(Set<TableName> tables, String targetGroup) throws IOException {
82 wrapped.moveTables(tables, targetGroup);
83 verify();
84 }
85
86 @Override
87 public void removeRSGroup(String name) throws IOException {
88 wrapped.removeRSGroup(name);
89 verify();
90 }
91
92 @Override
93 public boolean balanceRSGroup(String name) throws IOException {
94 return wrapped.balanceRSGroup(name);
95 }
96
97 @Override
98 public List<RSGroupInfo> listRSGroups() throws IOException {
99 return wrapped.listRSGroups();
100 }
101
102 @Override
103 public RSGroupInfo getRSGroupOfServer(Address server) throws IOException {
104 return wrapped.getRSGroupOfServer(server);
105 }
106
107 @Override
108 public void removeServers(Set<Address> servers) throws IOException {
109 wrapped.removeServers(servers);
110 verify();
111 }
112
113 @Override
114 public void renameRSGroup(String oldName, String newName) throws IOException {
115 wrapped.renameRSGroup(oldName, newName);
116 verify();
117 }
118
119 public void verify() throws IOException {
120 Map<String, RSGroupInfo> groupMap = Maps.newHashMap();
121 Set<RSGroupInfo> zList = Sets.newHashSet();
122
123 for (Result result : table.getScanner(new Scan())) {
124 RSGroupProtos.RSGroupInfo proto =
125 RSGroupProtos.RSGroupInfo.parseFrom(
126 result.getValue(
127 RSGroupInfoManager.META_FAMILY_BYTES,
128 RSGroupInfoManager.META_QUALIFIER_BYTES));
129 groupMap.put(proto.getName(), RSGroupProtobufUtil.toGroupInfo(proto));
130 }
131 Assert.assertEquals(Sets.newHashSet(groupMap.values()),
132 Sets.newHashSet(wrapped.listRSGroups()));
133 try {
134 String groupBasePath = ZKUtil.joinZNode(zkw.baseZNode, "rsgroup");
135 for(String znode: ZKUtil.listChildrenNoWatch(zkw, groupBasePath)) {
136 byte[] data = ZKUtil.getData(zkw, ZKUtil.joinZNode(groupBasePath, znode));
137 if(data.length > 0) {
138 ProtobufUtil.expectPBMagicPrefix(data);
139 ByteArrayInputStream bis = new ByteArrayInputStream(
140 data, ProtobufUtil.lengthOfPBMagic(), data.length);
141 zList.add(RSGroupProtobufUtil.toGroupInfo(RSGroupProtos.RSGroupInfo.parseFrom(bis)));
142 }
143 }
144 Assert.assertEquals(zList.size(), groupMap.size());
145 for(RSGroupInfo RSGroupInfo : zList) {
146 Assert.assertTrue(groupMap.get(RSGroupInfo.getName()).equals(RSGroupInfo));
147 }
148 } catch (KeeperException e) {
149 throw new IOException("ZK verification failed", e);
150 } catch (DeserializationException e) {
151 throw new IOException("ZK verification failed", e);
152 } catch (InterruptedException e) {
153 throw new IOException("ZK verification failed", e);
154 }
155 }
156
157 @Override
158 public void moveServersAndTables(Set<Address> servers, Set<TableName> tables,
159 String targetGroup) throws IOException {
160 wrapped.moveServersAndTables(servers, tables, targetGroup);
161 verify();
162 }
163
164 @Override
165 public void close() throws IOException {
166 }
167 }