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