View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    * <p/>
10   * http://www.apache.org/licenses/LICENSE-2.0
11   * <p/>
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.test;
19  
20  import java.io.IOException;
21  import java.util.Random;
22  import java.util.Set;
23  import java.util.UUID;
24  import java.util.regex.Matcher;
25  import java.util.regex.Pattern;
26  
27  import com.google.common.collect.Lists;
28  import com.google.common.collect.Sets;
29  import org.apache.commons.cli.CommandLine;
30  import org.apache.hadoop.conf.Configuration;
31  import org.apache.hadoop.fs.FileSystem;
32  import org.apache.hadoop.fs.Path;
33  import org.apache.hadoop.hbase.HBaseConfiguration;
34  import org.apache.hadoop.hbase.HColumnDescriptor;
35  import org.apache.hadoop.hbase.HConstants;
36  import org.apache.hadoop.hbase.HTableDescriptor;
37  import org.apache.hadoop.hbase.IntegrationTestBase;
38  import org.apache.hadoop.hbase.IntegrationTestingUtility;
39  import org.apache.hadoop.hbase.IntegrationTests;
40  import org.apache.hadoop.hbase.KeyValue;
41  import org.apache.hadoop.hbase.TableName;
42  import org.apache.hadoop.hbase.client.HBaseAdmin;
43  import org.apache.hadoop.hbase.client.HTable;
44  import org.apache.hadoop.hbase.client.Put;
45  import org.apache.hadoop.hbase.client.Result;
46  import org.apache.hadoop.hbase.client.Scan;
47  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
48  import org.apache.hadoop.hbase.mapreduce.NMapInputFormat;
49  import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
50  import org.apache.hadoop.hbase.mapreduce.TableMapper;
51  import org.apache.hadoop.hbase.util.Bytes;
52  import org.apache.hadoop.io.BytesWritable;
53  import org.apache.hadoop.io.NullWritable;
54  import org.apache.hadoop.io.Text;
55  import org.apache.hadoop.mapreduce.Counter;
56  import org.apache.hadoop.mapreduce.Job;
57  import org.apache.hadoop.mapreduce.Mapper;
58  import org.apache.hadoop.mapreduce.Reducer;
59  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
60  import org.apache.hadoop.util.ToolRunner;
61  import org.junit.After;
62  import org.junit.Before;
63  import org.junit.Test;
64  import org.junit.experimental.categories.Category;
65  
66  import static org.junit.Assert.assertEquals;
67  import static org.junit.Assert.assertTrue;
68  
69  /**
70   * A large test which loads a lot of data that has internal references, and
71   * verifies the data.
72   *
73   * In load step, 200 map tasks are launched, which in turn write loadmapper.num_to_write
74   * (default 100K) rows to an hbase table. Rows are written in blocks, for a total of
75   * 100 blocks. Each row in a block, contains loadmapper.backrefs (default 50) references
76   * to random rows in the prev block.
77   *
78   * Verify step is scans the table, and verifies that for every referenced row, the row is
79   * actually there (no data loss). Failed rows are output from reduce to be saved in the
80   * job output dir in hdfs and inspected later.
81   *
82   * This class can be run as a unit test, as an integration test, or from the command line
83   *
84   * Originally taken from Apache Bigtop.
85   */
86  @Category(IntegrationTests.class)
87  public class IntegrationTestLoadAndVerify  extends IntegrationTestBase  {
88    private static final String TEST_NAME = "IntegrationTestLoadAndVerify";
89    private static final byte[] TEST_FAMILY = Bytes.toBytes("f1");
90    private static final byte[] TEST_QUALIFIER = Bytes.toBytes("q1");
91  
92    private static final String NUM_TO_WRITE_KEY =
93      "loadmapper.num_to_write";
94    private static final long NUM_TO_WRITE_DEFAULT = 100*1000;
95  
96    private static final String TABLE_NAME_KEY = "loadmapper.table";
97    private static final String TABLE_NAME_DEFAULT = "table";
98  
99    private static final String NUM_BACKREFS_KEY = "loadmapper.backrefs";
100   private static final int NUM_BACKREFS_DEFAULT = 50;
101 
102   private static final String NUM_MAP_TASKS_KEY = "loadmapper.map.tasks";
103   private static final String NUM_REDUCE_TASKS_KEY = "verify.reduce.tasks";
104   private static final int NUM_MAP_TASKS_DEFAULT = 200;
105   private static final int NUM_REDUCE_TASKS_DEFAULT = 35;
106 
107   private static final int SCANNER_CACHING = 500;
108 
109   private IntegrationTestingUtility util;
110 
111   private String toRun = null;
112 
113   private enum Counters {
114     ROWS_WRITTEN,
115     REFERENCES_WRITTEN,
116     REFERENCES_CHECKED;
117   }
118 
119   @Before
120   public void setUp() throws Exception {
121     util = getTestingUtil(getConf());
122     util.initializeCluster(3);
123     this.setConf(util.getConfiguration());
124     getConf().setLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT / 100);
125     getConf().setInt(NUM_MAP_TASKS_KEY, NUM_MAP_TASKS_DEFAULT / 100);
126     getConf().setInt(NUM_REDUCE_TASKS_KEY, NUM_REDUCE_TASKS_DEFAULT / 10);
127   }
128 
129   /**
130    * Converts a "long" value between endian systems.
131    * Borrowed from Apache Commons IO
132    * @param value value to convert
133    * @return the converted value
134    */
135   public static long swapLong(long value)
136   {
137     return
138       ( ( ( value >> 0 ) & 0xff ) << 56 ) +
139       ( ( ( value >> 8 ) & 0xff ) << 48 ) +
140       ( ( ( value >> 16 ) & 0xff ) << 40 ) +
141       ( ( ( value >> 24 ) & 0xff ) << 32 ) +
142       ( ( ( value >> 32 ) & 0xff ) << 24 ) +
143       ( ( ( value >> 40 ) & 0xff ) << 16 ) +
144       ( ( ( value >> 48 ) & 0xff ) << 8 ) +
145       ( ( ( value >> 56 ) & 0xff ) << 0 );
146   }
147 
148   @Override
149   @After
150   public void cleanUp() throws Exception {
151     util.restoreCluster();
152   }
153 
154   public static class LoadMapper
155       extends Mapper<NullWritable, NullWritable, NullWritable, NullWritable>
156   {
157     private long recordsToWrite;
158     private HTable table;
159     private Configuration conf;
160     private int numBackReferencesPerRow;
161     private String shortTaskId;
162 
163     private Random rand = new Random();
164 
165     private Counter rowsWritten, refsWritten;
166 
167     @Override
168     public void setup(Context context) throws IOException {
169       conf = context.getConfiguration();
170       recordsToWrite = conf.getLong(NUM_TO_WRITE_KEY, NUM_TO_WRITE_DEFAULT);
171       String tableName = conf.get(TABLE_NAME_KEY, TABLE_NAME_DEFAULT);
172       numBackReferencesPerRow = conf.getInt(NUM_BACKREFS_KEY, NUM_BACKREFS_DEFAULT);
173       table = new HTable(conf, tableName);
174       table.setWriteBufferSize(4*1024*1024);
175       table.setAutoFlush(false);
176 
177       String taskId = conf.get("mapred.task.id");
178       Matcher matcher = Pattern.compile(".+_m_(\\d+_\\d+)").matcher(taskId);
179       if (!matcher.matches()) {
180         throw new RuntimeException("Strange task ID: " + taskId);
181       }
182       shortTaskId = matcher.group(1);
183 
184       rowsWritten = context.getCounter(Counters.ROWS_WRITTEN);
185       refsWritten = context.getCounter(Counters.REFERENCES_WRITTEN);
186     }
187 
188     @Override
189     public void cleanup(Context context) throws IOException {
190       table.flushCommits();
191       table.close();
192     }
193 
194     @Override
195     protected void map(NullWritable key, NullWritable value,
196         Context context) throws IOException, InterruptedException {
197 
198       String suffix = "/" + shortTaskId;
199       byte[] row = Bytes.add(new byte[8], Bytes.toBytes(suffix));
200 
201       int BLOCK_SIZE = (int)(recordsToWrite / 100);
202 
203       for (long i = 0; i < recordsToWrite;) {
204         long blockStart = i;
205         for (long idxInBlock = 0;
206              idxInBlock < BLOCK_SIZE && i < recordsToWrite;
207              idxInBlock++, i++) {
208 
209           long byteSwapped = swapLong(i);
210           Bytes.putLong(row, 0, byteSwapped);
211 
212           Put p = new Put(row);
213           p.add(TEST_FAMILY, TEST_QUALIFIER, HConstants.EMPTY_BYTE_ARRAY);
214           if (blockStart > 0) {
215             for (int j = 0; j < numBackReferencesPerRow; j++) {
216               long referredRow = blockStart - BLOCK_SIZE + rand.nextInt(BLOCK_SIZE);
217               Bytes.putLong(row, 0, swapLong(referredRow));
218               p.add(TEST_FAMILY, row, HConstants.EMPTY_BYTE_ARRAY);
219             }
220             refsWritten.increment(1);
221           }
222           rowsWritten.increment(1);
223           table.put(p);
224 
225           if (i % 100 == 0) {
226             context.setStatus("Written " + i + "/" + recordsToWrite + " records");
227             context.progress();
228           }
229         }
230         // End of block, flush all of them before we start writing anything
231         // pointing to these!
232         table.flushCommits();
233       }
234     }
235   }
236 
237   public static class VerifyMapper extends TableMapper<BytesWritable, BytesWritable> {
238     static final BytesWritable EMPTY = new BytesWritable(HConstants.EMPTY_BYTE_ARRAY);
239 
240     @Override
241     protected void map(ImmutableBytesWritable key, Result value, Context context)
242         throws IOException, InterruptedException {
243       BytesWritable bwKey = new BytesWritable(key.get());
244       BytesWritable bwVal = new BytesWritable();
245       for (KeyValue kv : value.list()) {
246         if (Bytes.compareTo(TEST_QUALIFIER, 0, TEST_QUALIFIER.length,
247                             kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength()) == 0) {
248           context.write(bwKey, EMPTY);
249         } else {
250           bwVal.set(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength());
251           context.write(bwVal, bwKey);
252         }
253       }
254     }
255   }
256 
257   public static class VerifyReducer extends Reducer<BytesWritable, BytesWritable, Text, Text> {
258     private Counter refsChecked;
259     private Counter rowsWritten;
260 
261     @Override
262     public void setup(Context context) throws IOException {
263       refsChecked = context.getCounter(Counters.REFERENCES_CHECKED);
264       rowsWritten = context.getCounter(Counters.ROWS_WRITTEN);
265     }
266 
267     @Override
268     protected void reduce(BytesWritable referredRow, Iterable<BytesWritable> referrers,
269         VerifyReducer.Context ctx) throws IOException, InterruptedException {
270       boolean gotOriginalRow = false;
271       int refCount = 0;
272 
273       for (BytesWritable ref : referrers) {
274         if (ref.getLength() == 0) {
275           assert !gotOriginalRow;
276           gotOriginalRow = true;
277         } else {
278           refCount++;
279         }
280       }
281       refsChecked.increment(refCount);
282 
283       if (!gotOriginalRow) {
284         String parsedRow = makeRowReadable(referredRow.getBytes(), referredRow.getLength());
285         String binRow = Bytes.toStringBinary(referredRow.getBytes(), 0, referredRow.getLength());
286         ctx.write(new Text(binRow), new Text(parsedRow));
287         rowsWritten.increment(1);
288       }
289     }
290 
291     private String makeRowReadable(byte[] bytes, int length) {
292       long rowIdx = swapLong(Bytes.toLong(bytes, 0));
293       String suffix = Bytes.toString(bytes, 8, length - 8);
294 
295       return "Row #" + rowIdx + " suffix " + suffix;
296     }
297   }
298 
299   private void doLoad(Configuration conf, HTableDescriptor htd) throws Exception {
300     Path outputDir = getTestDir(TEST_NAME, "load-output");
301 
302     NMapInputFormat.setNumMapTasks(conf, conf.getInt(NUM_MAP_TASKS_KEY, NUM_MAP_TASKS_DEFAULT));
303     conf.set(TABLE_NAME_KEY, htd.getTableName().getNameAsString());
304 
305     Job job = new Job(conf);
306     job.setJobName(TEST_NAME + " Load for " + htd.getTableName());
307     job.setJarByClass(this.getClass());
308     job.setMapperClass(LoadMapper.class);
309     job.setInputFormatClass(NMapInputFormat.class);
310     job.setNumReduceTasks(0);
311     FileOutputFormat.setOutputPath(job, outputDir);
312 
313     TableMapReduceUtil.addDependencyJars(job);
314     TableMapReduceUtil.addDependencyJars(
315         job.getConfiguration(), HTable.class, Lists.class);
316     TableMapReduceUtil.initCredentials(job);
317     assertTrue(job.waitForCompletion(true));
318   }
319 
320   private void doVerify(Configuration conf, HTableDescriptor htd) throws Exception {
321     Path outputDir = getTestDir(TEST_NAME, "verify-output");
322 
323     Job job = new Job(conf);
324     job.setJarByClass(this.getClass());
325     job.setJobName(TEST_NAME + " Verification for " + htd.getTableName());
326 
327     Scan scan = new Scan();
328 
329     TableMapReduceUtil.initTableMapperJob(
330         htd.getTableName().getNameAsString(), scan, VerifyMapper.class,
331         BytesWritable.class, BytesWritable.class, job);
332     int scannerCaching = conf.getInt("verify.scannercaching", SCANNER_CACHING);
333     TableMapReduceUtil.setScannerCaching(job, scannerCaching);
334 
335     job.setReducerClass(VerifyReducer.class);
336     job.setNumReduceTasks(conf.getInt(NUM_REDUCE_TASKS_KEY, NUM_REDUCE_TASKS_DEFAULT));
337     FileOutputFormat.setOutputPath(job, outputDir);
338     assertTrue(job.waitForCompletion(true));
339 
340     long numOutputRecords = job.getCounters().findCounter(Counters.ROWS_WRITTEN).getValue();
341     assertEquals(0, numOutputRecords);
342   }
343 
344   public Path getTestDir(String testName, String subdir) throws IOException {
345     //HBaseTestingUtility.getDataTestDirOnTestFs() has not been backported.
346     FileSystem fs = FileSystem.get(getConf());
347     Path base = new Path(fs.getWorkingDirectory(), "test-data");
348     String randomStr = UUID.randomUUID().toString();
349     Path testDir = new Path(base, randomStr);
350     fs.deleteOnExit(testDir);
351 
352     return new Path(new Path(testDir, testName), subdir);
353   }
354 
355   @Test
356   public void testLoadAndVerify() throws Exception {
357     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TEST_NAME));
358     htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
359 
360     HBaseAdmin admin = getTestingUtil(getConf()).getHBaseAdmin();
361     int numPreCreate = 40;
362     admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPreCreate);
363 
364     doLoad(getConf(), htd);
365     doVerify(getConf(), htd);
366 
367     // Only disable and drop if we succeeded to verify - otherwise it's useful
368     // to leave it around for post-mortem
369     deleteTable(admin, htd);
370   }
371 
372   private void deleteTable(HBaseAdmin admin, HTableDescriptor htd)
373     throws IOException, InterruptedException {
374     // Use disableTestAsync because disable can take a long time to complete
375     System.out.print("Disabling table " + htd.getTableName() +" ");
376     admin.disableTableAsync(htd.getTableName());
377 
378     long start = System.currentTimeMillis();
379     // NOTE tables can be both admin.isTableEnabled=false and
380     // isTableDisabled=false, when disabling must use isTableDisabled!
381     while (!admin.isTableDisabled(htd.getTableName())) {
382       System.out.print(".");
383       Thread.sleep(1000);
384     }
385     long delta = System.currentTimeMillis() - start;
386     System.out.println(" " + delta +" ms");
387     System.out.println("Deleting table " + htd.getTableName() +" ");
388     admin.deleteTable(htd.getTableName());
389   }
390 
391   public void usage() {
392     System.err.println(this.getClass().getSimpleName() + " [-Doptions] <load|verify|loadAndVerify>");
393     System.err.println("  Loads a table with row dependencies and verifies the dependency chains");
394     System.err.println("Options");
395     System.err.println("  -Dloadmapper.table=<name>        Table to write/verify (default autogen)");
396     System.err.println("  -Dloadmapper.backrefs=<n>        Number of backreferences per row (default 50)");
397     System.err.println("  -Dloadmapper.num_to_write=<n>    Number of rows per mapper (default 100,000 per mapper)");
398     System.err.println("  -Dloadmapper.deleteAfter=<bool>  Delete after a successful verify (default true)");
399     System.err.println("  -Dloadmapper.numPresplits=<n>    Number of presplit regions to start with (default 40)");
400     System.err.println("  -Dloadmapper.map.tasks=<n>       Number of map tasks for load (default 200)");
401     System.err.println("  -Dverify.reduce.tasks=<n>        Number of reduce tasks for verify (default 35)");
402     System.err.println("  -Dverify.scannercaching=<n>      Number hbase scanner caching rows to read (default 50)");
403   }
404 
405 
406   @Override
407   protected void processOptions(CommandLine cmd) {
408     super.processOptions(cmd);
409 
410     String[] args = cmd.getArgs();
411     if (args == null || args.length < 1 || args.length > 1) {
412       usage();
413       throw new RuntimeException("Incorrect Number of args.");
414     }
415     toRun = args[0];
416   }
417 
418   public int runTestFromCommandLine() throws Exception {
419     IntegrationTestingUtility.setUseDistributedCluster(getConf());
420     boolean doLoad = false;
421     boolean doVerify = false;
422     boolean doDelete = getConf().getBoolean("loadmapper.deleteAfter",true);
423     int numPresplits = getConf().getInt("loadmapper.numPresplits", 40);
424 
425     if (toRun.equals("load")) {
426       doLoad = true;
427     } else if (toRun.equals("verify")) {
428       doVerify= true;
429     } else if (toRun.equals("loadAndVerify")) {
430       doLoad=true;
431       doVerify= true;
432     } else {
433       System.err.println("Invalid argument " + toRun);
434       usage();
435       return 1;
436     }
437 
438     // create HTableDescriptor for specified table
439     String table = getTablename();
440     HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
441     htd.addFamily(new HColumnDescriptor(TEST_FAMILY));
442 
443     HBaseAdmin admin = new HBaseAdmin(getConf());
444     if (doLoad) {
445       admin.createTable(htd, Bytes.toBytes(0L), Bytes.toBytes(-1L), numPresplits);
446       doLoad(getConf(), htd);
447     }
448     if (doVerify) {
449       doVerify(getConf(), htd);
450       if (doDelete) {
451         deleteTable(admin, htd);
452       }
453     }
454     return 0;
455   }
456 
457   @Override
458   public String getTablename() {
459     return getConf().get(TABLE_NAME_KEY, TEST_NAME);
460   }
461 
462   @Override
463   protected Set<String> getColumnFamilies() {
464     return Sets.newHashSet(Bytes.toString(TEST_FAMILY));
465   }
466 
467   public static void main(String argv[]) throws Exception {
468     Configuration conf = HBaseConfiguration.create();
469     IntegrationTestingUtility.setUseDistributedCluster(conf);
470     int ret = ToolRunner.run(conf, new IntegrationTestLoadAndVerify(), argv);
471     System.exit(ret);
472   }
473 }