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