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 import java.util.Arrays;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.conf.Configured;
28 import org.apache.hadoop.hbase.*;
29 import org.apache.hadoop.hbase.client.HConnectable;
30 import org.apache.hadoop.hbase.client.HConnection;
31 import org.apache.hadoop.hbase.client.HConnectionManager;
32 import org.apache.hadoop.hbase.client.HTable;
33 import org.apache.hadoop.hbase.client.ConnectionFactory;
34 import org.apache.hadoop.hbase.client.Get;
35 import org.apache.hadoop.hbase.client.Put;
36 import org.apache.hadoop.hbase.client.Result;
37 import org.apache.hadoop.hbase.client.ResultScanner;
38 import org.apache.hadoop.hbase.client.Scan;
39 import org.apache.hadoop.hbase.client.Table;
40 import org.apache.hadoop.hbase.filter.Filter;
41 import org.apache.hadoop.hbase.filter.FilterList;
42 import org.apache.hadoop.hbase.filter.PrefixFilter;
43 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
44 import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
45 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
46 import org.apache.hadoop.hbase.mapreduce.TableMapper;
47 import org.apache.hadoop.hbase.mapreduce.TableSplit;
48 import org.apache.hadoop.hbase.replication.ReplicationException;
49 import org.apache.hadoop.hbase.replication.ReplicationFactory;
50 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
51 import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
52 import org.apache.hadoop.hbase.replication.ReplicationPeers;
53 import org.apache.hadoop.hbase.util.Bytes;
54 import org.apache.hadoop.hbase.util.Pair;
55 import org.apache.hadoop.hbase.util.Threads;
56 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
57 import org.apache.hadoop.mapreduce.Job;
58 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
59 import org.apache.hadoop.util.Tool;
60 import org.apache.hadoop.util.ToolRunner;
61
62
63
64
65
66
67
68
69
70
71
72 public class VerifyReplication extends Configured implements Tool {
73
74 private static final Log LOG =
75 LogFactory.getLog(VerifyReplication.class);
76
77 public final static String NAME = "verifyrep";
78 private final static String PEER_CONFIG_PREFIX = NAME + ".peer.";
79 static long startTime = 0;
80 static long endTime = Long.MAX_VALUE;
81 static int batch = -1;
82 static int versions = -1;
83 static String tableName = null;
84 static String families = null;
85 static String delimiter = "";
86 static String peerId = null;
87 static String rowPrefixes = null;
88 static int sleepMsBeforeReCompare = 0;
89 static boolean verbose = false;
90 static boolean includeDeletedCells = false;
91
92
93
94
95 public static class Verifier
96 extends TableMapper<ImmutableBytesWritable, Put> {
97
98 public static enum Counters {
99 GOODROWS, BADROWS, ONLY_IN_SOURCE_TABLE_ROWS, ONLY_IN_PEER_TABLE_ROWS, CONTENT_DIFFERENT_ROWS}
100
101 private Table sourceTable;
102 private ResultScanner replicatedScanner;
103 private Result currentCompareRowInPeerTable;
104 private Table replicatedTable;
105 private int sleepMsBeforeReCompare;
106 private String delimiter = "";
107 private boolean verbose = false;
108 private int batch = -1;
109
110
111
112
113
114
115
116
117
118 @Override
119 public void map(ImmutableBytesWritable row, final Result value,
120 Context context)
121 throws IOException {
122 if (replicatedScanner == null) {
123 Configuration conf = context.getConfiguration();
124 sleepMsBeforeReCompare = conf.getInt(NAME +".sleepMsBeforeReCompare", 0);
125 delimiter = conf.get(NAME + ".delimiter", "");
126 verbose = conf.getBoolean(NAME +".verbose", false);
127 batch = conf.getInt(NAME + ".batch", -1);
128 final Scan scan = new Scan();
129 if (batch > 0) {
130 scan.setBatch(batch);
131 }
132 scan.setCacheBlocks(false);
133 scan.setCaching(conf.getInt(TableInputFormat.SCAN_CACHEDROWS, 1));
134 long startTime = conf.getLong(NAME + ".startTime", 0);
135 long endTime = conf.getLong(NAME + ".endTime", Long.MAX_VALUE);
136 String families = conf.get(NAME + ".families", null);
137 if(families != null) {
138 String[] fams = families.split(",");
139 for(String fam : fams) {
140 scan.addFamily(Bytes.toBytes(fam));
141 }
142 }
143 boolean includeDeletedCells = conf.getBoolean(NAME + ".includeDeletedCells", false);
144 scan.setRaw(includeDeletedCells);
145 String rowPrefixes = conf.get(NAME + ".rowPrefixes", null);
146 setRowPrefixFilter(scan, rowPrefixes);
147 scan.setTimeRange(startTime, endTime);
148 int versions = conf.getInt(NAME+".versions", -1);
149 LOG.info("Setting number of version inside map as: " + versions);
150 if (versions >= 0) {
151 scan.setMaxVersions(versions);
152 }
153 TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName"));
154 sourceTable = ConnectionFactory.createConnection(conf).getTable(tableName);
155
156 final TableSplit tableSplit = (TableSplit)(context.getInputSplit());
157 HConnectionManager.execute(new HConnectable<Void>(conf) {
158 @Override
159 public Void connect(HConnection conn) throws IOException {
160 String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");
161 Configuration peerConf = HBaseConfiguration.createClusterConf(conf,
162 zkClusterKey, PEER_CONFIG_PREFIX);
163
164 TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName"));
165 replicatedTable = new HTable(peerConf, tableName);
166 scan.setStartRow(value.getRow());
167 scan.setStopRow(tableSplit.getEndRow());
168 replicatedScanner = replicatedTable.getScanner(scan);
169 return null;
170 }
171 });
172 currentCompareRowInPeerTable = replicatedScanner.next();
173 }
174 while (true) {
175 if (currentCompareRowInPeerTable == null) {
176
177 logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_SOURCE_TABLE_ROWS, value);
178 break;
179 }
180 int rowCmpRet = Bytes.compareTo(value.getRow(), currentCompareRowInPeerTable.getRow());
181 if (rowCmpRet == 0) {
182
183 try {
184 Result.compareResults(value, currentCompareRowInPeerTable);
185 context.getCounter(Counters.GOODROWS).increment(1);
186 if (verbose) {
187 LOG.info("Good row key: " + delimiter
188 + Bytes.toStringBinary(value.getRow()) + delimiter);
189 }
190 } catch (Exception e) {
191 logFailRowAndIncreaseCounter(context, Counters.CONTENT_DIFFERENT_ROWS, value);
192 }
193 currentCompareRowInPeerTable = replicatedScanner.next();
194 break;
195 } else if (rowCmpRet < 0) {
196
197 logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_SOURCE_TABLE_ROWS, value);
198 break;
199 } else {
200
201 logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_PEER_TABLE_ROWS,
202 currentCompareRowInPeerTable);
203 currentCompareRowInPeerTable = replicatedScanner.next();
204 }
205 }
206 }
207
208 private void logFailRowAndIncreaseCounter(Context context, Counters counter, Result row) {
209 if (sleepMsBeforeReCompare > 0) {
210 Threads.sleep(sleepMsBeforeReCompare);
211 try {
212 Result sourceResult = sourceTable.get(new Get(row.getRow()));
213 Result replicatedResult = replicatedTable.get(new Get(row.getRow()));
214 Result.compareResults(sourceResult, replicatedResult);
215 if (!sourceResult.isEmpty()) {
216 context.getCounter(Counters.GOODROWS).increment(1);
217 if (verbose) {
218 LOG.info("Good row key (with recompare): " + delimiter + Bytes.toStringBinary(row.getRow())
219 + delimiter);
220 }
221 }
222 return;
223 } catch (Exception e) {
224 LOG.error("recompare fail after sleep, rowkey=" + delimiter +
225 Bytes.toStringBinary(row.getRow()) + delimiter);
226 }
227 }
228 context.getCounter(counter).increment(1);
229 context.getCounter(Counters.BADROWS).increment(1);
230 LOG.error(counter.toString() + ", rowkey=" + delimiter + Bytes.toStringBinary(row.getRow()) +
231 delimiter);
232 }
233
234 @Override
235 protected void cleanup(Context context) {
236 if (replicatedScanner != null) {
237 try {
238 while (currentCompareRowInPeerTable != null) {
239 logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_PEER_TABLE_ROWS,
240 currentCompareRowInPeerTable);
241 currentCompareRowInPeerTable = replicatedScanner.next();
242 }
243 } catch (Exception e) {
244 LOG.error("fail to scan peer table in cleanup", e);
245 } finally {
246 replicatedScanner.close();
247 replicatedScanner = null;
248 }
249 }
250 if (sourceTable != null) {
251 TableName tableName = sourceTable.getName();
252 try {
253 sourceTable.close();
254 } catch (IOException ioe) {
255 LOG.warn("Exception closing source table: " + tableName, ioe);
256 }
257 }
258 if (replicatedTable != null) {
259 TableName tableName = replicatedTable.getName();
260 try {
261 replicatedTable.close();
262 } catch (IOException ioe) {
263 LOG.warn("Exception closing replicated table: " + tableName, ioe);
264 }
265 }
266 }
267 }
268
269 private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
270 final Configuration conf) throws IOException {
271 ZooKeeperWatcher localZKW = null;
272 ReplicationPeerZKImpl peer = null;
273 try {
274 localZKW = new ZooKeeperWatcher(conf, "VerifyReplication",
275 new Abortable() {
276 @Override public void abort(String why, Throwable e) {}
277 @Override public boolean isAborted() {return false;}
278 });
279
280 ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf, localZKW);
281 rp.init();
282
283 Pair<ReplicationPeerConfig, Configuration> pair = rp.getPeerConf(peerId);
284 if (pair == null) {
285 throw new IOException("Couldn't get peer conf!");
286 }
287
288 return pair;
289 } catch (ReplicationException e) {
290 throw new IOException(
291 "An error occurred while trying to connect to the remote peer cluster", e);
292 } finally {
293 if (peer != null) {
294 peer.close();
295 }
296 if (localZKW != null) {
297 localZKW.close();
298 }
299 }
300 }
301
302
303
304
305
306
307
308
309
310 public static Job createSubmittableJob(Configuration conf, String[] args)
311 throws IOException {
312 if (!doCommandLine(args)) {
313 return null;
314 }
315 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
316 HConstants.REPLICATION_ENABLE_DEFAULT)) {
317 throw new IOException("Replication needs to be enabled to verify it.");
318 }
319 conf.set(NAME+".peerId", peerId);
320 conf.set(NAME+".tableName", tableName);
321 conf.setLong(NAME+".startTime", startTime);
322 conf.setLong(NAME+".endTime", endTime);
323 conf.setInt(NAME +".sleepMsBeforeReCompare", sleepMsBeforeReCompare);
324 conf.set(NAME + ".delimiter", delimiter);
325 conf.setInt(NAME + ".batch", batch);
326 conf.setBoolean(NAME +".verbose", verbose);
327 conf.setBoolean(NAME +".includeDeletedCells", includeDeletedCells);
328 if (families != null) {
329 conf.set(NAME+".families", families);
330 }
331 if (rowPrefixes != null){
332 conf.set(NAME+".rowPrefixes", rowPrefixes);
333 }
334
335 Pair<ReplicationPeerConfig, Configuration> peerConfigPair = getPeerQuorumConfig(conf);
336 ReplicationPeerConfig peerConfig = peerConfigPair.getFirst();
337 String peerQuorumAddress = peerConfig.getClusterKey();
338 LOG.info("Peer Quorum Address: " + peerQuorumAddress + ", Peer Configuration: " +
339 peerConfig.getConfiguration());
340 conf.set(NAME + ".peerQuorumAddress", peerQuorumAddress);
341 HBaseConfiguration.setWithPrefix(conf, PEER_CONFIG_PREFIX,
342 peerConfig.getConfiguration().entrySet());
343
344 conf.setInt(NAME + ".versions", versions);
345 LOG.info("Number of version: " + versions);
346
347 Job job = new Job(conf, NAME + "_" + tableName);
348 job.setJarByClass(VerifyReplication.class);
349
350 Scan scan = new Scan();
351 scan.setTimeRange(startTime, endTime);
352 scan.setRaw(includeDeletedCells);
353 scan.setCacheBlocks(false);
354 if (batch > 0) {
355 scan.setBatch(batch);
356 }
357 if (versions >= 0) {
358 scan.setMaxVersions(versions);
359 LOG.info("Number of versions set to " + versions);
360 }
361 if(families != null) {
362 String[] fams = families.split(",");
363 for(String fam : fams) {
364 scan.addFamily(Bytes.toBytes(fam));
365 }
366 }
367
368 setRowPrefixFilter(scan, rowPrefixes);
369
370 TableMapReduceUtil.initTableMapperJob(tableName, scan,
371 Verifier.class, null, null, job);
372
373 Configuration peerClusterConf = peerConfigPair.getSecond();
374
375 TableMapReduceUtil.initCredentialsForCluster(job, peerClusterConf);
376
377 job.setOutputFormatClass(NullOutputFormat.class);
378 job.setNumReduceTasks(0);
379 return job;
380 }
381
382 private static void setRowPrefixFilter(Scan scan, String rowPrefixes) {
383 if (rowPrefixes != null && !rowPrefixes.isEmpty()) {
384 String[] rowPrefixArray = rowPrefixes.split(",");
385 Arrays.sort(rowPrefixArray);
386 FilterList filterList = new FilterList(FilterList.Operator.MUST_PASS_ONE);
387 for (String prefix : rowPrefixArray) {
388 Filter filter = new PrefixFilter(Bytes.toBytes(prefix));
389 filterList.addFilter(filter);
390 }
391 scan.setFilter(filterList);
392 byte[] startPrefixRow = Bytes.toBytes(rowPrefixArray[0]);
393 byte[] lastPrefixRow = Bytes.toBytes(rowPrefixArray[rowPrefixArray.length -1]);
394 setStartAndStopRows(scan, startPrefixRow, lastPrefixRow);
395 }
396 }
397
398 private static void setStartAndStopRows(Scan scan, byte[] startPrefixRow, byte[] lastPrefixRow) {
399 scan.setStartRow(startPrefixRow);
400 byte[] stopRow = Bytes.add(Bytes.head(lastPrefixRow, lastPrefixRow.length - 1),
401 new byte[]{(byte) (lastPrefixRow[lastPrefixRow.length - 1] + 1)});
402 scan.setStopRow(stopRow);
403 }
404
405 private static boolean doCommandLine(final String[] args) {
406 if (args.length < 2) {
407 printUsage(null);
408 return false;
409 }
410
411
412
413 restoreDefaults();
414 try {
415 for (int i = 0; i < args.length; i++) {
416 String cmd = args[i];
417 if (cmd.equals("-h") || cmd.startsWith("--h")) {
418 printUsage(null);
419 return false;
420 }
421
422 final String startTimeArgKey = "--starttime=";
423 if (cmd.startsWith(startTimeArgKey)) {
424 startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
425 continue;
426 }
427
428 final String endTimeArgKey = "--endtime=";
429 if (cmd.startsWith(endTimeArgKey)) {
430 endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
431 continue;
432 }
433
434 final String includeDeletedCellsArgKey = "--raw";
435 if (cmd.equals(includeDeletedCellsArgKey)) {
436 includeDeletedCells = true;
437 continue;
438 }
439
440 final String versionsArgKey = "--versions=";
441 if (cmd.startsWith(versionsArgKey)) {
442 versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
443 continue;
444 }
445
446 final String batchArgKey = "--batch=";
447 if (cmd.startsWith(batchArgKey)) {
448 batch = Integer.parseInt(cmd.substring(batchArgKey.length()));
449 continue;
450 }
451
452 final String familiesArgKey = "--families=";
453 if (cmd.startsWith(familiesArgKey)) {
454 families = cmd.substring(familiesArgKey.length());
455 continue;
456 }
457
458 final String rowPrefixesKey = "--row-prefixes=";
459 if (cmd.startsWith(rowPrefixesKey)){
460 rowPrefixes = cmd.substring(rowPrefixesKey.length());
461 continue;
462 }
463
464 final String delimiterArgKey = "--delimiter=";
465 if (cmd.startsWith(delimiterArgKey)) {
466 delimiter = cmd.substring(delimiterArgKey.length());
467 continue;
468 }
469
470 final String sleepToReCompareKey = "--recomparesleep=";
471 if (cmd.startsWith(sleepToReCompareKey)) {
472 sleepMsBeforeReCompare = Integer.parseInt(cmd.substring(sleepToReCompareKey.length()));
473 continue;
474 }
475
476 final String verboseKey = "--verbose";
477 if (cmd.startsWith(verboseKey)) {
478 verbose = true;
479 continue;
480 }
481
482 if (cmd.startsWith("--")) {
483 printUsage("Invalid argument '" + cmd + "'");
484 }
485
486 if (i == args.length-2) {
487 peerId = cmd;
488 }
489
490 if (i == args.length-1) {
491 tableName = cmd;
492 }
493 }
494 } catch (Exception e) {
495 e.printStackTrace();
496 printUsage("Can't start because " + e.getMessage());
497 return false;
498 }
499 return true;
500 }
501
502 private static void restoreDefaults() {
503 startTime = 0;
504 endTime = Long.MAX_VALUE;
505 batch = -1;
506 versions = -1;
507 tableName = null;
508 families = null;
509 peerId = null;
510 rowPrefixes = null;
511 includeDeletedCells = false;
512 }
513
514
515
516
517 private static void printUsage(final String errorMsg) {
518 if (errorMsg != null && errorMsg.length() > 0) {
519 System.err.println("ERROR: " + errorMsg);
520 }
521 System.err.println("Usage: verifyrep [--starttime=X]" +
522 " [--endtime=Y] [--families=A] [--row-prefixes=B] [--delimiter=] [--recomparesleep=] " +
523 "[--batch=] [--verbose] <peerid> <tablename>");
524 System.err.println();
525 System.err.println("Options:");
526 System.err.println(" starttime beginning of the time range");
527 System.err.println(" without endtime means from starttime to forever");
528 System.err.println(" endtime end of the time range");
529 System.err.println(" versions number of cell versions to verify");
530 System.err.println(" batch batch count for scan, " +
531 "note that result row counts will no longer be actual number of rows when you use this option");
532 System.err.println(" raw includes raw scan if given in options");
533 System.err.println(" families comma-separated list of families to copy");
534 System.err.println(" row-prefixes comma-separated list of row key prefixes to filter on ");
535 System.err.println(" delimiter the delimiter used in display around rowkey");
536 System.err.println(" recomparesleep milliseconds to sleep before recompare row, " +
537 "default value is 0 which disables the recompare.");
538 System.err.println(" verbose logs row keys of good rows");
539 System.err.println();
540 System.err.println("Args:");
541 System.err.println(" peerid Id of the peer used for verification, must match the one given for replication");
542 System.err.println(" tablename Name of the table to verify");
543 System.err.println();
544 System.err.println("Examples:");
545 System.err.println(" To verify the data replicated from TestTable for a 1 hour window with peer #5 ");
546 System.err.println(" $ bin/hbase " +
547 "org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication" +
548 " --starttime=1265875194289 --endtime=1265878794289 5 TestTable ");
549 }
550
551 @Override
552 public int run(String[] args) throws Exception {
553 Configuration conf = this.getConf();
554 Job job = createSubmittableJob(conf, args);
555 if (job != null) {
556 return job.waitForCompletion(true) ? 0 : 1;
557 }
558 return 1;
559 }
560
561
562
563
564
565
566
567 public static void main(String[] args) throws Exception {
568 int res = ToolRunner.run(HBaseConfiguration.create(), new VerifyReplication(), args);
569 System.exit(res);
570 }
571 }