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
295
296
297
298
299
300 protected void bulkLoadPhase(final HTable table, final HConnection conn,
301 ExecutorService pool, Deque<LoadQueueItem> queue,
302 final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
303
304 Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<Future<List<LoadQueueItem>>>();
305 for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
306 final byte[] first = e.getKey().array();
307 final Collection<LoadQueueItem> lqis = e.getValue();
308
309 final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
310 public List<LoadQueueItem> call() throws Exception {
311 List<LoadQueueItem> toRetry = tryAtomicRegionLoad(conn, table.getTableName(), first, lqis);
312 return toRetry;
313 }
314 };
315 loadingFutures.add(pool.submit(call));
316 }
317
318
319 for (Future<List<LoadQueueItem>> future : loadingFutures) {
320 try {
321 List<LoadQueueItem> toRetry = future.get();
322
323
324 queue.addAll(toRetry);
325
326 } catch (ExecutionException e1) {
327 Throwable t = e1.getCause();
328 if (t instanceof IOException) {
329
330
331 throw new IOException("BulkLoad encountered an unrecoverable problem", t);
332 }
333 LOG.error("Unexpected execution exception during bulk load", e1);
334 throw new IllegalStateException(t);
335 } catch (InterruptedException e1) {
336 LOG.error("Unexpected interrupted exception during bulk load", e1);
337 throw new IllegalStateException(e1);
338 }
339 }
340 }
341
342
343
344
345
346 private Multimap<ByteBuffer, LoadQueueItem> groupOrSplitPhase(final HTable table,
347 ExecutorService pool, Deque<LoadQueueItem> queue,
348 final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
349
350
351 Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
352 final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
353
354
355 Set<Future<List<LoadQueueItem>>> splittingFutures = new HashSet<Future<List<LoadQueueItem>>>();
356 while (!queue.isEmpty()) {
357 final LoadQueueItem item = queue.remove();
358
359 final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
360 public List<LoadQueueItem> call() throws Exception {
361 List<LoadQueueItem> splits = groupOrSplit(regionGroups, item, table, startEndKeys);
362 return splits;
363 }
364 };
365 splittingFutures.add(pool.submit(call));
366 }
367
368
369 for (Future<List<LoadQueueItem>> lqis : splittingFutures) {
370 try {
371 List<LoadQueueItem> splits = lqis.get();
372 if (splits != null) {
373 queue.addAll(splits);
374 }
375 } catch (ExecutionException e1) {
376 Throwable t = e1.getCause();
377 if (t instanceof IOException) {
378 LOG.error("IOException during splitting", e1);
379 throw (IOException)t;
380 }
381 LOG.error("Unexpected execution exception during splitting", e1);
382 throw new IllegalStateException(t);
383 } catch (InterruptedException e1) {
384 LOG.error("Unexpected interrupted exception during splitting", e1);
385 throw new IllegalStateException(e1);
386 }
387 }
388 return regionGroups;
389 }
390
391
392 String getUniqueName(byte[] tableName) {
393 String name = Bytes.toStringBinary(tableName) + "," + regionCount.incrementAndGet();
394 return name;
395 }
396
397 protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item,
398 final HTable table, byte[] startKey,
399 byte[] splitKey) throws IOException {
400 final Path hfilePath = item.hfilePath;
401
402
403
404 final Path tmpDir = new Path(item.hfilePath.getParent(), "_tmp");
405
406 LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
407 "region. Splitting...");
408
409 String uniqueName = getUniqueName(table.getTableName());
410 HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
411 Path botOut = new Path(tmpDir, uniqueName + ".bottom");
412 Path topOut = new Path(tmpDir, uniqueName + ".top");
413 splitStoreFile(getConf(), hfilePath, familyDesc, splitKey,
414 botOut, topOut);
415
416
417
418 List<LoadQueueItem> lqis = new ArrayList<LoadQueueItem>(2);
419 lqis.add(new LoadQueueItem(item.family, botOut));
420 lqis.add(new LoadQueueItem(item.family, topOut));
421
422 LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
423 return lqis;
424 }
425
426
427
428
429
430
431
432
433
434 protected List<LoadQueueItem> groupOrSplit(Multimap<ByteBuffer, LoadQueueItem> regionGroups,
435 final LoadQueueItem item, final HTable table,
436 final Pair<byte[][], byte[][]> startEndKeys)
437 throws IOException {
438 final Path hfilePath = item.hfilePath;
439 final FileSystem fs = hfilePath.getFileSystem(getConf());
440 HFile.Reader hfr = HFile.createReader(fs, hfilePath,
441 new CacheConfig(getConf()));
442 final byte[] first, last;
443 try {
444 hfr.loadFileInfo();
445 first = hfr.getFirstRowKey();
446 last = hfr.getLastRowKey();
447 } finally {
448 hfr.close();
449 }
450
451 LOG.info("Trying to load hfile=" + hfilePath +
452 " first=" + Bytes.toStringBinary(first) +
453 " last=" + Bytes.toStringBinary(last));
454 if (first == null || last == null) {
455 assert first == null && last == null;
456
457 LOG.info("hfile " + hfilePath + " has no entries, skipping");
458 return null;
459 }
460 if (Bytes.compareTo(first, last) > 0) {
461 throw new IllegalArgumentException(
462 "Invalid range: " + Bytes.toStringBinary(first) +
463 " > " + Bytes.toStringBinary(last));
464 }
465 int idx = Arrays.binarySearch(startEndKeys.getFirst(), first,
466 Bytes.BYTES_COMPARATOR);
467 if (idx < 0) {
468
469
470 idx = -(idx + 1) - 1;
471 }
472 final int indexForCallable = idx;
473 boolean lastKeyInRange =
474 Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 ||
475 Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
476 if (!lastKeyInRange) {
477 List<LoadQueueItem> lqis = splitStoreFile(item, table,
478 startEndKeys.getFirst()[indexForCallable],
479 startEndKeys.getSecond()[indexForCallable]);
480 return lqis;
481 }
482
483
484 regionGroups.put(ByteBuffer.wrap(startEndKeys.getFirst()[idx]), item);
485 return null;
486 }
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501 protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
502 byte[] tableName, final byte[] first, Collection<LoadQueueItem> lqis) throws IOException {
503
504 final List<Pair<byte[], String>> famPaths =
505 new ArrayList<Pair<byte[], String>>(lqis.size());
506 for (LoadQueueItem lqi : lqis) {
507 famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
508 }
509
510 final ServerCallable<Boolean> svrCallable = new ServerCallable<Boolean>(conn,
511 tableName, first) {
512 @Override
513 public Boolean call() throws Exception {
514 SecureBulkLoadClient secureClient = null;
515 boolean success = false;
516
517 try {
518 LOG.debug("Going to connect to server " + location + " for row "
519 + Bytes.toStringBinary(row));
520 byte[] regionName = location.getRegionInfo().getRegionName();
521 if(!useSecure) {
522 success = server.bulkLoadHFiles(famPaths, regionName);
523 } else {
524 HTable table = new HTable(conn.getConfiguration(), tableName);
525 secureClient = new SecureBulkLoadClient(table, location.getRegionInfo().getStartKey());
526 success = secureClient.bulkLoadHFiles(famPaths, userToken, bulkToken);
527 }
528 return success;
529 } finally {
530
531
532
533 if(secureClient != null && !success) {
534 FileSystem fs = FileSystem.get(cfg);
535 for(Pair<byte[], String> el : famPaths) {
536 Path hfileStagingPath = null;
537 Path hfileOrigPath = new Path(el.getSecond());
538 try {
539 hfileStagingPath= new Path(secureClient.getStagingPath(bulkToken, el.getFirst()),
540 hfileOrigPath.getName());
541 if(fs.rename(hfileStagingPath, hfileOrigPath)) {
542 LOG.debug("Moved back file " + hfileOrigPath + " from " +
543 hfileStagingPath);
544 } else if(fs.exists(hfileStagingPath)){
545 LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
546 hfileStagingPath);
547 }
548 } catch(Exception ex) {
549 LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
550 hfileStagingPath, ex);
551 }
552 }
553 }
554 }
555 }
556
557 };
558
559 try {
560 List<LoadQueueItem> toRetry = new ArrayList<LoadQueueItem>();
561 boolean success = svrCallable.withRetries();
562 if (!success) {
563 LOG.warn("Attempt to bulk load region containing "
564 + Bytes.toStringBinary(first) + " into table "
565 + Bytes.toStringBinary(tableName) + " with files " + lqis
566 + " failed. This is recoverable and they will be retried.");
567 toRetry.addAll(lqis);
568 }
569
570 return toRetry;
571 } catch (IOException e) {
572 LOG.error("Encountered unrecoverable error from region server", e);
573 throw e;
574 }
575 }
576
577
578
579
580
581 static void splitStoreFile(
582 Configuration conf, Path inFile,
583 HColumnDescriptor familyDesc, byte[] splitKey,
584 Path bottomOut, Path topOut) throws IOException
585 {
586
587 Reference topReference = new Reference(splitKey, Range.top);
588 Reference bottomReference = new Reference(splitKey, Range.bottom);
589
590 copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
591 copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
592 }
593
594
595
596
597 private static void copyHFileHalf(
598 Configuration conf, Path inFile, Path outFile, Reference reference,
599 HColumnDescriptor familyDescriptor)
600 throws IOException {
601 FileSystem fs = inFile.getFileSystem(conf);
602 CacheConfig cacheConf = new CacheConfig(conf);
603 HalfStoreFileReader halfReader = null;
604 StoreFile.Writer halfWriter = null;
605 HFileDataBlockEncoder dataBlockEncoder = new HFileDataBlockEncoderImpl(
606 familyDescriptor.getDataBlockEncodingOnDisk(),
607 familyDescriptor.getDataBlockEncoding());
608 try {
609 halfReader = new HalfStoreFileReader(fs, inFile, cacheConf,
610 reference, DataBlockEncoding.NONE);
611 Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
612
613 int blocksize = familyDescriptor.getBlocksize();
614 Algorithm compression = familyDescriptor.getCompression();
615 BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
616
617 halfWriter = new StoreFile.WriterBuilder(conf, cacheConf,
618 fs, blocksize)
619 .withFilePath(outFile)
620 .withCompression(compression)
621 .withDataBlockEncoder(dataBlockEncoder)
622 .withBloomType(bloomFilterType)
623 .withChecksumType(Store.getChecksumType(conf))
624 .withBytesPerChecksum(Store.getBytesPerChecksum(conf))
625 .build();
626 HFileScanner scanner = halfReader.getScanner(false, false, false);
627 scanner.seekTo();
628 do {
629 KeyValue kv = scanner.getKeyValue();
630 halfWriter.append(kv);
631 } while (scanner.next());
632
633 for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
634 if (shouldCopyHFileMetaKey(entry.getKey())) {
635 halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
636 }
637 }
638 } finally {
639 if (halfWriter != null) halfWriter.close();
640 if (halfReader != null) halfReader.close(cacheConf.shouldEvictOnClose());
641 }
642 }
643
644 private static boolean shouldCopyHFileMetaKey(byte[] key) {
645 return !HFile.isReservedFileInfoKey(key);
646 }
647
648 private boolean doesTableExist(String tableName) throws Exception {
649 return hbAdmin.tableExists(tableName);
650 }
651
652
653
654
655
656
657
658
659
660
661
662
663
664 public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
665 ArrayList<byte[]> keysArray = new ArrayList<byte[]>();
666 int runningValue = 0;
667 byte[] currStartKey = null;
668 boolean firstBoundary = true;
669
670 for (Map.Entry<byte[], Integer> item: bdryMap.entrySet()) {
671 if (runningValue == 0) currStartKey = item.getKey();
672 runningValue += item.getValue();
673 if (runningValue == 0) {
674 if (!firstBoundary) keysArray.add(currStartKey);
675 firstBoundary = false;
676 }
677 }
678
679 return keysArray.toArray(new byte[0][0]);
680 }
681
682
683
684
685
686 private void createTable(String tableName, String dirPath) throws Exception {
687 Path hfofDir = new Path(dirPath);
688 FileSystem fs = hfofDir.getFileSystem(getConf());
689
690 if (!fs.exists(hfofDir)) {
691 throw new FileNotFoundException("HFileOutputFormat dir " +
692 hfofDir + " not found");
693 }
694
695 FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
696 if (familyDirStatuses == null) {
697 throw new FileNotFoundException("No families found in " + hfofDir);
698 }
699
700 HTableDescriptor htd = new HTableDescriptor(tableName);
701 HColumnDescriptor hcd = null;
702
703
704
705 byte[][] keys = null;
706 TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
707
708 for (FileStatus stat : familyDirStatuses) {
709 if (!stat.isDir()) {
710 LOG.warn("Skipping non-directory " + stat.getPath());
711 continue;
712 }
713 Path familyDir = stat.getPath();
714
715 if (familyDir.getName().startsWith("_")) continue;
716 byte[] family = familyDir.getName().getBytes();
717
718 hcd = new HColumnDescriptor(family);
719 htd.addFamily(hcd);
720
721 Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
722 for (Path hfile : hfiles) {
723 if (hfile.getName().startsWith("_")) continue;
724 HFile.Reader reader = HFile.createReader(fs, hfile,
725 new CacheConfig(getConf()));
726 final byte[] first, last;
727 try {
728 if (hcd.getCompressionType() != reader.getCompressionAlgorithm()) {
729 hcd.setCompressionType(reader.getCompressionAlgorithm());
730 LOG.info("Setting compression " + hcd.getCompressionType().name() +
731 " for family " + hcd.toString());
732 }
733 reader.loadFileInfo();
734 first = reader.getFirstRowKey();
735 last = reader.getLastRowKey();
736
737 LOG.info("Trying to figure out region boundaries hfile=" + hfile +
738 " first=" + Bytes.toStringBinary(first) +
739 " last=" + Bytes.toStringBinary(last));
740
741
742 Integer value = map.containsKey(first)?(Integer)map.get(first):0;
743 map.put(first, value+1);
744
745 value = map.containsKey(last)?(Integer)map.get(last):0;
746 map.put(last, value-1);
747 } finally {
748 reader.close();
749 }
750 }
751 }
752
753 keys = LoadIncrementalHFiles.inferBoundaries(map);
754 this.hbAdmin.createTable(htd,keys);
755
756 LOG.info("Table "+ tableName +" is available!!");
757 }
758
759 @Override
760 public int run(String[] args) throws Exception {
761 if (args.length != 2) {
762 usage();
763 return -1;
764 }
765
766 String dirPath = args[0];
767 String tableName = args[1];
768
769 boolean tableExists = this.doesTableExist(tableName);
770 if (!tableExists) this.createTable(tableName,dirPath);
771
772 Path hfofDir = new Path(dirPath);
773 HTable table = new HTable(this.cfg, tableName);
774
775 doBulkLoad(hfofDir, table);
776 return 0;
777 }
778
779 public static void main(String[] args) throws Exception {
780 int ret = ToolRunner.run(new LoadIncrementalHFiles(HBaseConfiguration.create()), args);
781 System.exit(ret);
782 }
783
784 }