1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapreduce.replication;
20
21 import java.io.IOException;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.hbase.*;
27 import org.apache.hadoop.hbase.client.HConnectable;
28 import org.apache.hadoop.hbase.client.HConnection;
29 import org.apache.hadoop.hbase.client.HConnectionManager;
30 import org.apache.hadoop.hbase.client.HTable;
31 import org.apache.hadoop.hbase.client.Put;
32 import org.apache.hadoop.hbase.client.Result;
33 import org.apache.hadoop.hbase.client.ResultScanner;
34 import org.apache.hadoop.hbase.client.Scan;
35 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
36 import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
37 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
38 import org.apache.hadoop.hbase.mapreduce.TableMapper;
39 import org.apache.hadoop.hbase.replication.ReplicationException;
40 import org.apache.hadoop.hbase.replication.ReplicationFactory;
41 import org.apache.hadoop.hbase.replication.ReplicationPeer;
42 import org.apache.hadoop.hbase.replication.ReplicationPeers;
43 import org.apache.hadoop.hbase.util.Bytes;
44 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
45 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
46 import org.apache.hadoop.mapreduce.Job;
47 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
48
49
50
51
52
53
54
55
56
57
58
59 public class VerifyReplication {
60
61 private static final Log LOG =
62 LogFactory.getLog(VerifyReplication.class);
63
64 public final static String NAME = "verifyrep";
65 static long startTime = 0;
66 static long endTime = Long.MAX_VALUE;
67 static String tableName = null;
68 static String families = null;
69 static String peerId = null;
70
71
72
73
74 public static class Verifier
75 extends TableMapper<ImmutableBytesWritable, Put> {
76
77 public static enum Counters {GOODROWS, BADROWS}
78
79 private ResultScanner replicatedScanner;
80
81
82
83
84
85
86
87
88
89 @Override
90 public void map(ImmutableBytesWritable row, final Result value,
91 Context context)
92 throws IOException {
93 if (replicatedScanner == null) {
94 Configuration conf = context.getConfiguration();
95 final Scan scan = new Scan();
96 scan.setCaching(conf.getInt(TableInputFormat.SCAN_CACHEDROWS, 1));
97 long startTime = conf.getLong(NAME + ".startTime", 0);
98 long endTime = conf.getLong(NAME + ".endTime", Long.MAX_VALUE);
99 String families = conf.get(NAME + ".families", null);
100 if(families != null) {
101 String[] fams = families.split(",");
102 for(String fam : fams) {
103 scan.addFamily(Bytes.toBytes(fam));
104 }
105 }
106 scan.setTimeRange(startTime, endTime);
107 HConnectionManager.execute(new HConnectable<Void>(conf) {
108 @Override
109 public Void connect(HConnection conn) throws IOException {
110 String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");
111 Configuration peerConf = HBaseConfiguration.create(conf);
112 ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey);
113
114 HTable replicatedTable = new HTable(peerConf, conf.get(NAME + ".tableName"));
115 scan.setStartRow(value.getRow());
116 replicatedScanner = replicatedTable.getScanner(scan);
117 return null;
118 }
119 });
120 }
121 Result res = replicatedScanner.next();
122 try {
123 Result.compareResults(value, res);
124 context.getCounter(Counters.GOODROWS).increment(1);
125 } catch (Exception e) {
126 LOG.warn("Bad row", e);
127 context.getCounter(Counters.BADROWS).increment(1);
128 }
129 }
130
131 protected void cleanup(Context context) {
132 if (replicatedScanner != null) {
133 replicatedScanner.close();
134 replicatedScanner = null;
135 }
136 }
137 }
138
139 private static String getPeerQuorumAddress(final Configuration conf) throws IOException {
140 ZooKeeperWatcher localZKW = null;
141 ReplicationPeer peer = null;
142 try {
143 localZKW = new ZooKeeperWatcher(conf, "VerifyReplication",
144 new Abortable() {
145 @Override public void abort(String why, Throwable e) {}
146 @Override public boolean isAborted() {return false;}
147 });
148
149 ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf, localZKW);
150 rp.init();
151
152 Configuration peerConf = rp.getPeerConf(peerId);
153 if (peerConf == null) {
154 throw new IOException("Couldn't get peer conf!");
155 }
156
157 return ZKUtil.getZooKeeperClusterKey(peerConf);
158 } catch (ReplicationException e) {
159 throw new IOException(
160 "An error occured while trying to connect to the remove peer cluster", e);
161 } finally {
162 if (peer != null) {
163 peer.close();
164 }
165 if (localZKW != null) {
166 localZKW.close();
167 }
168 }
169 }
170
171
172
173
174
175
176
177
178
179 public static Job createSubmittableJob(Configuration conf, String[] args)
180 throws IOException {
181 if (!doCommandLine(args)) {
182 return null;
183 }
184 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
185 HConstants.REPLICATION_ENABLE_DEFAULT)) {
186 throw new IOException("Replication needs to be enabled to verify it.");
187 }
188 conf.set(NAME+".peerId", peerId);
189 conf.set(NAME+".tableName", tableName);
190 conf.setLong(NAME+".startTime", startTime);
191 conf.setLong(NAME+".endTime", endTime);
192 if (families != null) {
193 conf.set(NAME+".families", families);
194 }
195
196 String peerQuorumAddress = getPeerQuorumAddress(conf);
197 conf.set(NAME + ".peerQuorumAddress", peerQuorumAddress);
198 LOG.info("Peer Quorum Address: " + peerQuorumAddress);
199
200 Job job = new Job(conf, NAME + "_" + tableName);
201 job.setJarByClass(VerifyReplication.class);
202
203 Scan scan = new Scan();
204 scan.setTimeRange(startTime, endTime);
205 if(families != null) {
206 String[] fams = families.split(",");
207 for(String fam : fams) {
208 scan.addFamily(Bytes.toBytes(fam));
209 }
210 }
211 TableMapReduceUtil.initTableMapperJob(tableName, scan,
212 Verifier.class, null, null, job);
213
214
215 TableMapReduceUtil.initCredentialsForCluster(job, peerQuorumAddress);
216
217 job.setOutputFormatClass(NullOutputFormat.class);
218 job.setNumReduceTasks(0);
219 return job;
220 }
221
222 private static boolean doCommandLine(final String[] args) {
223 if (args.length < 2) {
224 printUsage(null);
225 return false;
226 }
227 try {
228 for (int i = 0; i < args.length; i++) {
229 String cmd = args[i];
230 if (cmd.equals("-h") || cmd.startsWith("--h")) {
231 printUsage(null);
232 return false;
233 }
234
235 final String startTimeArgKey = "--starttime=";
236 if (cmd.startsWith(startTimeArgKey)) {
237 startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
238 continue;
239 }
240
241 final String endTimeArgKey = "--endtime=";
242 if (cmd.startsWith(endTimeArgKey)) {
243 endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
244 continue;
245 }
246
247 final String familiesArgKey = "--families=";
248 if (cmd.startsWith(familiesArgKey)) {
249 families = cmd.substring(familiesArgKey.length());
250 continue;
251 }
252
253 if (i == args.length-2) {
254 peerId = cmd;
255 }
256
257 if (i == args.length-1) {
258 tableName = cmd;
259 }
260 }
261 } catch (Exception e) {
262 e.printStackTrace();
263 printUsage("Can't start because " + e.getMessage());
264 return false;
265 }
266 return true;
267 }
268
269
270
271
272 private static void printUsage(final String errorMsg) {
273 if (errorMsg != null && errorMsg.length() > 0) {
274 System.err.println("ERROR: " + errorMsg);
275 }
276 System.err.println("Usage: verifyrep [--starttime=X]" +
277 " [--stoptime=Y] [--families=A] <peerid> <tablename>");
278 System.err.println();
279 System.err.println("Options:");
280 System.err.println(" starttime beginning of the time range");
281 System.err.println(" without endtime means from starttime to forever");
282 System.err.println(" endtime end of the time range");
283 System.err.println(" families comma-separated list of families to copy");
284 System.err.println();
285 System.err.println("Args:");
286 System.err.println(" peerid Id of the peer used for verification, must match the one given for replication");
287 System.err.println(" tablename Name of the table to verify");
288 System.err.println();
289 System.err.println("Examples:");
290 System.err.println(" To verify the data replicated from TestTable for a 1 hour window with peer #5 ");
291 System.err.println(" $ bin/hbase " +
292 "org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication" +
293 " --starttime=1265875194289 --endtime=1265878794289 5 TestTable ");
294 }
295
296
297
298
299
300
301
302 public static void main(String[] args) throws Exception {
303 Configuration conf = HBaseConfiguration.create();
304 Job job = createSubmittableJob(conf, args);
305 if (job != null) {
306 System.exit(job.waitForCompletion(true) ? 0 : 1);
307 }
308 }
309 }