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