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