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.ReplicationPeer;
43  import org.apache.hadoop.hbase.replication.ReplicationPeers;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
46  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
47  import org.apache.hadoop.mapreduce.Job;
48  import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
49  import org.apache.hadoop.util.Tool;
50  import org.apache.hadoop.util.ToolRunner;
51  
52  /**
53   * This map-only job compares the data from a local table with a remote one.
54   * Every cell is compared and must have exactly the same keys (even timestamp)
55   * as well as same value. It is possible to restrict the job by time range and
56   * families. The peer id that's provided must match the one given when the
57   * replication stream was setup.
58   * <p>
59   * Two counters are provided, Verifier.Counters.GOODROWS and BADROWS. The reason
60   * for a why a row is different is shown in the map's log.
61   */
62  public class VerifyReplication extends Configured implements Tool {
63  
64    private static final Log LOG =
65        LogFactory.getLog(VerifyReplication.class);
66  
67    public final static String NAME = "verifyrep";
68    static long startTime = 0;
69    static long endTime = 0;
70    static String tableName = null;
71    static String families = null;
72    static String peerId = null;
73  
74    /**
75     * Map-only comparator for 2 tables
76     */
77    public static class Verifier
78        extends TableMapper<ImmutableBytesWritable, Put> {
79  
80      public static enum Counters {GOODROWS, BADROWS}
81  
82      private ResultScanner replicatedScanner;
83  
84      /**
85       * Map method that compares every scanned row with the equivalent from
86       * a distant cluster.
87       * @param row  The current table row key.
88       * @param value  The columns.
89       * @param context  The current context.
90       * @throws IOException When something is broken with the data.
91       */
92      @Override
93      public void map(ImmutableBytesWritable row, final Result value,
94                      Context context)
95          throws IOException {
96        if (replicatedScanner == null) {
97          Configuration conf = context.getConfiguration();
98          final Scan scan = new Scan();
99          scan.setCaching(conf.getInt(TableInputFormat.SCAN_CACHEDROWS, 1));
100         long startTime = conf.getLong(NAME + ".startTime", 0);
101         long endTime = conf.getLong(NAME + ".endTime", 0);
102         String families = conf.get(NAME + ".families", null);
103         if(families != null) {
104           String[] fams = families.split(",");
105           for(String fam : fams) {
106             scan.addFamily(Bytes.toBytes(fam));
107           }
108         }
109         if (startTime != 0) {
110           scan.setTimeRange(startTime,
111               endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime);
112         }
113         HConnectionManager.execute(new HConnectable<Void>(conf) {
114           @Override
115           public Void connect(HConnection conn) throws IOException {
116             String zkClusterKey = conf.get(NAME + ".peerQuorumAddress");
117             Configuration peerConf = HBaseConfiguration.create(conf);
118             ZKUtil.applyClusterKeyToConf(peerConf, zkClusterKey);
119 
120             HTable replicatedTable = new HTable(peerConf, conf.get(NAME + ".tableName"));
121             scan.setStartRow(value.getRow());
122             replicatedScanner = replicatedTable.getScanner(scan);
123             return null;
124           }
125         });
126       }
127       Result res = replicatedScanner.next();
128       try {
129         Result.compareResults(value, res);
130         context.getCounter(Counters.GOODROWS).increment(1);
131       } catch (Exception e) {
132         LOG.warn("Bad row", e);
133         context.getCounter(Counters.BADROWS).increment(1);
134       }
135     }
136 
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     ReplicationPeer 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       Configuration peerConf = rp.getPeerConf(peerId);
159       if (peerConf == null) {
160         throw new IOException("Couldn't get peer conf!");
161       }
162 
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     if (startTime != 0) {
211       scan.setTimeRange(startTime,
212           endTime == 0 ? HConstants.LATEST_TIMESTAMP : endTime);
213     }
214     if(families != null) {
215       String[] fams = families.split(",");
216       for(String fam : fams) {
217         scan.addFamily(Bytes.toBytes(fam));
218       }
219     }
220     TableMapReduceUtil.initTableMapperJob(tableName, scan,
221         Verifier.class, null, null, job);
222 
223     // Obtain the auth token from peer cluster
224     TableMapReduceUtil.initCredentialsForCluster(job, peerQuorumAddress);
225 
226     job.setOutputFormatClass(NullOutputFormat.class);
227     job.setNumReduceTasks(0);
228     return job;
229   }
230 
231   private static boolean doCommandLine(final String[] args) {
232     if (args.length < 2) {
233       printUsage(null);
234       return false;
235     }
236     try {
237       for (int i = 0; i < args.length; i++) {
238         String cmd = args[i];
239         if (cmd.equals("-h") || cmd.startsWith("--h")) {
240           printUsage(null);
241           return false;
242         }
243 
244         final String startTimeArgKey = "--starttime=";
245         if (cmd.startsWith(startTimeArgKey)) {
246           startTime = Long.parseLong(cmd.substring(startTimeArgKey.length()));
247           continue;
248         }
249 
250         final String endTimeArgKey = "--endtime=";
251         if (cmd.startsWith(endTimeArgKey)) {
252           endTime = Long.parseLong(cmd.substring(endTimeArgKey.length()));
253           continue;
254         }
255 
256         final String familiesArgKey = "--families=";
257         if (cmd.startsWith(familiesArgKey)) {
258           families = cmd.substring(familiesArgKey.length());
259           continue;
260         }
261 
262         if (i == args.length-2) {
263           peerId = cmd;
264         }
265 
266         if (i == args.length-1) {
267           tableName = cmd;
268         }
269       }
270     } catch (Exception e) {
271       e.printStackTrace();
272       printUsage("Can't start because " + e.getMessage());
273       return false;
274     }
275     return true;
276   }
277 
278   /*
279    * @param errorMsg Error message.  Can be null.
280    */
281   private static void printUsage(final String errorMsg) {
282     if (errorMsg != null && errorMsg.length() > 0) {
283       System.err.println("ERROR: " + errorMsg);
284     }
285     System.err.println("Usage: verifyrep [--starttime=X]" +
286         " [--stoptime=Y] [--families=A] <peerid> <tablename>");
287     System.err.println();
288     System.err.println("Options:");
289     System.err.println(" starttime    beginning of the time range");
290     System.err.println("              without endtime means from starttime to forever");
291     System.err.println(" stoptime     end of the time range");
292     System.err.println(" families     comma-separated list of families to copy");
293     System.err.println();
294     System.err.println("Args:");
295     System.err.println(" peerid       Id of the peer used for verification, must match the one given for replication");
296     System.err.println(" tablename    Name of the table to verify");
297     System.err.println();
298     System.err.println("Examples:");
299     System.err.println(" To verify the data replicated from TestTable for a 1 hour window with peer #5 ");
300     System.err.println(" $ bin/hbase " +
301         "org.apache.hadoop.hbase.mapreduce.replication.VerifyReplication" +
302         " --starttime=1265875194289 --stoptime=1265878794289 5 TestTable ");
303   }
304 
305   @Override
306   public int run(String[] args) throws Exception {
307     Configuration conf = this.getConf();
308     Job job = createSubmittableJob(conf, args);
309     if (job != null) {
310       return job.waitForCompletion(true) ? 0 : 1;
311     } 
312     return 1;
313   }
314 
315   /**
316    * Main entry point.
317    *
318    * @param args  The command line parameters.
319    * @throws Exception When running the job fails.
320    */
321   public static void main(String[] args) throws Exception {
322     int res = ToolRunner.run(HBaseConfiguration.create(), new VerifyReplication(), args);
323     System.exit(res);
324   }
325 }