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