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