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