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