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
98
99
100 public ReplicationAdmin(Configuration conf) throws IOException {
101 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
102 HConstants.REPLICATION_ENABLE_DEFAULT)) {
103 throw new RuntimeException("hbase.replication isn't true, please " +
104 "enable it in order to use replication");
105 }
106 this.connection = HConnectionManager.getConnection(conf);
107 ZooKeeperWatcher zkw = createZooKeeperWatcher();
108 try {
109 this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
110 this.replicationPeers.init();
111 this.replicationQueuesClient =
112 ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
113 this.replicationQueuesClient.init();
114
115 } catch (ReplicationException e) {
116 throw new IOException("Error initializing the replication admin client.", e);
117 }
118 }
119
120 private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
121 return new ZooKeeperWatcher(connection.getConfiguration(),
122 "Replication Admin", new Abortable() {
123 @Override
124 public void abort(String why, Throwable e) {
125 LOG.error(why, e);
126 System.exit(1);
127 }
128
129 @Override
130 public boolean isAborted() {
131 return false;
132 }
133
134 });
135 }
136
137
138
139
140
141
142
143
144
145
146 @Deprecated
147 public void addPeer(String id, String clusterKey) throws ReplicationException {
148 this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey), null);
149 }
150
151 @Deprecated
152 public void addPeer(String id, String clusterKey, String tableCFs)
153 throws ReplicationException {
154 this.replicationPeers.addPeer(id,
155 new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs);
156 }
157
158
159
160
161
162
163
164
165
166
167 public void addPeer(String id, ReplicationPeerConfig peerConfig,
168 Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
169 this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs));
170 }
171
172 @VisibleForTesting
173 static String getTableCfsStr(Map<TableName, ? extends Collection<String>> tableCfs) {
174 String tableCfsStr = null;
175 if (tableCfs != null) {
176
177 StringBuilder builder = new StringBuilder();
178 for (Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
179 if (builder.length() > 0) {
180 builder.append(";");
181 }
182 builder.append(entry.getKey());
183 if (entry.getValue() != null && !entry.getValue().isEmpty()) {
184 builder.append(":");
185 builder.append(StringUtils.join(entry.getValue(), ","));
186 }
187 }
188 tableCfsStr = builder.toString();
189 }
190 return tableCfsStr;
191 }
192
193
194
195
196
197 public void removePeer(String id) throws ReplicationException {
198 this.replicationPeers.removePeer(id);
199 }
200
201
202
203
204
205 public void enablePeer(String id) throws ReplicationException {
206 this.replicationPeers.enablePeer(id);
207 }
208
209
210
211
212
213 public void disablePeer(String id) throws ReplicationException {
214 this.replicationPeers.disablePeer(id);
215 }
216
217
218
219
220
221 public int getPeersCount() {
222 return this.replicationPeers.getAllPeerIds().size();
223 }
224
225
226
227
228
229
230 @Deprecated
231 public Map<String, String> listPeers() {
232 Map<String, ReplicationPeerConfig> peers = this.listPeerConfigs();
233 Map<String, String> ret = new HashMap<String, String>(peers.size());
234
235 for (Map.Entry<String, ReplicationPeerConfig> entry : peers.entrySet()) {
236 ret.put(entry.getKey(), entry.getValue().getClusterKey());
237 }
238 return ret;
239 }
240
241 public Map<String, ReplicationPeerConfig> listPeerConfigs() {
242 return this.replicationPeers.getAllPeerConfigs();
243 }
244
245 public ReplicationPeerConfig getPeerConfig(String id) throws ReplicationException {
246 return this.replicationPeers.getReplicationPeerConfig(id);
247 }
248
249
250
251
252
253 public String getPeerTableCFs(String id) throws ReplicationException {
254 return this.replicationPeers.getPeerTableCFsConfig(id);
255 }
256
257
258
259
260
261
262 @Deprecated
263 public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException {
264 this.replicationPeers.setPeerTableCFsConfig(id, tableCFs);
265 }
266
267
268
269
270
271
272
273
274
275 public void setPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
276 throws ReplicationException {
277 this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs));
278 }
279
280
281
282
283
284
285
286 public boolean getPeerState(String id) throws ReplicationException {
287 return this.replicationPeers.getStatusOfPeerFromBackingStore(id);
288 }
289
290 @Override
291 public void close() throws IOException {
292 if (this.connection != null) {
293 this.connection.close();
294 }
295 }
296
297
298
299
300
301
302
303
304
305
306
307
308
309 public List<HashMap<String, String>> listReplicated() throws IOException {
310 List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
311 HTableDescriptor[] tables = this.connection.listTables();
312
313 for (HTableDescriptor table : tables) {
314 HColumnDescriptor[] columns = table.getColumnFamilies();
315 String tableName = table.getNameAsString();
316 for (HColumnDescriptor column : columns) {
317 if (column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) {
318
319 HashMap<String, String> replicationEntry = new HashMap<String, String>();
320 replicationEntry.put(TNAME, tableName);
321 replicationEntry.put(CFNAME, column.getNameAsString());
322 replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
323 replicationColFams.add(replicationEntry);
324 }
325 }
326 }
327
328 return replicationColFams;
329 }
330 }