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