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