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.conf.Configuration;
33 import org.apache.hadoop.hbase.Abortable;
34 import org.apache.hadoop.hbase.HColumnDescriptor;
35 import org.apache.hadoop.hbase.HConstants;
36 import org.apache.hadoop.hbase.HTableDescriptor;
37 import org.apache.hadoop.hbase.TableName;
38 import org.apache.hadoop.hbase.TableNotFoundException;
39 import org.apache.hadoop.hbase.classification.InterfaceAudience;
40 import org.apache.hadoop.hbase.classification.InterfaceStability;
41 import org.apache.hadoop.hbase.client.HBaseAdmin;
42 import org.apache.hadoop.hbase.client.HConnection;
43 import org.apache.hadoop.hbase.client.HConnectionManager;
44 import org.apache.hadoop.hbase.client.HTable;
45 import org.apache.hadoop.hbase.replication.ReplicationException;
46 import org.apache.hadoop.hbase.replication.ReplicationFactory;
47 import org.apache.hadoop.hbase.replication.ReplicationPeer;
48 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
49 import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
50 import org.apache.hadoop.hbase.replication.ReplicationPeers;
51 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
52 import org.apache.hadoop.hbase.util.Pair;
53 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
54 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
55 import org.apache.zookeeper.KeeperException;
56 import org.apache.zookeeper.data.Stat;
57
58 import com.google.common.annotations.VisibleForTesting;
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83 @InterfaceAudience.Public
84 @InterfaceStability.Evolving
85 public class ReplicationAdmin implements Closeable {
86 private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);
87
88 public static final String TNAME = "tableName";
89 public static final String CFNAME = "columnFamlyName";
90
91
92
93 public static final String REPLICATIONTYPE = "replicationType";
94 public static final String REPLICATIONGLOBAL = Integer
95 .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
96
97 private final HConnection connection;
98
99
100 private final ReplicationQueuesClient replicationQueuesClient;
101 private final ReplicationPeers replicationPeers;
102
103
104
105
106 private final ZooKeeperWatcher zkw;
107
108
109
110
111
112
113
114 public ReplicationAdmin(Configuration conf) throws IOException {
115 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
116 HConstants.REPLICATION_ENABLE_DEFAULT)) {
117 throw new RuntimeException("hbase.replication isn't true, please " +
118 "enable it in order to use replication");
119 }
120 this.connection = HConnectionManager.getConnection(conf);
121 try {
122 zkw = createZooKeeperWatcher();
123 try {
124 this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
125 this.replicationPeers.init();
126 this.replicationQueuesClient =
127 ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
128 this.replicationQueuesClient.init();
129 } catch (Exception exception) {
130 if (zkw != null) {
131 zkw.close();
132 }
133 throw exception;
134 }
135 } catch (Exception exception) {
136 if (connection != null) {
137 connection.close();
138 }
139 if (exception instanceof IOException) {
140 throw (IOException) exception;
141 } else if (exception instanceof RuntimeException) {
142 throw (RuntimeException) exception;
143 } else {
144 throw new IOException("Error initializing the replication admin client.", exception);
145 }
146 }
147 }
148
149 private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
150
151 return new ZooKeeperWatcher(connection.getConfiguration(), "ReplicationAdmin", new Abortable() {
152 @Override
153 public void abort(String why, Throwable e) {
154 LOG.error(why, e);
155
156
157 }
158
159 @Override
160 public boolean isAborted() {
161 return false;
162 }
163 });
164 }
165
166
167
168
169
170
171
172
173
174
175 @Deprecated
176 public void addPeer(String id, String clusterKey) throws ReplicationException {
177 this.addPeer(id, new ReplicationPeerConfig().setClusterKey(clusterKey), null);
178 }
179
180 @Deprecated
181 public void addPeer(String id, String clusterKey, String tableCFs)
182 throws ReplicationException {
183 this.replicationPeers.addPeer(id,
184 new ReplicationPeerConfig().setClusterKey(clusterKey), tableCFs);
185 }
186
187
188
189
190
191
192
193
194
195
196 public void addPeer(String id, ReplicationPeerConfig peerConfig,
197 Map<TableName, ? extends Collection<String>> tableCfs) throws ReplicationException {
198 this.replicationPeers.addPeer(id, peerConfig, getTableCfsStr(tableCfs));
199 }
200
201 @VisibleForTesting
202 static String getTableCfsStr(Map<TableName, ? extends Collection<String>> tableCfs) {
203 String tableCfsStr = null;
204 if (tableCfs != null) {
205
206 StringBuilder builder = new StringBuilder();
207 for (Entry<TableName, ? extends Collection<String>> entry : tableCfs.entrySet()) {
208 if (builder.length() > 0) {
209 builder.append(";");
210 }
211 builder.append(entry.getKey());
212 if (entry.getValue() != null && !entry.getValue().isEmpty()) {
213 builder.append(":");
214 builder.append(StringUtils.join(entry.getValue(), ","));
215 }
216 }
217 tableCfsStr = builder.toString();
218 }
219 return tableCfsStr;
220 }
221
222
223
224
225
226 public void removePeer(String id) throws ReplicationException {
227 this.replicationPeers.removePeer(id);
228 }
229
230
231
232
233
234 public void enablePeer(String id) throws ReplicationException {
235 this.replicationPeers.enablePeer(id);
236 }
237
238
239
240
241
242 public void disablePeer(String id) throws ReplicationException {
243 this.replicationPeers.disablePeer(id);
244 }
245
246
247
248
249
250 public int getPeersCount() {
251 return this.replicationPeers.getAllPeerIds().size();
252 }
253
254
255
256
257
258
259 @Deprecated
260 public Map<String, String> listPeers() {
261 Map<String, ReplicationPeerConfig> peers = this.listPeerConfigs();
262 Map<String, String> ret = new HashMap<String, String>(peers.size());
263
264 for (Map.Entry<String, ReplicationPeerConfig> entry : peers.entrySet()) {
265 ret.put(entry.getKey(), entry.getValue().getClusterKey());
266 }
267 return ret;
268 }
269
270 public Map<String, ReplicationPeerConfig> listPeerConfigs() {
271 return this.replicationPeers.getAllPeerConfigs();
272 }
273
274 public ReplicationPeerConfig getPeerConfig(String id) throws ReplicationException {
275 return this.replicationPeers.getReplicationPeerConfig(id);
276 }
277
278
279
280
281
282 public String getPeerTableCFs(String id) throws ReplicationException {
283 return this.replicationPeers.getPeerTableCFsConfig(id);
284 }
285
286
287
288
289
290
291 @Deprecated
292 public void setPeerTableCFs(String id, String tableCFs) throws ReplicationException {
293 this.replicationPeers.setPeerTableCFsConfig(id, tableCFs);
294 }
295
296
297
298
299
300
301
302
303
304 public void setPeerTableCFs(String id, Map<TableName, ? extends Collection<String>> tableCfs)
305 throws ReplicationException {
306 this.replicationPeers.setPeerTableCFsConfig(id, getTableCfsStr(tableCfs));
307 }
308
309
310
311
312
313
314
315 public boolean getPeerState(String id) throws ReplicationException {
316 return this.replicationPeers.getStatusOfPeerFromBackingStore(id);
317 }
318
319 @Override
320 public void close() throws IOException {
321 if (this.zkw != null) {
322 this.zkw.close();
323 }
324 if (this.connection != null) {
325 this.connection.close();
326 }
327 }
328
329
330
331
332
333
334
335
336
337
338
339
340
341 public List<HashMap<String, String>> listReplicated() throws IOException {
342 List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
343 HTableDescriptor[] tables = this.connection.listTables();
344
345 for (HTableDescriptor table : tables) {
346 HColumnDescriptor[] columns = table.getColumnFamilies();
347 String tableName = table.getNameAsString();
348 for (HColumnDescriptor column : columns) {
349 if (column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) {
350
351 HashMap<String, String> replicationEntry = new HashMap<String, String>();
352 replicationEntry.put(TNAME, tableName);
353 replicationEntry.put(CFNAME, column.getNameAsString());
354 replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
355 replicationColFams.add(replicationEntry);
356 }
357 }
358 }
359
360 return replicationColFams;
361 }
362
363
364
365
366
367
368 public void enableTableRep(final TableName tableName) throws IOException {
369 if (tableName == null) {
370 throw new IllegalArgumentException("Table name cannot be null");
371 }
372 HBaseAdmin admin = null;
373 try {
374 admin = new HBaseAdmin(this.connection.getConfiguration());
375 if (!admin.tableExists(tableName)) {
376 throw new TableNotFoundException("Table '" + tableName.getNamespaceAsString()
377 + "' does not exists.");
378 }
379 } finally {
380 try {
381 admin.close();
382 } catch (IOException e) {
383 LOG.warn("Failed to close admin connection.");
384 LOG.debug("Details on failure to close admin connection.", e);
385 }
386 }
387 byte[][] splits = getTableSplitRowKeys(tableName);
388 checkAndSyncTableDescToPeers(tableName, splits);
389 setTableRep(tableName, true);
390 }
391
392
393
394
395
396
397 public void disableTableRep(final TableName tableName) throws IOException {
398 if (tableName == null) {
399 throw new IllegalArgumentException("Table name is null");
400 }
401
402 HBaseAdmin admin = null;
403 try {
404 admin = new HBaseAdmin(this.connection.getConfiguration());
405 if (!admin.tableExists(tableName)) {
406 throw new TableNotFoundException("Table '" + tableName.getNamespaceAsString()
407 + "' does not exists.");
408 }
409 } finally {
410 try {
411 admin.close();
412 } catch (IOException e) {
413 LOG.warn("Failed to close admin connection.");
414 LOG.debug("Details on failure to close admin connection.", e);
415 }
416 }
417 setTableRep(tableName, false);
418 }
419
420
421
422
423
424
425
426 private byte[][] getTableSplitRowKeys(TableName tableName) throws IOException {
427 HTable table = null;
428 try {
429 table = new HTable(this.connection.getConfiguration(), tableName);
430 byte[][] startKeys = table.getStartKeys();
431 if (startKeys.length == 1) {
432 return null;
433 }
434 byte[][] splits = new byte[startKeys.length - 1][];
435 for (int i = 1; i < startKeys.length; i++) {
436 splits[i - 1] = startKeys[i];
437 }
438 return splits;
439 } finally {
440 if (table != null) {
441 try {
442 table.close();
443 } catch (IOException e) {
444 LOG.warn("Unable to close table");
445 }
446 }
447 }
448 }
449
450
451
452
453
454
455
456
457
458
459
460 private void checkAndSyncTableDescToPeers(final TableName tableName, final byte[][] splits)
461 throws IOException {
462 List<ReplicationPeer> repPeers = listValidReplicationPeers();
463 if (repPeers == null || repPeers.size() <= 0) {
464 throw new IllegalArgumentException("Found no peer cluster for replication.");
465 }
466 for (ReplicationPeer repPeer : repPeers) {
467 Configuration peerConf = repPeer.getConfiguration();
468 HTableDescriptor htd = null;
469 HBaseAdmin repHBaseAdmin = null;
470 try {
471 repHBaseAdmin = new HBaseAdmin(peerConf);
472 htd = this.connection.getHTableDescriptor(tableName);
473 HTableDescriptor peerHtd = null;
474 if (!repHBaseAdmin.tableExists(tableName)) {
475 repHBaseAdmin.createTable(htd, splits);
476 } else {
477 peerHtd = repHBaseAdmin.getTableDescriptor(tableName);
478 if (peerHtd == null) {
479 throw new IllegalArgumentException("Failed to get table descriptor for table "
480 + tableName.getNameAsString() + " from peer cluster " + repPeer.getId());
481 } else if (!peerHtd.equals(htd)) {
482 throw new IllegalArgumentException("Table " + tableName.getNameAsString()
483 + " exists in peer cluster " + repPeer.getId()
484 + ", but the table descriptors are not same when comapred with source cluster."
485 + " Thus can not enable the table's replication switch.");
486 }
487 }
488 } finally {
489 if (repHBaseAdmin != null) {
490 try {
491 repHBaseAdmin.close();
492 } catch (IOException e) {
493 LOG.warn("Failed to close admin connection.");
494 LOG.debug("Details on failure to close admin connection.", e);
495 }
496 }
497 }
498 }
499 }
500
501 private List<ReplicationPeer> listValidReplicationPeers() {
502 Map<String, ReplicationPeerConfig> peers = listPeerConfigs();
503 if (peers == null || peers.size() <= 0) {
504 return null;
505 }
506 List<ReplicationPeer> validPeers = new ArrayList<ReplicationPeer>(peers.size());
507 for (Entry<String, ReplicationPeerConfig> peerEntry : peers.entrySet()) {
508 String peerId = peerEntry.getKey();
509 String clusterKey = peerEntry.getValue().getClusterKey();
510 Configuration peerConf = new Configuration(this.connection.getConfiguration());
511 Stat s = null;
512 try {
513 ZKUtil.applyClusterKeyToConf(peerConf, clusterKey);
514 Pair<ReplicationPeerConfig, Configuration> pair = this.replicationPeers.getPeerConf(peerId);
515 ReplicationPeer peer = new ReplicationPeerZKImpl(peerConf, peerId, pair.getFirst());
516 s =
517 zkw.getRecoverableZooKeeper().exists(peerConf.get(HConstants.ZOOKEEPER_ZNODE_PARENT),
518 null);
519 if (null == s) {
520 LOG.info(peerId + ' ' + clusterKey + " is invalid now.");
521 continue;
522 }
523 validPeers.add(peer);
524 } catch (ReplicationException e) {
525 LOG.warn("Failed to get valid replication peers. "
526 + "Error connecting to peer cluster with peerId=" + peerId);
527 LOG.debug("Failure details to get valid replication peers.", e);
528 continue;
529 } catch (KeeperException e) {
530 LOG.warn("Failed to get valid replication peers. KeeperException code="
531 + e.code().intValue());
532 LOG.debug("Failure details to get valid replication peers.", e);
533 continue;
534 } catch (InterruptedException e) {
535 LOG.warn("Failed to get valid replication peers due to InterruptedException.");
536 LOG.debug("Failure details to get valid replication peers.", e);
537 Thread.currentThread().interrupt();
538 continue;
539 } catch (IOException e) {
540 LOG.warn("Failed to get valid replication peers due to IOException.");
541 LOG.debug("Failure details to get valid replication peers.", e);
542 continue;
543 }
544 }
545 return validPeers;
546 }
547
548
549
550
551
552
553
554 private void setTableRep(final TableName tableName, boolean isRepEnabled) throws IOException {
555 HBaseAdmin admin = null;
556 try {
557 admin = new HBaseAdmin(this.connection.getConfiguration());
558 HTableDescriptor htd = admin.getTableDescriptor(tableName);
559 if (isTableRepEnabled(htd) ^ isRepEnabled) {
560 boolean isOnlineSchemaUpdateEnabled =
561 this.connection.getConfiguration()
562 .getBoolean("hbase.online.schema.update.enable", true);
563 if (!isOnlineSchemaUpdateEnabled) {
564 admin.disableTable(tableName);
565 }
566 for (HColumnDescriptor hcd : htd.getFamilies()) {
567 hcd.setScope(isRepEnabled ? HConstants.REPLICATION_SCOPE_GLOBAL
568 : HConstants.REPLICATION_SCOPE_LOCAL);
569 }
570 admin.modifyTable(tableName, htd);
571 if (!isOnlineSchemaUpdateEnabled) {
572 admin.enableTable(tableName);
573 }
574 }
575 } finally {
576 if (admin != null) {
577 try {
578 admin.close();
579 } catch (IOException e) {
580 LOG.warn("Failed to close admin connection.");
581 LOG.debug("Details on failure to close admin connection.", e);
582 }
583 }
584 }
585 }
586
587
588
589
590
591 private boolean isTableRepEnabled(HTableDescriptor htd) {
592 for (HColumnDescriptor hcd : htd.getFamilies()) {
593 if (hcd.getScope() != HConstants.REPLICATION_SCOPE_GLOBAL) {
594 return false;
595 }
596 }
597 return true;
598 }
599 }