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