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