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 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.HConstants;
26 import org.apache.hadoop.hbase.client.HConnection;
27 import org.apache.hadoop.hbase.client.HConnectionManager;
28 import org.apache.hadoop.hbase.replication.ReplicationFactory;
29 import org.apache.hadoop.hbase.replication.ReplicationPeers;
30 import org.apache.hadoop.hbase.replication.ReplicationQueuesClient;
31 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
32 import org.apache.zookeeper.KeeperException;
33 import org.apache.hadoop.hbase.HTableDescriptor;
34 import org.apache.hadoop.hbase.HColumnDescriptor;
35
36 import java.io.Closeable;
37 import java.io.IOException;
38 import java.util.Map;
39 import java.util.List;
40 import java.util.ArrayList;
41 import java.util.HashMap;
42 import java.lang.Integer;
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 public class ReplicationAdmin implements Closeable {
68 private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);
69
70 public static final String TNAME = "tableName";
71 public static final String CFNAME = "columnFamlyName";
72
73
74
75 public static final String REPLICATIONTYPE = "replicationType";
76 public static final String REPLICATIONGLOBAL = Integer
77 .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
78
79 private final HConnection connection;
80 private final ReplicationQueuesClient replicationQueuesClient;
81 private final ReplicationPeers replicationPeers;
82
83
84
85
86
87
88
89 public ReplicationAdmin(Configuration conf) throws IOException {
90 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
91 throw new RuntimeException("hbase.replication isn't true, please " +
92 "enable it in order to use replication");
93 }
94 this.connection = HConnectionManager.getConnection(conf);
95 ZooKeeperWatcher zkw = createZooKeeperWatcher();
96 try {
97 this.replicationPeers = ReplicationFactory.getReplicationPeers(zkw, conf, this.connection);
98 this.replicationPeers.init();
99 this.replicationQueuesClient =
100 ReplicationFactory.getReplicationQueuesClient(zkw, conf, this.connection);
101 this.replicationQueuesClient.init();
102
103 } catch (KeeperException e) {
104 throw new IOException("Unable setup the ZooKeeper connection", e);
105 }
106 }
107
108 private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
109 return new ZooKeeperWatcher(connection.getConfiguration(),
110 "Replication Admin", new Abortable() {
111 @Override
112 public void abort(String why, Throwable e) {
113 LOG.error(why, e);
114 System.exit(1);
115 }
116
117 @Override
118 public boolean isAborted() {
119 return false;
120 }
121
122 });
123 }
124
125
126
127
128
129
130
131
132
133
134 public void addPeer(String id, String clusterKey) throws IOException {
135 this.replicationPeers.addPeer(id, clusterKey);
136 }
137
138
139
140
141
142 public void removePeer(String id) throws IOException {
143 this.replicationPeers.removePeer(id);
144 }
145
146
147
148
149
150 public void enablePeer(String id) throws IOException {
151 this.replicationPeers.enablePeer(id);
152 }
153
154
155
156
157
158 public void disablePeer(String id) throws IOException {
159 this.replicationPeers.disablePeer(id);
160 }
161
162
163
164
165
166 public int getPeersCount() {
167 return this.replicationPeers.getAllPeerIds().size();
168 }
169
170
171
172
173
174 public Map<String, String> listPeers() {
175 return this.replicationPeers.getAllPeerClusterKeys();
176 }
177
178
179
180
181
182
183
184 public boolean getPeerState(String id) throws IOException {
185 return this.replicationPeers.getStatusOfPeerFromBackingStore(id);
186 }
187
188 @Override
189 public void close() throws IOException {
190 if (this.connection != null) {
191 this.connection.close();
192 }
193 }
194
195
196
197
198
199
200
201
202
203
204
205
206
207 public List<HashMap<String, String>> listReplicated() throws IOException {
208 List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
209 HTableDescriptor[] tables = this.connection.listTables();
210
211 for (HTableDescriptor table : tables) {
212 HColumnDescriptor[] columns = table.getColumnFamilies();
213 String tableName = table.getNameAsString();
214 for (HColumnDescriptor column : columns) {
215 if (column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) {
216
217 HashMap<String, String> replicationEntry = new HashMap<String, String>();
218 replicationEntry.put(TNAME, tableName);
219 replicationEntry.put(CFNAME, column.getNameAsString());
220 replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
221 replicationColFams.add(replicationEntry);
222 }
223 }
224 }
225
226 return replicationColFams;
227 }
228 }