1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.client.replication;
21
22 import java.io.Closeable;
23 import java.io.IOException;
24 import java.util.Map;
25 import java.util.List;
26 import java.util.ArrayList;
27 import java.util.HashMap;
28 import java.lang.Integer;
29
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.HConstants;
32 import org.apache.hadoop.hbase.client.HConnection;
33 import org.apache.hadoop.hbase.client.HConnectionManager;
34 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
35 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
36 import org.apache.zookeeper.KeeperException;
37 import org.apache.hadoop.hbase.HTableDescriptor;
38 import org.apache.hadoop.hbase.HColumnDescriptor;
39
40
41
42
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
68
69
70
71
72
73 public class ReplicationAdmin implements Closeable {
74
75 public static final String TNAME = "tableName";
76 public static final String CFNAME = "columnFamlyName";
77
78
79
80 public static final String REPLICATIONTYPE = "replicationType";
81 public static final String REPLICATIONGLOBAL = Integer
82 .toString(HConstants.REPLICATION_SCOPE_GLOBAL);
83
84 private final ReplicationZookeeper replicationZk;
85 private final HConnection connection;
86
87
88
89
90
91
92
93 public ReplicationAdmin(Configuration conf) throws IOException {
94 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
95 throw new RuntimeException("hbase.replication isn't true, please " +
96 "enable it in order to use replication");
97 }
98 this.connection = HConnectionManager.getConnection(conf);
99 ZooKeeperWatcher zkw = this.connection.getZooKeeperWatcher();
100 try {
101 this.replicationZk = new ReplicationZookeeper(this.connection, conf, zkw);
102 } catch (Exception exception) {
103 if (connection != null) {
104 connection.close();
105 }
106 if (exception instanceof IOException) {
107 throw (IOException) exception;
108 } else if (exception instanceof RuntimeException) {
109 throw (RuntimeException) exception;
110 } else {
111 throw new IOException("Unable setup the ZooKeeper connection", exception);
112 }
113 }
114 }
115
116
117
118
119
120
121
122
123
124 public void addPeer(String id, String clusterKey) throws IOException {
125 this.replicationZk.addPeer(id, clusterKey);
126 }
127
128
129
130
131
132 public void removePeer(String id) throws IOException {
133 this.replicationZk.removePeer(id);
134 }
135
136
137
138
139
140 public void enablePeer(String id) throws IOException {
141 this.replicationZk.enablePeer(id);
142 }
143
144
145
146
147
148 public void disablePeer(String id) throws IOException {
149 this.replicationZk.disablePeer(id);
150 }
151
152
153
154
155
156 public int getPeersCount() {
157 return this.replicationZk.listPeersIdsAndWatch().size();
158 }
159
160
161
162
163
164 public Map<String, String> listPeers() {
165 return this.replicationZk.listPeers();
166 }
167
168
169
170
171
172
173
174 public String getPeerState(String id) throws IOException {
175 try {
176 return this.replicationZk.getPeerState(id).name();
177 } catch (KeeperException e) {
178 throw new IOException("Couldn't get the state of the peer " + id, e);
179 }
180 }
181
182
183
184
185
186
187 public boolean getReplicating() throws IOException {
188 try {
189 return this.replicationZk.getReplication();
190 } catch (KeeperException e) {
191 throw new IOException("Couldn't get the replication status");
192 }
193 }
194
195
196
197
198
199
200
201 public boolean setReplicating(boolean newState) throws IOException {
202 boolean prev = true;
203 try {
204 prev = getReplicating();
205 this.replicationZk.setReplicating(newState);
206 } catch (KeeperException e) {
207 throw new IOException("Unable to set the replication state", e);
208 }
209 return prev;
210 }
211
212
213
214
215
216 ReplicationZookeeper getReplicationZk() {
217 return replicationZk;
218 }
219
220 @Override
221 public void close() throws IOException {
222 if (this.connection != null) {
223 this.connection.close();
224 }
225 }
226
227
228
229
230
231
232
233
234
235
236
237
238 public List<HashMap<String, String>> listReplicated() throws IOException {
239 List<HashMap<String, String>> replicationColFams = new ArrayList<HashMap<String, String>>();
240 HTableDescriptor[] tables = this.connection.listTables();
241
242 for (HTableDescriptor table : tables) {
243 HColumnDescriptor[] columns = table.getColumnFamilies();
244 String tableName = table.getNameAsString();
245 for (HColumnDescriptor column : columns) {
246 if (column.getScope() != HConstants.REPLICATION_SCOPE_LOCAL) {
247
248 HashMap<String, String> replicationEntry = new HashMap<String, String>();
249 replicationEntry.put(TNAME, tableName);
250 replicationEntry.put(CFNAME, column.getNameAsString());
251 replicationEntry.put(REPLICATIONTYPE, REPLICATIONGLOBAL);
252 replicationColFams.add(replicationEntry);
253 }
254 }
255 }
256
257 return replicationColFams;
258 }
259 }