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