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