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