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