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