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.ReplicationZookeeper;
29 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
30 import org.apache.zookeeper.KeeperException;
31
32 import java.io.Closeable;
33 import java.io.IOException;
34 import java.util.Map;
35
36
37
38
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 public class ReplicationAdmin implements Closeable {
70 private static final Log LOG = LogFactory.getLog(ReplicationAdmin.class);
71
72 private final ReplicationZookeeper replicationZk;
73 private final HConnection connection;
74
75
76
77
78
79
80
81 public ReplicationAdmin(Configuration conf) throws IOException {
82 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY, false)) {
83 throw new RuntimeException("hbase.replication isn't true, please " +
84 "enable it in order to use replication");
85 }
86 this.connection = HConnectionManager.getConnection(conf);
87 ZooKeeperWatcher zkw = createZooKeeperWatcher();
88 try {
89 this.replicationZk = new ReplicationZookeeper(this.connection, conf, zkw);
90 } catch (KeeperException e) {
91 throw new IOException("Unable setup the ZooKeeper connection", e);
92 }
93 }
94
95 private ZooKeeperWatcher createZooKeeperWatcher() throws IOException {
96 return new ZooKeeperWatcher(connection.getConfiguration(),
97 "Replication Admin", new Abortable() {
98 @Override
99 public void abort(String why, Throwable e) {
100 LOG.error(why, e);
101 System.exit(1);
102 }
103
104 @Override
105 public boolean isAborted() {
106 return false;
107 }
108
109 });
110 }
111
112
113
114
115
116
117
118
119
120
121 public void addPeer(String id, String clusterKey) throws IOException {
122 this.replicationZk.addPeer(id, clusterKey);
123 }
124
125
126
127
128
129 public void removePeer(String id) throws IOException {
130 this.replicationZk.removePeer(id);
131 }
132
133
134
135
136
137 public void enablePeer(String id) throws IOException {
138 this.replicationZk.enablePeer(id);
139 }
140
141
142
143
144
145 public void disablePeer(String id) throws IOException {
146 this.replicationZk.disablePeer(id);
147 }
148
149
150
151
152
153 public int getPeersCount() {
154 return this.replicationZk.listPeersIdsAndWatch().size();
155 }
156
157
158
159
160
161 public Map<String, String> listPeers() {
162 return this.replicationZk.listPeers();
163 }
164
165
166
167
168
169
170 public boolean getReplicating() throws IOException {
171 try {
172 return this.replicationZk.getReplication();
173 } catch (KeeperException e) {
174 throw new IOException("Couldn't get the replication status");
175 }
176 }
177
178
179
180
181
182
183
184 public boolean setReplicating(boolean newState) throws IOException {
185 boolean prev = true;
186 try {
187 prev = getReplicating();
188 this.replicationZk.setReplication(newState);
189 } catch (KeeperException e) {
190 throw new IOException("Unable to set the replication state", e);
191 }
192 return prev;
193 }
194
195
196
197
198
199 ReplicationZookeeper getReplicationZk() {
200 return replicationZk;
201 }
202
203 @Override
204 public void close() throws IOException {
205 if (this.connection != null) {
206 this.connection.close();
207 }
208 }
209 }