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.client.replication;
21
22 import java.io.Closeable;
23 import java.io.IOException;
24 import java.util.Map;
25 import java.util.List;
26 import java.util.ArrayList;
27 import java.util.HashMap;
28 import java.lang.Integer;
29
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.client.HConnection;
33 import org.apache.hadoop.hbase.client.HConnectionManager;
34 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
35 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
36 import org.apache.zookeeper.KeeperException;
37 import org.apache.hadoop.hbase.HTableDescriptor;
38 import org.apache.hadoop.hbase.HColumnDescriptor;
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73 public class ReplicationAdmin implements Closeable {
74
75 public static final String TNAME = "tableName";
76 public static final String CFNAME = "columnFamlyName";
77
78
79
80 public static final String REPLICATIONTYPE = "replicationType";
81 public static final String REPLICATIONGLOBAL = Integer
82 .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
83
84 private final ReplicationZookeeper replicationZk;
85 private final HConnection connection;
86
87
88
89
90
91
92
93 public ReplicationAdmin(Configuration conf) throws IOException {
94 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
95 throw new RuntimeException("hbase.replication isn't true, please " +
96 "enable it in order to use replication");
97 }
98 this.connection = HConnectionManager.getConnection(conf);
99 ZooKeeperWatcher zkw = this.connection.getZooKeeperWatcher();
100 try {
101 this.replicationZk = new ReplicationZookeeper(this.connection, conf, zkw);
102 } catch (KeeperException e) {
103 throw new IOException("Unable setup the ZooKeeper connection", e);
104 }
105 }
106
107
108
109
110
111
112
113
114
115 public void addPeer(String id, String clusterKey) throws IOException {
116 this.replicationZk.addPeer(id, clusterKey);
117 }
118
119
120
121
122
123 public void removePeer(String id) throws IOException {
124 this.replicationZk.removePeer(id);
125 }
126
127
128
129
130
131 public void enablePeer(String id) throws IOException {
132 this.replicationZk.enablePeer(id);
133 }
134
135
136
137
138
139 public void disablePeer(String id) throws IOException {
140 this.replicationZk.disablePeer(id);
141 }
142
143
144
145
146
147 public int getPeersCount() {
148 return this.replicationZk.listPeersIdsAndWatch().size();
149 }
150
151
152
153
154
155 public Map<String, String> listPeers() {
156 return this.replicationZk.listPeers();
157 }
158
159
160
161
162
163
164
165 public String getPeerState(String id) throws IOException {
166 try {
167 return this.replicationZk.getPeerState(id).name();
168 } catch (KeeperException e) {
169 throw new IOException("Couldn't get the state of the peer " + id, e);
170 }
171 }
172
173
174
175
176
177
178 public boolean getReplicating() throws IOException {
179 try {
180 return this.replicationZk.getReplication();
181 } catch (KeeperException e) {
182 throw new IOException("Couldn't get the replication status");
183 }
184 }
185
186
187
188
189
190
191
192 public boolean setReplicating(boolean newState) throws IOException {
193 boolean prev = true;
194 try {
195 prev = getReplicating();
196 this.replicationZk.setReplicating(newState);
197 } catch (KeeperException e) {
198 throw new IOException("Unable to set the replication state", e);
199 }
200 return prev;
201 }
202
203
204
205
206
207 ReplicationZookeeper getReplicationZk() {
208 return replicationZk;
209 }
210
211 @Override
212 public void close() throws IOException {
213 if (this.connection != null) {
214 this.connection.close();
215 }
216 }
217
218
219
220
221
222
223
224
225
226
227
228
229 public List<HashMap<String, String>> listReplicated() throws IOException {
230 List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
231 HTableDescriptor[] tables = this.connection.listTables();
232
233 for (HTableDescriptor table : tables) {
234 HColumnDescriptor[] columns = table.getColumnFamilies();
235 String tableName = table.getNameAsString();
236 for (HColumnDescriptor column : columns) {
237 if (column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) {
238
239 HashMap<String, String> replicationEntry = new HashMap<String, String>();
240 replicationEntry.put(TNAME, tableName);
241 replicationEntry.put(CFNAME, column.getNameAsString());
242 replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
243 replicationColFams.add(replicationEntry);
244 }
245 }
246 }
247
248 return replicationColFams;
249 }
250 }