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