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