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.replication;
20
21 import java.io.Closeable;
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.Collection;
25 import java.util.HashMap;
26 import java.util.List;
27 import java.util.Map;
28 import java.util.Map.Entry;
29 import org.apache.commons.lang.StringUtils;
30 import org.apache.commons.logging.Log;
31 import org.apache.commons.logging.LogFactory;
32 import org.apache.hadoop.hbase.classification.InterfaceAudience;
33 import org.apache.hadoop.hbase.classification.InterfaceStability;
34 import org.apache.hadoop.conf.Configuration;
35 import org.apache.hadoop.hbase.Abortable;
36 import org.apache.hadoop.hbase.HColumnDescriptor;
37 import org.apache.hadoop.hbase.HConstants;
38 import org.apache.hadoop.hbase.HTableDescriptor;
39 import org.apache.hadoop.hbase.TableName;
40 import org.apache.hadoop.hbase.client.HConnection;
41 import org.apache.hadoop.hbase.client.HConnectionManager;
42 import org.apache.hadoop.hbase.replication.ReplicationException;
43 import org.apache.hadoop.hbase.replication.ReplicationFactory;
44 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
45 import org.apache.hadoop.hbase.replication.ReplicationPeers;
46 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
47 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
48
49 import com.google.common.annotations.VisibleForTesting;
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74 @InterfaceAudience.Public
75 @InterfaceStability.Evolving
76 public class ReplicationAdmin implements Closeable {
77 private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);
78
79 public static final String TNAME = "tableName";
80 public static final String CFNAME = "columnFamlyName";
81
82
83
84 public static final String REPLICATIONTYPE = "replicationType";
85 public static final String REPLICATIONGLOBAL = Integer
86 .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
87
88 private final HConnection connection;
89
90
91 private final ReplicationQueuesClient replicationQueuesClient;
92 private final ReplicationPeers replicationPeers;
93
94
95
96
97 private final ZooKeeperWatcher zkw;
98
99
100
101
102
103
104
105 public ReplicationAdmin(Configuration conf) throws IOException {
106 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
107 HConstants.REPLICATION_ENABLE_DEFAULT)) {
108 throw new RuntimeException("hbase.replication isn't true, please " +
109 "enable it in order to use replication");
110 }
111 this.connection = HConnectionManager.getConnection(conf);
112 zkw = createZooKeeperWatcher();
113 try {
114 this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
115 this.replicationPeers.init();
116 this.replicationQueuesClient =
117 ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
118 this.replicationQueuesClient.init();
119
120 } catch (ReplicationException e) {
121 throw new IOException("Error initializing the replication admin client.", e);
122 }
123 }
124
125 private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
126
127 return new ZooKeeperWatcher(connection.getConfiguration(), "ReplicationAdmin", new Abortable() {
128 @Override
129 public void abort(String why, Throwable e) {
130 LOG.error(why, e);
131
132
133 }
134
135 @Override
136 public boolean isAborted() {
137 return false;
138 }
139 });
140 }
141
142
143
144
145
146
147
148
149
150
151 @Deprecated
152 public void addPeer(String id, String clusterKey) throws ReplicationException {
153 this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey), null);
154 }
155
156 @Deprecated
157 public void addPeer(String id, String clusterKey, String tableCFs)
158 throws ReplicationException {
159 this.replicationPeers.addPeer(id,
160 new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs);
161 }
162
163
164
165
166
167
168
169
170
171
172 public void addPeer(String id, ReplicationPeerConfig peerConfig,
173 Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
174 this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs));
175 }
176
177 @VisibleForTesting
178 static String getTableCfsStr(Map<TableName, ? extends Collection<String>> tableCfs) {
179 String tableCfsStr = null;
180 if (tableCfs != null) {
181
182 StringBuilder builder = new StringBuilder();
183 for (Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
184 if (builder.length() > 0) {
185 builder.append(";");
186 }
187 builder.append(entry.getKey());
188 if (entry.getValue() != null && !entry.getValue().isEmpty()) {
189 builder.append(":");
190 builder.append(StringUtils.join(entry.getValue(), ","));
191 }
192 }
193 tableCfsStr = builder.toString();
194 }
195 return tableCfsStr;
196 }
197
198
199
200
201
202 public void removePeer(String id) throws ReplicationException {
203 this.replicationPeers.removePeer(id);
204 }
205
206
207
208
209
210 public void enablePeer(String id) throws ReplicationException {
211 this.replicationPeers.enablePeer(id);
212 }
213
214
215
216
217
218 public void disablePeer(String id) throws ReplicationException {
219 this.replicationPeers.disablePeer(id);
220 }
221
222
223
224
225
226 public int getPeersCount() {
227 return this.replicationPeers.getAllPeerIds().size();
228 }
229
230
231
232
233
234
235 @Deprecated
236 public Map<String, String> listPeers() {
237 Map<String, ReplicationPeerConfig> peers = this.listPeerConfigs();
238 Map<String, String> ret = new HashMap<String, String>(peers.size());
239
240 for (Map.Entry<String, ReplicationPeerConfig> entry : peers.entrySet()) {
241 ret.put(entry.getKey(), entry.getValue().getClusterKey());
242 }
243 return ret;
244 }
245
246 public Map<String, ReplicationPeerConfig> listPeerConfigs() {
247 return this.replicationPeers.getAllPeerConfigs();
248 }
249
250 public ReplicationPeerConfig getPeerConfig(String id) throws ReplicationException {
251 return this.replicationPeers.getReplicationPeerConfig(id);
252 }
253
254
255
256
257
258 public String getPeerTableCFs(String id) throws ReplicationException {
259 return this.replicationPeers.getPeerTableCFsConfig(id);
260 }
261
262
263
264
265
266
267 @Deprecated
268 public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException {
269 this.replicationPeers.setPeerTableCFsConfig(id, tableCFs);
270 }
271
272
273
274
275
276
277
278
279
280 public void setPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
281 throws ReplicationException {
282 this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs));
283 }
284
285
286
287
288
289
290
291 public boolean getPeerState(String id) throws ReplicationException {
292 return this.replicationPeers.getStatusOfPeerFromBackingStore(id);
293 }
294
295 @Override
296 public void close() throws IOException {
297 if (this.zkw != null) {
298 this.zkw.close();
299 }
300 if (this.connection != null) {
301 this.connection.close();
302 }
303 }
304
305
306
307
308
309
310
311
312
313
314
315
316
317 public List<HashMap<String, String>> listReplicated() throws IOException {
318 List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
319 HTableDescriptor[] tables = this.connection.listTables();
320
321 for (HTableDescriptor table : tables) {
322 HColumnDescriptor[] columns = table.getColumnFamilies();
323 String tableName = table.getNameAsString();
324 for (HColumnDescriptor column : columns) {
325 if (column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) {
326
327 HashMap<String, String> replicationEntry = new HashMap<String, String>();
328 replicationEntry.put(TNAME, tableName);
329 replicationEntry.put(CFNAME, column.getNameAsString());
330 replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
331 replicationColFams.add(replicationEntry);
332 }
333 }
334 }
335
336 return replicationColFams;
337 }
338 }