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    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
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.mapreduce;
19  
20  import static org.junit.Assert.assertEquals;
21  import static org.junit.Assert.assertTrue;
22  import static org.junit.Assert.fail;
23  
24  import java.io.IOException;
25  import java.nio.ByteBuffer;
26  import java.util.Collection;
27  import java.util.Deque;
28  import java.util.List;
29  import java.util.NavigableMap;
30  import java.util.concurrent.ExecutorService;
31  import java.util.concurrent.atomic.AtomicInteger;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.fs.FileSystem;
37  import org.apache.hadoop.fs.Path;
38  import org.apache.hadoop.hbase.HConstants;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.HBaseTestingUtility;
41  import org.apache.hadoop.hbase.HColumnDescriptor;
42  import org.apache.hadoop.hbase.HRegionInfo;
43  import org.apache.hadoop.hbase.HRegionLocation;
44  import org.apache.hadoop.hbase.HTableDescriptor;
45  import org.apache.hadoop.hbase.LargeTests;
46  import org.apache.hadoop.hbase.ServerName;
47  import org.apache.hadoop.hbase.TableExistsException;
48  import org.apache.hadoop.hbase.catalog.CatalogTracker;
49  import org.apache.hadoop.hbase.catalog.MetaEditor;
50  import org.apache.hadoop.hbase.catalog.MetaReader;
51  import org.apache.hadoop.hbase.client.HConnection;
52  import org.apache.hadoop.hbase.client.HTable;
53  import org.apache.hadoop.hbase.client.Result;
54  import org.apache.hadoop.hbase.client.ResultScanner;
55  import org.apache.hadoop.hbase.client.Scan;
56  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
57  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
58  import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest;
59  import org.apache.hadoop.hbase.regionserver.HRegionServer;
60  import org.apache.hadoop.hbase.regionserver.TestHRegionServerBulkLoad;
61  import org.apache.hadoop.hbase.util.Bytes;
62  import org.apache.hadoop.hbase.util.Pair;
63  import org.junit.AfterClass;
64  import org.junit.BeforeClass;
65  import org.junit.Test;
66  import org.junit.experimental.categories.Category;
67  import org.mockito.Mockito;
68  
69  import com.google.common.collect.Multimap;
70  import com.google.protobuf.RpcController;
71  import com.google.protobuf.ServiceException;
72  
73  /**
74   * Test cases for the atomic load error handling of the bulk load functionality.
75   */
76  @Category(LargeTests.class)
77  public class TestLoadIncrementalHFilesSplitRecovery {
78    final static Log LOG = LogFactory.getLog(TestHRegionServerBulkLoad.class);
79  
80    static HBaseTestingUtility util;
81    //used by secure subclass
82    static boolean useSecure = false;
83  
84    final static int NUM_CFS = 10;
85    final static byte[] QUAL = Bytes.toBytes("qual");
86    final static int ROWCOUNT = 100;
87  
88    private final static byte[][] families = new byte[NUM_CFS][];
89    static {
90      for (int i = 0; i < NUM_CFS; i++) {
91        families[i] = Bytes.toBytes(family(i));
92      }
93    }
94  
95    static byte[] rowkey(int i) {
96      return Bytes.toBytes(String.format("row_%08d", i));
97    }
98  
99    static String family(int i) {
100     return String.format("family_%04d", i);
101   }
102 
103   static byte[] value(int i) {
104     return Bytes.toBytes(String.format("%010d", i));
105   }
106 
107   public static void buildHFiles(FileSystem fs, Path dir, int value)
108       throws IOException {
109     byte[] val = value(value);
110     for (int i = 0; i < NUM_CFS; i++) {
111       Path testIn = new Path(dir, family(i));
112 
113       TestHRegionServerBulkLoad.createHFile(fs, new Path(testIn, "hfile_" + i),
114           Bytes.toBytes(family(i)), QUAL, val, ROWCOUNT);
115     }
116   }
117 
118   /**
119    * Creates a table with given table name and specified number of column
120    * families if the table does not already exist.
121    */
122   private void setupTable(String table, int cfs) throws IOException {
123     try {
124       LOG.info("Creating table " + table);
125       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
126       for (int i = 0; i < cfs; i++) {
127         htd.addFamily(new HColumnDescriptor(family(i)));
128       }
129 
130       util.getHBaseAdmin().createTable(htd);
131     } catch (TableExistsException tee) {
132       LOG.info("Table " + table + " already exists");
133     }
134   }
135 
136   /**
137    * Creates a table with given table name,specified number of column families<br>
138    * and splitkeys if the table does not already exist.
139    * @param table
140    * @param cfs
141    * @param SPLIT_KEYS
142    */
143   private void setupTableWithSplitkeys(String table, int cfs, byte[][] SPLIT_KEYS)
144       throws IOException {
145     try {
146       LOG.info("Creating table " + table);
147       HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(table));
148       for (int i = 0; i < cfs; i++) {
149         htd.addFamily(new HColumnDescriptor(family(i)));
150       }
151 
152       util.getHBaseAdmin().createTable(htd, SPLIT_KEYS);
153     } catch (TableExistsException tee) {
154       LOG.info("Table " + table + " already exists");
155     }
156   }
157 
158   private Path buildBulkFiles(String table, int value) throws Exception {
159     Path dir = util.getDataTestDirOnTestFS(table);
160     Path bulk1 = new Path(dir, table+value);
161     FileSystem fs = util.getTestFileSystem();
162     buildHFiles(fs, bulk1, value);
163     return bulk1;
164   }
165 
166   /**
167    * Populate table with known values.
168    */
169   private void populateTable(String table, int value) throws Exception {
170     // create HFiles for different column families
171     LoadIncrementalHFiles lih = new LoadIncrementalHFiles(util.getConfiguration());
172     Path bulk1 = buildBulkFiles(table, value);
173     HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table));
174     lih.doBulkLoad(bulk1, t);
175   }
176 
177   /**
178    * Split the known table in half.  (this is hard coded for this test suite)
179    */
180   private void forceSplit(String table) {
181     try {
182       // need to call regions server to by synchronous but isn't visible.
183       HRegionServer hrs = util.getRSForFirstRegionInTable(Bytes
184           .toBytes(table));
185 
186       for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
187         if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) {
188           // splitRegion doesn't work if startkey/endkey are null
189           ProtobufUtil.split(hrs, hri, rowkey(ROWCOUNT / 2)); // hard code split
190         }
191       }
192 
193       // verify that split completed.
194       int regions;
195       do {
196         regions = 0;
197         for (HRegionInfo hri : ProtobufUtil.getOnlineRegions(hrs)) {
198           if (Bytes.equals(hri.getTable().getName(), Bytes.toBytes(table))) {
199             regions++;
200           }
201         }
202         if (regions != 2) {
203           LOG.info("Taking some time to complete split...");
204           Thread.sleep(250);
205         }
206       } while (regions != 2);
207     } catch (IOException e) {
208       e.printStackTrace();
209     } catch (InterruptedException e) {
210       e.printStackTrace();
211     }
212   }
213 
214   @BeforeClass
215   public static void setupCluster() throws Exception {
216     util = new HBaseTestingUtility();
217     util.startMiniCluster(1);
218   }
219 
220   @AfterClass
221   public static void teardownCluster() throws Exception {
222     util.shutdownMiniCluster();
223   }
224 
225   /**
226    * Checks that all columns have the expected value and that there is the
227    * expected number of rows.
228    * @throws IOException 
229    */
230   void assertExpectedTable(String table, int count, int value) throws IOException {
231     HTable t = null;
232     try {
233       assertEquals(util.getHBaseAdmin().listTables(table).length, 1);
234       t = new HTable(util.getConfiguration(), table);
235       Scan s = new Scan();
236       ResultScanner sr = t.getScanner(s);
237       int i = 0;
238       for (Result r : sr) {
239         i++;
240         for (NavigableMap<byte[], byte[]> nm : r.getNoVersionMap().values()) {
241           for (byte[] val : nm.values()) {
242             assertTrue(Bytes.equals(val, value(value)));
243           }
244         }
245       }
246       assertEquals(count, i);
247     } catch (IOException e) {
248       fail("Failed due to exception");
249     } finally {
250       if (t != null) t.close();
251     }
252   }
253 
254   /**
255    * Test that shows that exception thrown from the RS side will result in an
256    * exception on the LIHFile client.
257    */
258   @Test(expected=IOException.class)
259   public void testBulkLoadPhaseFailure() throws Exception {
260     String table = "bulkLoadPhaseFailure";
261     setupTable(table, 10);
262 
263     final AtomicInteger attmptedCalls = new AtomicInteger();
264     final AtomicInteger failedCalls = new AtomicInteger();
265     LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
266         util.getConfiguration()) {
267 
268       protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
269           TableName tableName, final byte[] first, Collection<LoadQueueItem> lqis)
270       throws IOException {
271         int i = attmptedCalls.incrementAndGet();
272         if (i == 1) {
273           HConnection errConn = null;
274           try {
275             errConn = getMockedConnection(util.getConfiguration());
276           } catch (Exception e) {
277             LOG.fatal("mocking cruft, should never happen", e);
278             throw new RuntimeException("mocking cruft, should never happen");
279           }
280           failedCalls.incrementAndGet();
281           return super.tryAtomicRegionLoad(errConn, tableName, first, lqis);
282         }
283 
284         return super.tryAtomicRegionLoad(conn, tableName, first, lqis);
285       }
286     };
287 
288     // create HFiles for different column families
289     Path dir = buildBulkFiles(table, 1);
290     HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table));
291     lih.doBulkLoad(dir, t);
292 
293     fail("doBulkLoad should have thrown an exception");
294   }
295 
296   private HConnection getMockedConnection(final Configuration conf)
297   throws IOException, ServiceException {
298     HConnection c = Mockito.mock(HConnection.class);
299     Mockito.when(c.getConfiguration()).thenReturn(conf);
300     Mockito.doNothing().when(c).close();
301     // Make it so we return a particular location when asked.
302     final HRegionLocation loc = new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO,
303         ServerName.valueOf("example.org", 1234, 0));
304     Mockito.when(c.getRegionLocation((TableName) Mockito.any(),
305         (byte[]) Mockito.any(), Mockito.anyBoolean())).
306       thenReturn(loc);
307     Mockito.when(c.locateRegion((TableName) Mockito.any(), (byte[]) Mockito.any())).
308       thenReturn(loc);
309     ClientProtos.ClientService.BlockingInterface hri =
310       Mockito.mock(ClientProtos.ClientService.BlockingInterface.class);
311     Mockito.when(hri.bulkLoadHFile((RpcController)Mockito.any(), (BulkLoadHFileRequest)Mockito.any())).
312       thenThrow(new ServiceException(new IOException("injecting bulk load error")));
313     Mockito.when(c.getClient(Mockito.any(ServerName.class))).
314       thenReturn(hri);
315     return c;
316   }
317 
318   /**
319    * This test exercises the path where there is a split after initial
320    * validation but before the atomic bulk load call. We cannot use presplitting
321    * to test this path, so we actually inject a split just before the atomic
322    * region load.
323    */
324   @Test
325   public void testSplitWhileBulkLoadPhase() throws Exception {
326     final String table = "splitWhileBulkloadPhase";
327     setupTable(table, 10);
328     populateTable(table,1);
329     assertExpectedTable(table, ROWCOUNT, 1);
330 
331     // Now let's cause trouble.  This will occur after checks and cause bulk
332     // files to fail when attempt to atomically import.  This is recoverable.
333     final AtomicInteger attemptedCalls = new AtomicInteger();
334     LoadIncrementalHFiles lih2 = new LoadIncrementalHFiles(
335         util.getConfiguration()) {
336 
337       protected void bulkLoadPhase(final HTable htable, final HConnection conn,
338           ExecutorService pool, Deque<LoadQueueItem> queue,
339           final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
340         int i = attemptedCalls.incrementAndGet();
341         if (i == 1) {
342           // On first attempt force a split.
343           forceSplit(table);
344         }
345 
346         super.bulkLoadPhase(htable, conn, pool, queue, regionGroups);
347       }
348     };
349 
350     // create HFiles for different column families
351     HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table));
352     Path bulk = buildBulkFiles(table, 2);
353     lih2.doBulkLoad(bulk, t);
354 
355     // check that data was loaded
356     // The three expected attempts are 1) failure because need to split, 2)
357     // load of split top 3) load of split bottom
358     assertEquals(attemptedCalls.get(), 3);
359     assertExpectedTable(table, ROWCOUNT, 2);
360   }
361 
362   /**
363    * This test splits a table and attempts to bulk load.  The bulk import files
364    * should be split before atomically importing.
365    */
366   @Test
367   public void testGroupOrSplitPresplit() throws Exception {
368     final String table = "groupOrSplitPresplit";
369     setupTable(table, 10);
370     populateTable(table, 1);
371     assertExpectedTable(table, ROWCOUNT, 1);
372     forceSplit(table);
373 
374     final AtomicInteger countedLqis= new AtomicInteger();
375     LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
376         util.getConfiguration()) {
377       protected List<LoadQueueItem> groupOrSplit(
378           Multimap<ByteBuffer, LoadQueueItem> regionGroups,
379           final LoadQueueItem item, final HTable htable,
380           final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
381         List<LoadQueueItem> lqis = super.groupOrSplit(regionGroups, item, htable, startEndKeys);
382         if (lqis != null) {
383           countedLqis.addAndGet(lqis.size());
384         }
385         return lqis;
386       }
387     };
388 
389     // create HFiles for different column families
390     Path bulk = buildBulkFiles(table, 2);
391     HTable ht = new HTable(util.getConfiguration(), Bytes.toBytes(table));
392     lih.doBulkLoad(bulk, ht);
393 
394     assertExpectedTable(table, ROWCOUNT, 2);
395     assertEquals(20, countedLqis.get());
396   }
397 
398   /**
399    * This simulates an remote exception which should cause LIHF to exit with an
400    * exception.
401    */
402   @Test(expected = IOException.class)
403   public void testGroupOrSplitFailure() throws Exception {
404     String table = "groupOrSplitFailure";
405     setupTable(table, 10);
406 
407     LoadIncrementalHFiles lih = new LoadIncrementalHFiles(
408         util.getConfiguration()) {
409       int i = 0;
410 
411       protected List<LoadQueueItem> groupOrSplit(
412           Multimap<ByteBuffer, LoadQueueItem> regionGroups,
413           final LoadQueueItem item, final HTable table,
414           final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
415         i++;
416 
417         if (i == 5) {
418           throw new IOException("failure");
419         }
420         return super.groupOrSplit(regionGroups, item, table, startEndKeys);
421       }
422     };
423 
424     // create HFiles for different column families
425     Path dir = buildBulkFiles(table,1);
426     HTable t = new HTable(util.getConfiguration(), Bytes.toBytes(table));
427     lih.doBulkLoad(dir, t);
428 
429     fail("doBulkLoad should have thrown an exception");
430   }
431 
432   @Test
433   public void testGroupOrSplitWhenRegionHoleExistsInMeta() throws Exception {
434     String tableName = "testGroupOrSplitWhenRegionHoleExistsInMeta";
435     byte[][] SPLIT_KEYS = new byte[][] { Bytes.toBytes("row_00000100") };
436 
437     setupTableWithSplitkeys(tableName, 10, SPLIT_KEYS);
438     HTable table = new HTable(util.getConfiguration(), Bytes.toBytes(tableName));
439     Path dir = buildBulkFiles(tableName, 2);
440 
441     final AtomicInteger countedLqis = new AtomicInteger();
442     LoadIncrementalHFiles loader = new LoadIncrementalHFiles(
443       util.getConfiguration()) {
444       
445     protected List<LoadQueueItem> groupOrSplit(
446         Multimap<ByteBuffer, LoadQueueItem> regionGroups,
447         final LoadQueueItem item, final HTable htable,
448         final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
449       List<LoadQueueItem> lqis = super.groupOrSplit(regionGroups, item, htable, startEndKeys);
450       if (lqis != null) {
451         countedLqis.addAndGet(lqis.size());
452       }
453       return lqis;
454     }
455   };
456 
457     // do bulkload when there is no region hole in hbase:meta.
458     try {
459       loader.doBulkLoad(dir, table);
460     } catch (Exception e) {
461       LOG.error("exeception=", e);
462     }
463     // check if all the data are loaded into the table.
464     this.assertExpectedTable(tableName, ROWCOUNT, 2);
465 
466     dir = buildBulkFiles(tableName, 3);
467 
468     // Mess it up by leaving a hole in the hbase:meta
469     CatalogTracker ct = new CatalogTracker(util.getConfiguration());
470     List<HRegionInfo> regionInfos = MetaReader.getTableRegions(ct, TableName.valueOf(tableName));
471     for (HRegionInfo regionInfo : regionInfos) {
472       if (Bytes.equals(regionInfo.getStartKey(), HConstants.EMPTY_BYTE_ARRAY)) {
473         MetaEditor.deleteRegion(ct, regionInfo);
474         break;
475       }
476     }
477 
478     try {
479       loader.doBulkLoad(dir, table);
480     } catch (Exception e) {
481       LOG.error("exeception=", e);
482       assertTrue("IOException expected", e instanceof IOException);
483     }
484 
485     table.close();
486 
487     this.assertExpectedTable(tableName, ROWCOUNT, 2);
488   }
489 }
490