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.conf.Configured;
27 import org.apache.hadoop.hbase.Abortable;
28 import org.apache.hadoop.hbase.HBaseConfiguration;
29 import org.apache.hadoop.hbase.HConstants;
30 import org.apache.hadoop.hbase.client.HConnectable;
31 import org.apache.hadoop.hbase.client.HConnection;
32 import org.apache.hadoop.hbase.client.HConnectionManager;
33 import org.apache.hadoop.hbase.client.HTable;
34 import org.apache.hadoop.hbase.client.Put;
35 import org.apache.hadoop.hbase.client.Result;
36 import org.apache.hadoop.hbase.client.ResultScanner;
37 import org.apache.hadoop.hbase.client.Scan;
38 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
39 import org.apache.hadoop.hbase.mapreduce.TableInputFormat;
40 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
41 import org.apache.hadoop.hbase.mapreduce.TableMapper;
42 import org.apache.hadoop.hbase.mapreduce.TableSplit;
43 import org.apache.hadoop.hbase.replication.ReplicationException;
44 import org.apache.hadoop.hbase.replication.ReplicationFactory;
45 import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
46 import org.apache.hadoop.hbase.replication.ReplicationPeerZKImpl;
47 import org.apache.hadoop.hbase.replication.ReplicationPeers;
48 import org.apache.hadoop.hbase.util.Bytes;
49 import org.apache.hadoop.hbase.util.Pair;
50 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
51 import org.apache.hadoop.mapreduce.Job;
52 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
53 import org.apache.hadoop.util.Tool;
54 import org.apache.hadoop.util.ToolRunner;
55
56
57
58
59
60
61
62
63
64
65
66 public class VerifyReplication extends Configured implements Tool {
67
68 private static final Log LOG =
69 LogFactory.getLog(VerifyReplication.class);
70
71 public final static String NAME = "verifyrep";
72 private final static String PEER_CONFIG_PREFIX = NAME + ".peer.";
73 static long startTime = 0;
74 static long endTime = Long.MAX_VALUE;
75 static int versions = -1;
76 static String tableName = null;
77 static String families = null;
78 static String peerId = null;
79
80
81
82
83 public static class Verifier
84 extends TableMapper<ImmutableBytesWritable, Put> {
85
86 public static enum Counters {
87 GOODROWS, BADROWS, ONLY_IN_SOURCE_TABLE_ROWS, ONLY_IN_PEER_TABLE_ROWS, CONTENT_DIFFERENT_ROWS}
88
89 private ResultScanner replicatedScanner;
90 private Result currentCompareRowInPeerTable;
91
92
93
94
95
96
97
98
99
100 @Override
101 public void map(ImmutableBytesWritable row, final Result value,
102 Context context)
103 throws IOException {
104 if (replicatedScanner == null) {
105 Configuration conf = context.getConfiguration();
106 final Scan scan = new Scan();
107 scan.setCaching(conf.getInt(TableInputFormat.SCAN_CACHEDROWS, 1));
108 long startTime = conf.getLong(NAME + ".startTime", 0);
109 long endTime = conf.getLong(NAME + ".endTime", Long.MAX_VALUE);
110 String families = conf.get(NAME + ".families", null);
111 if(families != null) {
112 String[] fams = families.split(",");
113 for(String fam : fams) {
114 scan.addFamily(Bytes.toBytes(fam));
115 }
116 }
117 scan.setTimeRange(startTime, endTime);
118 int versions = conf.getInt(NAME+".versions", -1);
119 LOG.info("Setting number of version inside map as: " + versions);
120 if (versions >= 0) {
121 scan.setMaxVersions(versions);
122 }
123
124 final TableSplit tableSplit = (TableSplit)(context.getInputSplit());
125 HConnectionManager.execute(new HConnectable<Void>(conf) {
126 @Override
127 public Void connect(HConnection conn) throws IOException {
128 String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");
129 Configuration peerConf = HBaseConfiguration.createClusterConf(conf,
130 zkClusterKey, PEER_CONFIG_PREFIX);
131
132 HTable replicatedTable = new HTable(peerConf, conf.get(NAME + ".tableName"));
133 scan.setStartRow(tableSplit.getStartRow());
134 scan.setStopRow(tableSplit.getEndRow());
135 replicatedScanner = replicatedTable.getScanner(scan);
136 return null;
137 }
138 });
139 currentCompareRowInPeerTable = replicatedScanner.next();
140 }
141 while (true) {
142 if (currentCompareRowInPeerTable == null) {
143
144 logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_SOURCE_TABLE_ROWS, value);
145 break;
146 }
147 int rowCmpRet = Bytes.compareTo(value.getRow(), currentCompareRowInPeerTable.getRow());
148 if (rowCmpRet == 0) {
149
150 try {
151 Result.compareResults(value, currentCompareRowInPeerTable);
152 context.getCounter(Counters.GOODROWS).increment(1);
153 } catch (Exception e) {
154 logFailRowAndIncreaseCounter(context, Counters.CONTENT_DIFFERENT_ROWS, value);
155 LOG.error("Exception while comparing row : " + e);
156 }
157 currentCompareRowInPeerTable = replicatedScanner.next();
158 break;
159 } else if (rowCmpRet < 0) {
160
161 logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_SOURCE_TABLE_ROWS, value);
162 break;
163 } else {
164
165 logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_PEER_TABLE_ROWS,
166 currentCompareRowInPeerTable);
167 currentCompareRowInPeerTable = replicatedScanner.next();
168 }
169 }
170 }
171
172 private void logFailRowAndIncreaseCounter(Context context, Counters counter, Result row) {
173 context.getCounter(counter).increment(1);
174 context.getCounter(Counters.BADROWS).increment(1);
175 LOG.error(counter.toString() + ", rowkey=" + Bytes.toString(row.getRow()));
176 }
177
178 @Override
179 protected void cleanup(Context context) {
180 if (replicatedScanner != null) {
181 try {
182 while (currentCompareRowInPeerTable != null) {
183 logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_PEER_TABLE_ROWS,
184 currentCompareRowInPeerTable);
185 currentCompareRowInPeerTable = replicatedScanner.next();
186 }
187 } catch (Exception e) {
188 LOG.error("fail to scan peer table in cleanup", e);
189 } finally {
190 replicatedScanner.close();
191 replicatedScanner = null;
192 }
193 }
194 }
195 }
196
197 private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
198 final Configuration conf) throws IOException {
199 ZooKeeperWatcher localZKW = null;
200 ReplicationPeerZKImpl peer = null;
201 try {
202 localZKW = new ZooKeeperWatcher(conf, "VerifyReplication",
203 new Abortable() {
204 @Override public void abort(String why, Throwable e) {}
205 @Override public boolean isAborted() {return false;}
206 });
207
208 ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf, localZKW);
209 rp.init();
210
211 Pair<ReplicationPeerConfig, Configuration> pair = rp.getPeerConf(peerId);
212 if (pair == null) {
213 throw new IOException("Couldn't get peer conf!");
214 }
215
216 return pair;
217 } catch (ReplicationException e) {
218 throw new IOException(
219 "An error occured while trying to connect to the remove peer cluster", e);
220 } finally {
221 if (peer != null) {
222 peer.close();
223 }
224 if (localZKW != null) {
225 localZKW.close();
226 }
227 }
228 }
229
230
231
232
233
234
235
236
237
238 public static Job createSubmittableJob(Configuration conf, String[] args)
239 throws IOException {
240 if (!doCommandLine(args)) {
241 return null;
242 }
243 if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
244 HConstants.REPLICATION_ENABLE_DEFAULT)) {
245 throw new IOException("Replication needs to be enabled to verify it.");
246 }
247 conf.set(NAME+".peerId", peerId);
248 conf.set(NAME+".tableName", tableName);
249 conf.setLong(NAME+".startTime", startTime);
250 conf.setLong(NAME+".endTime", endTime);
251 if (families != null) {
252 conf.set(NAME+".families", families);
253 }
254
255 Pair<ReplicationPeerConfig, Configuration> peerConfigPair = getPeerQuorumConfig(conf);
256 ReplicationPeerConfig peerConfig = peerConfigPair.getFirst();
257 String peerQuorumAddress = peerConfig.getClusterKey();
258 LOG.info("Peer Quorum Address: " + peerQuorumAddress + ", Peer Configuration: " +
259 peerConfig.getConfiguration());
260 conf.set(NAME + ".peerQuorumAddress", peerQuorumAddress);
261 HBaseConfiguration.setWithPrefix(conf, PEER_CONFIG_PREFIX,
262 peerConfig.getConfiguration().entrySet());
263
264 conf.setInt(NAME + ".versions", versions);
265 LOG.info("Number of version: " + versions);
266
267 Job job = new Job(conf, NAME + "_" + tableName);
268 job.setJarByClass(VerifyReplication.class);
269
270 Scan scan = new Scan();
271 scan.setTimeRange(startTime, endTime);
272 if (versions >= 0) {
273 scan.setMaxVersions(versions);
274 LOG.info("Number of versions set to " + versions);
275 }
276 if(families != null) {
277 String[] fams = families.split(",");
278 for(String fam : fams) {
279 scan.addFamily(Bytes.toBytes(fam));
280 }
281 }
282 TableMapReduceUtil.initTableMapperJob(tableName, scan,
283 Verifier.class, null, null, job);
284
285 Configuration peerClusterConf = peerConfigPair.getSecond();
286
287 TableMapReduceUtil.initCredentialsForCluster(job, peerClusterConf);
288
289 job.setOutputFormatClass(NullOutputFormat.class);
290 job.setNumReduceTasks(0);
291 return job;
292 }
293
294 private static boolean doCommandLine(final String[] args) {
295 if (args.length < 2) {
296 printUsage(null);
297 return false;
298 }
299 try {
300 for (int i = 0; i < args.length; i++) {
301 String cmd = args[i];
302 if (cmd.equals("-h") || cmd.startsWith("--h")) {
303 printUsage(null);
304 return false;
305 }
306
307 final String startTimeArgKey = "--starttime=";
308 if (cmd.startsWith(startTimeArgKey)) {
309 startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
310 continue;
311 }
312
313 final String endTimeArgKey = "--endtime=";
314 if (cmd.startsWith(endTimeArgKey)) {
315 endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
316 continue;
317 }
318
319 final String versionsArgKey = "--versions=";
320 if (cmd.startsWith(versionsArgKey)) {
321 versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
322 continue;
323 }
324
325 final String familiesArgKey = "--families=";
326 if (cmd.startsWith(familiesArgKey)) {
327 families = cmd.substring(familiesArgKey.length());
328 continue;
329 }
330
331 if (i == args.length-2) {
332 peerId = cmd;
333 }
334
335 if (i == args.length-1) {
336 tableName = cmd;
337 }
338 }
339 } catch (Exception e) {
340 e.printStackTrace();
341 printUsage("Can't start because " + e.getMessage());
342 return false;
343 }
344 return true;
345 }
346
347
348
349
350 private static void printUsage(final String errorMsg) {
351 if (errorMsg != null && errorMsg.length() > 0) {
352 System.err.println("ERROR: " + errorMsg);
353 }
354 System.err.println("Usage: verifyrep [--starttime=X]" +
355 " [--stoptime=Y] [--families=A] <peerid> <tablename>");
356 System.err.println();
357 System.err.println("Options:");
358 System.err.println(" starttime beginning of the time range");
359 System.err.println(" without endtime means from starttime to forever");
360 System.err.println(" endtime end of the time range");
361 System.err.println(" versions number of cell versions to verify");
362 System.err.println(" families comma-separated list of families to copy");
363 System.err.println();
364 System.err.println("Args:");
365 System.err.println(" peerid Id of the peer used for verification, must match the one given for replication");
366 System.err.println(" tablename Name of the table to verify");
367 System.err.println();
368 System.err.println("Examples:");
369 System.err.println(" To verify the data replicated from TestTable for a 1 hour window with peer #5 ");
370 System.err.println(" $ bin/hbase " +
371 "org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication" +
372 " --starttime=1265875194289 --endtime=1265878794289 5 TestTable ");
373 }
374
375 @Override
376 public int run(String[] args) throws Exception {
377 Configuration conf = this.getConf();
378 Job job = createSubmittableJob(conf, args);
379 if (job != null) {
380 return job.waitForCompletion(true) ? 0 : 1;
381 }
382 return 1;
383 }
384
385
386
387
388
389
390
391 public static void main(String[] args) throws Exception {
392 int res = ToolRunner.run(HBaseConfiguration.create(), new VerifyReplication(), args);
393 System.exit(res);
394 }
395 }