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