1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.mapreduce;
21
22 import java.io.FileNotFoundException;
23 import java.io.IOException;
24 import java.nio.ByteBuffer;
25 import java.util.ArrayList;
26 import java.util.Arrays;
27 import java.util.Collection;
28 import java.util.Deque;
29 import java.util.HashSet;
30 import java.util.LinkedList;
31 import java.util.List;
32 import java.util.Map;
33 import java.util.Map.Entry;
34 import java.util.Set;
35 import java.util.TreeMap;
36 import java.util.concurrent.Callable;
37 import java.util.concurrent.ExecutionException;
38 import java.util.concurrent.ExecutorService;
39 import java.util.concurrent.Future;
40 import java.util.concurrent.LinkedBlockingQueue;
41 import java.util.concurrent.ThreadPoolExecutor;
42 import java.util.concurrent.TimeUnit;
43 import java.util.concurrent.atomic.AtomicLong;
44
45 import org.apache.commons.logging.Log;
46 import org.apache.commons.logging.LogFactory;
47 import org.apache.hadoop.conf.Configuration;
48 import org.apache.hadoop.conf.Configured;
49 import org.apache.hadoop.fs.FileStatus;
50 import org.apache.hadoop.fs.FileSystem;
51 import org.apache.hadoop.fs.FileUtil;
52 import org.apache.hadoop.fs.Path;
53 import org.apache.hadoop.hbase.HBaseConfiguration;
54 import org.apache.hadoop.hbase.HColumnDescriptor;
55 import org.apache.hadoop.hbase.HConstants;
56 import org.apache.hadoop.hbase.HTableDescriptor;
57 import org.apache.hadoop.hbase.KeyValue;
58 import org.apache.hadoop.hbase.TableNotFoundException;
59 import org.apache.hadoop.hbase.client.HBaseAdmin;
60 import org.apache.hadoop.hbase.client.HConnection;
61 import org.apache.hadoop.hbase.client.HTable;
62 import org.apache.hadoop.hbase.client.ServerCallable;
63 import org.apache.hadoop.hbase.io.HalfStoreFileReader;
64 import org.apache.hadoop.hbase.io.Reference;
65 import org.apache.hadoop.hbase.io.Reference.Range;
66 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
67 import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
68 import org.apache.hadoop.hbase.io.hfile.HFile;
69 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
70 import org.apache.hadoop.hbase.regionserver.StoreFile;
71 import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
72 import org.apache.hadoop.hbase.util.Bytes;
73 import org.apache.hadoop.hbase.util.Pair;
74 import org.apache.hadoop.util.Tool;
75 import org.apache.hadoop.util.ToolRunner;
76
77 import com.google.common.collect.HashMultimap;
78 import com.google.common.collect.Multimap;
79 import com.google.common.collect.Multimaps;
80 import com.google.common.util.concurrent.ThreadFactoryBuilder;
81
82
83
84
85
86 public class LoadIncrementalHFiles extends Configured implements Tool {
87
88 private static Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
89 private static final int TABLE_CREATE_MAX_RETRIES = 20;
90 private static final long TABLE_CREATE_SLEEP = 60000;
91 static AtomicLong regionCount = new AtomicLong(0);
92 private HBaseAdmin hbAdmin;
93 private Configuration cfg;
94
95 public static String NAME = "completebulkload";
96
97 public LoadIncrementalHFiles(Configuration conf) throws Exception {
98 super(conf);
99 this.cfg = conf;
100 this.hbAdmin = new HBaseAdmin(conf);
101 }
102
103 private void usage() {
104 System.err.println("usage: " + NAME +
105 " /path/to/hfileoutputformat-output " +
106 "tablename");
107 }
108
109
110
111
112
113
114
115
116
117 static class LoadQueueItem {
118 final byte[] family;
119 final Path hfilePath;
120
121 public LoadQueueItem(byte[] family, Path hfilePath) {
122 this.family = family;
123 this.hfilePath = hfilePath;
124 }
125
126 public String toString() {
127 return "family:"+ Bytes.toString(family) + " path:" + hfilePath.toString();
128 }
129 }
130
131
132
133
134
135 private void discoverLoadQueue(Deque<LoadQueueItem> ret, Path hfofDir)
136 throws IOException {
137 FileSystem fs = hfofDir.getFileSystem(getConf());
138
139 if (!fs.exists(hfofDir)) {
140 throw new FileNotFoundException("HFileOutputFormat dir " +
141 hfofDir + " not found");
142 }
143
144 FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
145 if (familyDirStatuses == null) {
146 throw new FileNotFoundException("No families found in " + hfofDir);
147 }
148
149 for (FileStatus stat : familyDirStatuses) {
150 if (!stat.isDir()) {
151 LOG.warn("Skipping non-directory " + stat.getPath());
152 continue;
153 }
154 Path familyDir = stat.getPath();
155
156 if (familyDir.getName().startsWith("_")) continue;
157 byte[] family = familyDir.getName().getBytes();
158 Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
159 for (Path hfile : hfiles) {
160 if (hfile.getName().startsWith("_")) continue;
161 ret.add(new LoadQueueItem(family, hfile));
162 }
163 }
164 }
165
166
167
168
169
170
171
172
173
174
175 public void doBulkLoad(Path hfofDir, final HTable table)
176 throws TableNotFoundException, IOException
177 {
178 final HConnection conn = table.getConnection();
179
180 if (!conn.isTableAvailable(table.getTableName())) {
181 throw new TableNotFoundException("Table " +
182 Bytes.toStringBinary(table.getTableName()) +
183 "is not currently available.");
184 }
185
186
187 int nrThreads = cfg.getInt("hbase.loadincremental.threads.max",
188 Runtime.getRuntime().availableProcessors());
189 ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
190 builder.setNameFormat("LoadIncrementalHFiles-%1$d");
191 ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads,
192 60, TimeUnit.SECONDS,
193 new LinkedBlockingQueue<Runnable>(),
194 builder.build());
195 ((ThreadPoolExecutor)pool).allowCoreThreadTimeOut(true);
196
197
198
199 Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
200 try {
201 discoverLoadQueue(queue, hfofDir);
202 int count = 0;
203
204 if (queue.isEmpty()) {
205 LOG.warn("Bulk load operation did not find any files to load in " +
206 "directory " + hfofDir.toUri() + ". Does it contain files in " +
207 "subdirectories that correspond to column family names?");
208 return;
209 }
210
211 if (queue.isEmpty()) {
212 LOG.warn("Bulk load operation did not find any files to load in " +
213 "directory " + hfofDir.toUri() + ". Does it contain files in " +
214 "subdirectories that correspond to column family names?");
215 }
216
217
218 while (!queue.isEmpty()) {
219
220 final Pair<byte[][], byte[][]> startEndKeys = table.getStartEndKeys();
221 if (count != 0) {
222 LOG.info("Split occured while grouping HFiles, retry attempt " +
223 + count + " with " + queue.size() + " files remaining to group or split");
224 }
225
226 int maxRetries = cfg.getInt("hbase.bulkload.retries.number", 0);
227 if (maxRetries != 0 && count >= maxRetries) {
228 LOG.error("Retry attempted " + count + " times without completing, bailing out");
229 return;
230 }
231 count++;
232
233
234 Multimap<ByteBuffer, LoadQueueItem> regionGroups = groupOrSplitPhase(table,
235 pool, queue, startEndKeys);
236
237 bulkLoadPhase(table, conn, pool, queue, regionGroups);
238
239
240
241
242 }
243
244 } finally {
245 pool.shutdown();
246 if (queue != null && !queue.isEmpty()) {
247 StringBuilder err = new StringBuilder();
248 err.append("-------------------------------------------------\n");
249 err.append("Bulk load aborted with some files not yet loaded:\n");
250 err.append("-------------------------------------------------\n");
251 for (LoadQueueItem q : queue) {
252 err.append(" ").append(q.hfilePath).append('\n');
253 }
254 LOG.error(err);
255 }
256 }
257 }
258
259
260
261
262
263
264 protected void bulkLoadPhase(final HTable table, final HConnection conn,
265 ExecutorService pool, Deque<LoadQueueItem> queue,
266 final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
267
268 Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<Future<List<LoadQueueItem>>>();
269 for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
270 final byte[] first = e.getKey().array();
271 final Collection<LoadQueueItem> lqis = e.getValue();
272
273 final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
274 public List<LoadQueueItem> call() throws Exception {
275 List<LoadQueueItem> toRetry = tryAtomicRegionLoad(conn, table.getTableName(), first, lqis);
276 return toRetry;
277 }
278 };
279 loadingFutures.add(pool.submit(call));
280 }
281
282
283 for (Future<List<LoadQueueItem>> future : loadingFutures) {
284 try {
285 List<LoadQueueItem> toRetry = future.get();
286
287
288 queue.addAll(toRetry);
289
290 } catch (ExecutionException e1) {
291 Throwable t = e1.getCause();
292 if (t instanceof IOException) {
293
294
295 throw new IOException("BulkLoad encountered an unrecoverable problem", t);
296 }
297 LOG.error("Unexpected execution exception during bulk load", e1);
298 throw new IllegalStateException(t);
299 } catch (InterruptedException e1) {
300 LOG.error("Unexpected interrupted exception during bulk load", e1);
301 throw new IllegalStateException(e1);
302 }
303 }
304 }
305
306
307
308
309
310 private Multimap<ByteBuffer, LoadQueueItem> groupOrSplitPhase(final HTable table,
311 ExecutorService pool, Deque<LoadQueueItem> queue,
312 final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
313
314
315 Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
316 final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
317
318
319 Set<Future<List<LoadQueueItem>>> splittingFutures = new HashSet<Future<List<LoadQueueItem>>>();
320 while (!queue.isEmpty()) {
321 final LoadQueueItem item = queue.remove();
322
323 final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
324 public List<LoadQueueItem> call() throws Exception {
325 List<LoadQueueItem> splits = groupOrSplit(regionGroups, item, table, startEndKeys);
326 return splits;
327 }
328 };
329 splittingFutures.add(pool.submit(call));
330 }
331
332
333 for (Future<List<LoadQueueItem>> lqis : splittingFutures) {
334 try {
335 List<LoadQueueItem> splits = lqis.get();
336 if (splits != null) {
337 queue.addAll(splits);
338 }
339 } catch (ExecutionException e1) {
340 Throwable t = e1.getCause();
341 if (t instanceof IOException) {
342 LOG.error("IOException during splitting", e1);
343 throw (IOException)t;
344 }
345 LOG.error("Unexpected execution exception during splitting", e1);
346 throw new IllegalStateException(t);
347 } catch (InterruptedException e1) {
348 LOG.error("Unexpected interrupted exception during splitting", e1);
349 throw new IllegalStateException(e1);
350 }
351 }
352 return regionGroups;
353 }
354
355
356 String getUniqueName(byte[] tableName) {
357 String name = Bytes.toStringBinary(tableName) + "," + regionCount.incrementAndGet();
358 return name;
359 }
360
361 protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item,
362 final HTable table, byte[] startKey,
363 byte[] splitKey) throws IOException {
364 final Path hfilePath = item.hfilePath;
365
366
367
368 final Path tmpDir = new Path(item.hfilePath.getParent(), "_tmp");
369
370 LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
371 "region. Splitting...");
372
373 String uniqueName = getUniqueName(table.getTableName());
374 HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
375 Path botOut = new Path(tmpDir, uniqueName + ".bottom");
376 Path topOut = new Path(tmpDir, uniqueName + ".top");
377 splitStoreFile(getConf(), hfilePath, familyDesc, splitKey,
378 botOut, topOut);
379
380
381
382 List<LoadQueueItem> lqis = new ArrayList<LoadQueueItem>(2);
383 lqis.add(new LoadQueueItem(item.family, botOut));
384 lqis.add(new LoadQueueItem(item.family, topOut));
385
386 LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
387 return lqis;
388 }
389
390
391
392
393
394
395
396
397
398 protected List<LoadQueueItem> groupOrSplit(Multimap<ByteBuffer, LoadQueueItem> regionGroups,
399 final LoadQueueItem item, final HTable table,
400 final Pair<byte[][], byte[][]> startEndKeys)
401 throws IOException {
402 final Path hfilePath = item.hfilePath;
403 final FileSystem fs = hfilePath.getFileSystem(getConf());
404 HFile.Reader hfr = HFile.createReader(fs, hfilePath,
405 new CacheConfig(getConf()));
406 final byte[] first, last;
407 try {
408 hfr.loadFileInfo();
409 first = hfr.getFirstRowKey();
410 last = hfr.getLastRowKey();
411 } finally {
412 hfr.close();
413 }
414
415 LOG.info("Trying to load hfile=" + hfilePath +
416 " first=" + Bytes.toStringBinary(first) +
417 " last=" + Bytes.toStringBinary(last));
418 if (first == null || last == null) {
419 assert first == null && last == null;
420
421 LOG.info("hfile " + hfilePath + " has no entries, skipping");
422 return null;
423 }
424 if (Bytes.compareTo(first, last) > 0) {
425 throw new IllegalArgumentException(
426 "Invalid range: " + Bytes.toStringBinary(first) +
427 " > " + Bytes.toStringBinary(last));
428 }
429 int idx = Arrays.binarySearch(startEndKeys.getFirst(), first,
430 Bytes.BYTES_COMPARATOR);
431 if (idx < 0) {
432
433
434 idx = -(idx + 1) - 1;
435 }
436 final int indexForCallable = idx;
437 boolean lastKeyInRange =
438 Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 ||
439 Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
440 if (!lastKeyInRange) {
441 List<LoadQueueItem> lqis = splitStoreFile(item, table,
442 startEndKeys.getFirst()[indexForCallable],
443 startEndKeys.getSecond()[indexForCallable]);
444 return lqis;
445 }
446
447
448 regionGroups.put(ByteBuffer.wrap(startEndKeys.getFirst()[idx]), item);
449 return null;
450 }
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465 protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
466 byte[] tableName, final byte[] first, Collection<LoadQueueItem> lqis) throws IOException {
467
468 final List<Pair<byte[], String>> famPaths =
469 new ArrayList<Pair<byte[], String>>(lqis.size());
470 for (LoadQueueItem lqi : lqis) {
471 famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
472 }
473
474 final ServerCallable<Boolean> svrCallable = new ServerCallable<Boolean>(conn,
475 tableName, first) {
476 @Override
477 public Boolean call() throws Exception {
478 LOG.debug("Going to connect to server " + location + " for row "
479 + Bytes.toStringBinary(row));
480 byte[] regionName = location.getRegionInfo().getRegionName();
481 return server.bulkLoadHFiles(famPaths, regionName);
482 }
483 };
484
485 try {
486 List<LoadQueueItem> toRetry = new ArrayList<LoadQueueItem>();
487 boolean success = conn.getRegionServerWithRetries(svrCallable);
488 if (!success) {
489 LOG.warn("Attempt to bulk load region containing "
490 + Bytes.toStringBinary(first) + " into table "
491 + Bytes.toStringBinary(tableName) + " with files " + lqis
492 + " failed. This is recoverable and they will be retried.");
493 toRetry.addAll(lqis);
494 }
495
496 return toRetry;
497 } catch (IOException e) {
498 LOG.error("Encountered unrecoverable error from region server", e);
499 throw e;
500 }
501 }
502
503
504
505
506
507 static void splitStoreFile(
508 Configuration conf, Path inFile,
509 HColumnDescriptor familyDesc, byte[] splitKey,
510 Path bottomOut, Path topOut) throws IOException
511 {
512
513 Reference topReference = new Reference(splitKey, Range.top);
514 Reference bottomReference = new Reference(splitKey, Range.bottom);
515
516 copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
517 copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
518 }
519
520
521
522
523 private static void copyHFileHalf(
524 Configuration conf, Path inFile, Path outFile, Reference reference,
525 HColumnDescriptor familyDescriptor)
526 throws IOException {
527 FileSystem fs = inFile.getFileSystem(conf);
528 CacheConfig cacheConf = new CacheConfig(conf);
529 HalfStoreFileReader halfReader = null;
530 StoreFile.Writer halfWriter = null;
531 try {
532 halfReader = new HalfStoreFileReader(fs, inFile, cacheConf,
533 reference);
534 Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
535
536 int blocksize = familyDescriptor.getBlocksize();
537 Algorithm compression = familyDescriptor.getCompression();
538 BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
539
540 halfWriter = new StoreFile.Writer(
541 fs, outFile, blocksize, compression, conf, cacheConf,
542 KeyValue.COMPARATOR, bloomFilterType, 0);
543 HFileScanner scanner = halfReader.getScanner(false, false);
544 scanner.seekTo();
545 do {
546 KeyValue kv = scanner.getKeyValue();
547 halfWriter.append(kv);
548 } while (scanner.next());
549
550 for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
551 if (shouldCopyHFileMetaKey(entry.getKey())) {
552 halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
553 }
554 }
555 } finally {
556 if (halfWriter != null) halfWriter.close();
557 if (halfReader != null) halfReader.close(cacheConf.shouldEvictOnClose());
558 }
559 }
560
561 private static boolean shouldCopyHFileMetaKey(byte[] key) {
562 return !HFile.isReservedFileInfoKey(key);
563 }
564
565 private boolean doesTableExist(String tableName) throws Exception {
566 return hbAdmin.tableExists(tableName);
567 }
568
569
570
571
572
573
574
575
576
577
578
579
580
581 public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
582 ArrayList<byte[]> keysArray = new ArrayList<byte[]>();
583 int runningValue = 0;
584 byte[] currStartKey = null;
585 boolean firstBoundary = true;
586
587 for (Map.Entry<byte[], Integer> item: bdryMap.entrySet()) {
588 if (runningValue == 0) currStartKey = item.getKey();
589 runningValue += item.getValue();
590 if (runningValue == 0) {
591 if (!firstBoundary) keysArray.add(currStartKey);
592 firstBoundary = false;
593 }
594 }
595
596 return keysArray.toArray(new byte[0][0]);
597 }
598
599
600
601
602
603 private void createTable(String tableName, String dirPath) throws Exception {
604 Path hfofDir = new Path(dirPath);
605 FileSystem fs = hfofDir.getFileSystem(getConf());
606
607 if (!fs.exists(hfofDir)) {
608 throw new FileNotFoundException("HFileOutputFormat dir " +
609 hfofDir + " not found");
610 }
611
612 FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
613 if (familyDirStatuses == null) {
614 throw new FileNotFoundException("No families found in " + hfofDir);
615 }
616
617 HTableDescriptor htd = new HTableDescriptor(tableName);
618 HColumnDescriptor hcd = null;
619
620
621
622 byte[][] keys = null;
623 TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
624
625 for (FileStatus stat : familyDirStatuses) {
626 if (!stat.isDir()) {
627 LOG.warn("Skipping non-directory " + stat.getPath());
628 continue;
629 }
630 Path familyDir = stat.getPath();
631
632 if (familyDir.getName().startsWith("_")) continue;
633 byte[] family = familyDir.getName().getBytes();
634
635 hcd = new HColumnDescriptor(family);
636 htd.addFamily(hcd);
637
638 Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
639 for (Path hfile : hfiles) {
640 if (hfile.getName().startsWith("_")) continue;
641
642 HFile.Reader reader = HFile.createReader(fs, hfile,
643 new CacheConfig(getConf()));
644 final byte[] first, last;
645 try {
646 reader.loadFileInfo();
647 first = reader.getFirstRowKey();
648 last = reader.getLastRowKey();
649
650 LOG.info("Trying to figure out region boundaries hfile=" + hfile +
651 " first=" + Bytes.toStringBinary(first) +
652 " last=" + Bytes.toStringBinary(last));
653
654
655 Integer value = map.containsKey(first)?(Integer)map.get(first):0;
656 map.put(first, value+1);
657
658 value = map.containsKey(last)?(Integer)map.get(last):0;
659 map.put(last, value-1);
660 } finally {
661 reader.close();
662 }
663 }
664 }
665
666 keys = LoadIncrementalHFiles.inferBoundaries(map);
667 try {
668 this.hbAdmin.createTableAsync(htd, keys);
669 } catch (java.net.SocketTimeoutException e) {
670 System.err.println("Caught Socket timeout.. Mostly caused by a slow region assignment by master!");
671 }
672
673 HTable table = new HTable(this.cfg, tableName);
674
675 HConnection conn = table.getConnection();
676 int ctr = 0;
677 while (!conn.isTableAvailable(table.getTableName()) && (ctr<TABLE_CREATE_MAX_RETRIES)) {
678 LOG.info("Table " + tableName + "not yet available... Sleeping for 60 more seconds...");
679
680 Thread.sleep(TABLE_CREATE_SLEEP);
681 ctr++;
682 }
683 LOG.info("Table "+ tableName +" is finally available!!");
684 }
685
686 @Override
687 public int run(String[] args) throws Exception {
688 if (args.length != 2) {
689 usage();
690 return -1;
691 }
692
693 String dirPath = args[0];
694 String tableName = args[1];
695
696 boolean tableExists = this.doesTableExist(tableName);
697 if (!tableExists) this.createTable(tableName,dirPath);
698
699 Path hfofDir = new Path(dirPath);
700 HTable table = new HTable(this.cfg, tableName);
701
702 doBulkLoad(hfofDir, table);
703 return 0;
704 }
705
706 public static void main(String[] args) throws Exception {
707 int ret = ToolRunner.run(new LoadIncrementalHFiles(HBaseConfiguration.create()), args);
708 System.exit(ret);
709 }
710
711 }