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