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