1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication.master;
20
21 import org.apache.commons.logging.Log;
22 import org.apache.commons.logging.LogFactory;
23 import org.apache.hadoop.conf.Configuration;
24 import org.apache.hadoop.hbase.Abortable;
25 import org.apache.hadoop.hbase.HBaseConfiguration;
26 import org.apache.hadoop.hbase.classification.InterfaceAudience;
27 import org.apache.hadoop.hbase.classification.InterfaceStability;
28 import org.apache.hadoop.hbase.replication.ReplicationSerDeHelper;
29 import org.apache.hadoop.hbase.exceptions.DeserializationException;
30 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
31 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
32 import org.apache.hadoop.hbase.replication.ReplicationStateZKBase;
33 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
34 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
35 import org.apache.zookeeper.KeeperException;
36
37 import java.io.IOException;
38 import java.util.List;
39
40
41
42
43
44 @InterfaceAudience.Private
45 @InterfaceStability.Unstable
46 public class TableCFsUpdater extends ReplicationStateZKBase {
47
48 private static final Log LOG = LogFactory.getLog(TableCFsUpdater.class);
49
50 public TableCFsUpdater(ZooKeeperWatcher zookeeper,
51 Configuration conf, Abortable abortable) {
52 super(zookeeper, conf, abortable);
53 }
54
55 public void update() {
56 List<String> znodes = null;
57 try {
58 znodes = ZKUtil.listChildrenNoWatch(this.zookeeper, this.peersZNode);
59 } catch (KeeperException e) {
60 LOG.error("Failed to get peers znode", e);
61 }
62 if (znodes != null) {
63 for (String peerId : znodes) {
64 if (!update(peerId)) {
65 LOG.error("upgrade tableCFs failed for peerId=" + peerId);
66 }
67 }
68 }
69 }
70
71 public boolean update(String peerId) {
72 String tableCFsNode = getTableCFsNode(peerId);
73 try {
74 if (ZKUtil.checkExists(zookeeper, tableCFsNode) != -1) {
75 String peerNode = getPeerNode(peerId);
76 ReplicationPeerConfig rpc = getReplicationPeerConig(peerNode);
77
78 if (rpc.getTableCFsMap() == null || rpc.getTableCFsMap().size() == 0) {
79
80 LOG.info("copy tableCFs into peerNode:" + peerId);
81 ZooKeeperProtos.TableCF[] tableCFs =
82 ReplicationSerDeHelper.parseTableCFs(
83 ZKUtil.getData(this.zookeeper, tableCFsNode));
84 rpc.setTableCFsMap(ReplicationSerDeHelper.convert2Map(tableCFs));
85 ZKUtil.setData(this.zookeeper, peerNode,
86 ReplicationSerDeHelper.toByteArray(rpc));
87 } else {
88 LOG.info("No tableCFs in peerNode:" + peerId);
89 }
90 }
91 } catch (KeeperException e) {
92 LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
93 return false;
94 } catch (InterruptedException e) {
95 LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
96 return false;
97 } catch (IOException e) {
98 LOG.warn("NOTICE!! Update peerId failed, peerId=" + peerId, e);
99 return false;
100 }
101 return true;
102 }
103
104 private ReplicationPeerConfig getReplicationPeerConig(String peerNode)
105 throws KeeperException, InterruptedException {
106 byte[] data = null;
107 data = ZKUtil.getData(this.zookeeper, peerNode);
108 if (data == null) {
109 LOG.error("Could not get configuration for " +
110 "peer because it doesn't exist. peer=" + peerNode);
111 return null;
112 }
113 try {
114 return ReplicationSerDeHelper.parsePeerFrom(data);
115 } catch (DeserializationException e) {
116 LOG.warn("Failed to parse cluster key from peer=" + peerNode);
117 return null;
118 }
119 }
120
121 private static void printUsageAndExit() {
122 System.err.printf("Usage: bin/hbase org.apache.hadoop.hbase.replication.master.TableCFsUpdater [options]");
123 System.err.println(" where [options] are:");
124 System.err.println(" -h|-help Show this help and exit.");
125 System.err.println(" update Copy table-cfs to replication peer config");
126 System.err.println();
127 System.exit(1);
128 }
129
130 public static void main(String[] args) throws Exception {
131 if (args.length != 1) {
132 printUsageAndExit();
133 }
134 if (args[0].equals("-help") || args[0].equals("-h")) {
135 printUsageAndExit();
136 } else if (args[0].equals("update")) {
137 Configuration conf = HBaseConfiguration.create();
138 ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "TableCFsUpdater", null);
139 try {
140 TableCFsUpdater tableCFsUpdater = new TableCFsUpdater(zkw, conf, null);
141 tableCFsUpdater.update();
142 } finally {
143 zkw.close();
144 }
145 } else {
146 printUsageAndExit();
147 }
148 }
149 }