View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
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.TableName;
31  import org.apache.hadoop.hbase.client.Connection;
32  import org.apache.hadoop.hbase.client.ConnectionFactory;
33  import org.apache.hadoop.hbase.client.Put;
34  import org.apache.hadoop.hbase.client.Result;
35  import org.apache.hadoop.hbase.client.ResultScanner;
36  import org.apache.hadoop.hbase.client.Scan;
37  import org.apache.hadoop.hbase.client.Table;
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.ReplicationPeerZKImpl;
46  import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
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   * This map-only job compares the data from a local table with a remote one.
58   * Every cell is compared and must have exactly the same keys (even timestamp)
59   * as well as same value. It is possible to restrict the job by time range and
60   * families. The peer id that's provided must match the one given when the
61   * replication stream was setup.
62   * <p>
63   * Two counters are provided, Verifier.Counters.GOODROWS and BADROWS. The reason
64   * for a why a row is different is shown in the map's log.
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    private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
81  
82    /**
83     * Map-only comparator for 2 tables
84     */
85    public static class Verifier
86        extends TableMapper<ImmutableBytesWritable, Put> {
87  
88  
89  
90      public static enum Counters {
91        GOODROWS, BADROWS, ONLY_IN_SOURCE_TABLE_ROWS, ONLY_IN_PEER_TABLE_ROWS, CONTENT_DIFFERENT_ROWS}
92  
93      private Connection connection;
94      private Table replicatedTable;
95      private ResultScanner replicatedScanner;
96      private Result currentCompareRowInPeerTable;
97  
98      /**
99       * Map method that compares every scanned row with the equivalent from
100      * a distant cluster.
101      * @param row  The current table row key.
102      * @param value  The columns.
103      * @param context  The current context.
104      * @throws IOException When something is broken with the data.
105      */
106     @Override
107     public void map(ImmutableBytesWritable row, final Result value,
108                     Context context)
109         throws IOException {
110       if (replicatedScanner == null) {
111         Configuration conf = context.getConfiguration();
112         final Scan scan = new Scan();
113         scan.setCaching(conf.getInt(TableInputFormat.SCAN_CACHEDROWS, 1));
114         long startTime = conf.getLong(NAME + ".startTime", 0);
115         long endTime = conf.getLong(NAME + ".endTime", Long.MAX_VALUE);
116         String families = conf.get(NAME + ".families", null);
117         if(families != null) {
118           String[] fams = families.split(",");
119           for(String fam : fams) {
120             scan.addFamily(Bytes.toBytes(fam));
121           }
122         }
123         scan.setTimeRange(startTime, endTime);
124         int versions = conf.getInt(NAME+".versions", -1);
125         LOG.info("Setting number of version inside map as: " + versions);
126         if (versions >= 0) {
127           scan.setMaxVersions(versions);
128         }
129 
130         final TableSplit tableSplit = (TableSplit)(context.getInputSplit());
131 
132         String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");
133         Configuration peerConf = HBaseConfiguration.createClusterConf(conf,
134             zkClusterKey, PEER_CONFIG_PREFIX);
135 
136         TableName tableName = TableName.valueOf(conf.get(NAME + ".tableName"));
137         connection = ConnectionFactory.createConnection(peerConf);
138         replicatedTable = connection.getTable(tableName);
139         scan.setStartRow(value.getRow());
140         scan.setStopRow(tableSplit.getEndRow());
141         replicatedScanner = replicatedTable.getScanner(scan);
142         currentCompareRowInPeerTable = replicatedScanner.next();
143       }
144       while (true) {
145         if (currentCompareRowInPeerTable == null) {
146           // reach the region end of peer table, row only in source table
147           logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_SOURCE_TABLE_ROWS, value);
148           break;
149         }
150         int rowCmpRet = Bytes.compareTo(value.getRow(), currentCompareRowInPeerTable.getRow());
151         if (rowCmpRet == 0) {
152           // rowkey is same, need to compare the content of the row
153           try {
154             Result.compareResults(value, currentCompareRowInPeerTable);
155             context.getCounter(Counters.GOODROWS).increment(1);
156           } catch (Exception e) {
157             logFailRowAndIncreaseCounter(context, Counters.CONTENT_DIFFERENT_ROWS, value);
158             LOG.error("Exception while comparing row : " + e);
159           }
160           currentCompareRowInPeerTable = replicatedScanner.next();
161           break;
162         } else if (rowCmpRet < 0) {
163           // row only exists in source table
164           logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_SOURCE_TABLE_ROWS, value);
165           break;
166         } else {
167           // row only exists in peer table
168           logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_PEER_TABLE_ROWS,
169             currentCompareRowInPeerTable);
170           currentCompareRowInPeerTable = replicatedScanner.next();
171         }
172       }
173     }
174 
175     private void logFailRowAndIncreaseCounter(Context context, Counters counter, Result row) {
176       context.getCounter(counter).increment(1);
177       context.getCounter(Counters.BADROWS).increment(1);
178       LOG.error(counter.toString() + ", rowkey=" + Bytes.toString(row.getRow()));
179     }
180 
181     @Override
182     protected void cleanup(Context context) {
183       if (replicatedScanner != null) {
184         try {
185           while (currentCompareRowInPeerTable != null) {
186             logFailRowAndIncreaseCounter(context, Counters.ONLY_IN_PEER_TABLE_ROWS,
187               currentCompareRowInPeerTable);
188             currentCompareRowInPeerTable = replicatedScanner.next();
189           }
190         } catch (Exception e) {
191           LOG.error("fail to scan peer table in cleanup", e);
192         } finally {
193           replicatedScanner.close();
194           replicatedScanner = null;
195         }
196       }
197       if(replicatedTable != null){
198         try{
199           replicatedTable.close();
200         } catch (Exception e) {
201           LOG.error("fail to close table in cleanup", e);
202         }
203       }
204       if(connection != null){
205         try {
206           connection.close();
207         } catch (Exception e) {
208           LOG.error("fail to close connection in cleanup", e);
209         }
210       }
211     }
212   }
213 
214   private static Pair<ReplicationPeerConfig, Configuration> getPeerQuorumConfig(
215       final Configuration conf) throws IOException {
216     ZooKeeperWatcher localZKW = null;
217     ReplicationPeerZKImpl peer = null;
218     try {
219       localZKW = new ZooKeeperWatcher(conf, "VerifyReplication",
220           new Abortable() {
221             @Override public void abort(String why, Throwable e) {}
222             @Override public boolean isAborted() {return false;}
223           });
224 
225       ReplicationPeers rp = ReplicationFactory.getReplicationPeers(localZKW, conf, localZKW);
226       rp.init();
227 
228       Pair<ReplicationPeerConfig, Configuration> pair = rp.getPeerConf(peerId);
229       if (pair == null) {
230         throw new IOException("Couldn't get peer conf!");
231       }
232 
233       return pair;
234     } catch (ReplicationException e) {
235       throw new IOException(
236           "An error occured while trying to connect to the remove peer cluster", e);
237     } finally {
238       if (peer != null) {
239         peer.close();
240       }
241       if (localZKW != null) {
242         localZKW.close();
243       }
244     }
245   }
246 
247   /**
248    * Sets up the actual job.
249    *
250    * @param conf  The current configuration.
251    * @param args  The command line parameters.
252    * @return The newly created job.
253    * @throws java.io.IOException When setting up the job fails.
254    */
255   public static Job createSubmittableJob(Configuration conf, String[] args)
256   throws IOException {
257     if (!doCommandLine(args)) {
258       return null;
259     }
260     if (!conf.getBoolean(HConstants.REPLICATION_ENABLE_KEY,
261         HConstants.REPLICATION_ENABLE_DEFAULT)) {
262       throw new IOException("Replication needs to be enabled to verify it.");
263     }
264     conf.set(NAME+".peerId", peerId);
265     conf.set(NAME+".tableName", tableName);
266     conf.setLong(NAME+".startTime", startTime);
267     conf.setLong(NAME+".endTime", endTime);
268     if (families != null) {
269       conf.set(NAME+".families", families);
270     }
271 
272     Pair<ReplicationPeerConfig, Configuration> peerConfigPair = getPeerQuorumConfig(conf);
273     ReplicationPeerConfig peerConfig = peerConfigPair.getFirst();
274     String peerQuorumAddress = peerConfig.getClusterKey();
275     LOG.info("Peer Quorum Address: " + peerQuorumAddress + ", Peer Configuration: " +
276         peerConfig.getConfiguration());
277     conf.set(NAME + ".peerQuorumAddress", peerQuorumAddress);
278     HBaseConfiguration.setWithPrefix(conf, PEER_CONFIG_PREFIX,
279         peerConfig.getConfiguration().entrySet());
280 
281     conf.setInt(NAME + ".versions", versions);
282     LOG.info("Number of version: " + versions);
283 
284     Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + tableName));
285     job.setJarByClass(VerifyReplication.class);
286 
287     Scan scan = new Scan();
288     scan.setTimeRange(startTime, endTime);
289     if (versions >= 0) {
290       scan.setMaxVersions(versions);
291       LOG.info("Number of versions set to " + versions);
292     }
293     if(families != null) {
294       String[] fams = families.split(",");
295       for(String fam : fams) {
296         scan.addFamily(Bytes.toBytes(fam));
297       }
298     }
299     TableMapReduceUtil.initTableMapperJob(tableName, scan,
300         Verifier.class, null, null, job);
301 
302     Configuration peerClusterConf = peerConfigPair.getSecond();
303     // Obtain the auth token from peer cluster
304     TableMapReduceUtil.initCredentialsForCluster(job, peerClusterConf);
305 
306     job.setOutputFormatClass(NullOutputFormat.class);
307     job.setNumReduceTasks(0);
308     return job;
309   }
310 
311   private static boolean doCommandLine(final String[] args) {
312     if (args.length < 2) {
313       printUsage(null);
314       return false;
315     }
316     try {
317       for (int i = 0; i < args.length; i++) {
318         String cmd = args[i];
319         if (cmd.equals("-h") || cmd.startsWith("--h")) {
320           printUsage(null);
321           return false;
322         }
323 
324         final String startTimeArgKey = "--starttime=";
325         if (cmd.startsWith(startTimeArgKey)) {
326           startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
327           continue;
328         }
329 
330         final String endTimeArgKey = "--endtime=";
331         if (cmd.startsWith(endTimeArgKey)) {
332           endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
333           continue;
334         }
335 
336         final String versionsArgKey = "--versions=";
337         if (cmd.startsWith(versionsArgKey)) {
338           versions = Integer.parseInt(cmd.substring(versionsArgKey.length()));
339           continue;
340         }
341 
342         final String familiesArgKey = "--families=";
343         if (cmd.startsWith(familiesArgKey)) {
344           families = cmd.substring(familiesArgKey.length());
345           continue;
346         }
347 
348         if (i == args.length-2) {
349           peerId = cmd;
350         }
351 
352         if (i == args.length-1) {
353           tableName = cmd;
354         }
355       }
356     } catch (Exception e) {
357       e.printStackTrace();
358       printUsage("Can't start because " + e.getMessage());
359       return false;
360     }
361     return true;
362   }
363 
364   /*
365    * @param errorMsg Error message.  Can be null.
366    */
367   private static void printUsage(final String errorMsg) {
368     if (errorMsg != null && errorMsg.length() > 0) {
369       System.err.println("ERROR: " + errorMsg);
370     }
371     System.err.println("Usage: verifyrep [--starttime=X]" +
372         " [--stoptime=Y] [--families=A] <peerid> <tablename>");
373     System.err.println();
374     System.err.println("Options:");
375     System.err.println(" starttime    beginning of the time range");
376     System.err.println("              without endtime means from starttime to forever");
377     System.err.println(" endtime      end of the time range");
378     System.err.println(" versions     number of cell versions to verify");
379     System.err.println(" families     comma-separated list of families to copy");
380     System.err.println();
381     System.err.println("Args:");
382     System.err.println(" peerid       Id of the peer used for verification, must match the one given for replication");
383     System.err.println(" tablename    Name of the table to verify");
384     System.err.println();
385     System.err.println("Examples:");
386     System.err.println(" To verify the data replicated from TestTable for a 1 hour window with peer #5 ");
387     System.err.println(" $ bin/hbase " +
388         "org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication" +
389         " --starttime=1265875194289 --endtime=1265878794289 5 TestTable ");
390   }
391 
392   @Override
393   public int run(String[] args) throws Exception {
394     Configuration conf = this.getConf();
395     Job job = createSubmittableJob(conf, args);
396     if (job != null) {
397       return job.waitForCompletion(true) ? 0 : 1;
398     } 
399     return 1;
400   }
401 
402   /**
403    * Main entry point.
404    *
405    * @param args  The command line parameters.
406    * @throws Exception When running the job fails.
407    */
408   public static void main(String[] args) throws Exception {
409     int res = ToolRunner.run(HBaseConfiguration.create(), new VerifyReplication(), args);
410     System.exit(res);
411   }
412 }