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