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