View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.mapreduce;
21  
22  import java.io.FileNotFoundException;
23  import java.io.IOException;
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.HashSet;
30  import java.util.LinkedList;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.Map.Entry;
34  import java.util.Set;
35  import java.util.TreeMap;
36  import java.util.concurrent.Callable;
37  import java.util.concurrent.ExecutionException;
38  import java.util.concurrent.ExecutorService;
39  import java.util.concurrent.Future;
40  import java.util.concurrent.LinkedBlockingQueue;
41  import java.util.concurrent.ThreadPoolExecutor;
42  import java.util.concurrent.TimeUnit;
43  import java.util.concurrent.atomic.AtomicLong;
44  
45  import org.apache.commons.logging.Log;
46  import org.apache.commons.logging.LogFactory;
47  import org.apache.hadoop.conf.Configuration;
48  import org.apache.hadoop.conf.Configured;
49  import org.apache.hadoop.fs.FileStatus;
50  import org.apache.hadoop.fs.FileSystem;
51  import org.apache.hadoop.fs.FileUtil;
52  import org.apache.hadoop.fs.Path;
53  import org.apache.hadoop.hbase.HBaseConfiguration;
54  import org.apache.hadoop.hbase.HColumnDescriptor;
55  import org.apache.hadoop.hbase.HConstants;
56  import org.apache.hadoop.hbase.HTableDescriptor;
57  import org.apache.hadoop.hbase.KeyValue;
58  import org.apache.hadoop.hbase.TableNotFoundException;
59  import org.apache.hadoop.hbase.client.HBaseAdmin;
60  import org.apache.hadoop.hbase.client.HConnection;
61  import org.apache.hadoop.hbase.client.HTable;
62  import org.apache.hadoop.hbase.client.ServerCallable;
63  import org.apache.hadoop.hbase.coprocessor.SecureBulkLoadClient;
64  import org.apache.hadoop.hbase.io.HalfStoreFileReader;
65  import org.apache.hadoop.hbase.io.Reference;
66  import org.apache.hadoop.hbase.io.Reference.Range;
67  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
68  import org.apache.hadoop.hbase.io.hfile.CacheConfig;
69  import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
70  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoder;
71  import org.apache.hadoop.hbase.io.hfile.HFileDataBlockEncoderImpl;
72  import org.apache.hadoop.hbase.io.hfile.HFile;
73  import org.apache.hadoop.hbase.io.hfile.HFileScanner;
74  import org.apache.hadoop.hbase.regionserver.Store;
75  import org.apache.hadoop.hbase.regionserver.StoreFile;
76  import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
77  import org.apache.hadoop.hbase.security.User;
78  import org.apache.hadoop.hbase.util.Bytes;
79  import org.apache.hadoop.hbase.util.Pair;
80  import org.apache.hadoop.security.token.Token;
81  import org.apache.hadoop.util.Tool;
82  import org.apache.hadoop.util.ToolRunner;
83  
84  import com.google.common.collect.HashMultimap;
85  import com.google.common.collect.Multimap;
86  import com.google.common.collect.Multimaps;
87  import com.google.common.util.concurrent.ThreadFactoryBuilder;
88  
89  /**
90   * Tool to load the output of HFileOutputFormat into an existing table.
91   * @see #usage()
92   */
93  public class LoadIncrementalHFiles extends Configured implements Tool {
94  
95    private static Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
96    private static final int  TABLE_CREATE_MAX_RETRIES = 20;
97    private static final long TABLE_CREATE_SLEEP = 60000;
98    static AtomicLong regionCount = new AtomicLong(0);
99    private HBaseAdmin hbAdmin;
100   private Configuration cfg;
101 
102   public static String NAME = "completebulkload";
103 
104   private boolean useSecure;
105   private Token<?> userToken;
106   private String bulkToken;
107 
108   //package private for testing
109   LoadIncrementalHFiles(Configuration conf, Boolean useSecure) throws Exception {
110     super(conf);
111     this.cfg = conf;
112     this.hbAdmin = new HBaseAdmin(conf);
113     //added simple for testing
114     this.useSecure = useSecure != null ? useSecure : User.isHBaseSecurityEnabled(conf);
115   }
116 
117   public LoadIncrementalHFiles(Configuration conf) throws Exception {
118     this(conf, null);
119   }
120 
121   private void usage() {
122     System.err.println("usage: " + NAME +
123         " /path/to/hfileoutputformat-output " +
124         "tablename");
125   }
126 
127   /**
128    * Represents an HFile waiting to be loaded. An queue is used
129    * in this class in order to support the case where a region has
130    * split during the process of the load. When this happens,
131    * the HFile is split into two physical parts across the new
132    * region boundary, and each part is added back into the queue.
133    * The import process finishes when the queue is empty.
134    */
135   static class LoadQueueItem {
136     final byte[] family;
137     final Path hfilePath;
138 
139     public LoadQueueItem(byte[] family, Path hfilePath) {
140       this.family = family;
141       this.hfilePath = hfilePath;
142     }
143 
144     public String toString() {
145       return "family:"+ Bytes.toString(family) + " path:" + hfilePath.toString();
146     }
147   }
148 
149   /**
150    * Walk the given directory for all HFiles, and return a Queue
151    * containing all such files.
152    */
153   private void discoverLoadQueue(Deque<LoadQueueItem> ret, Path hfofDir)
154   throws IOException {
155     FileSystem fs = hfofDir.getFileSystem(getConf());
156 
157     if (!fs.exists(hfofDir)) {
158       throw new FileNotFoundException("HFileOutputFormat dir " +
159           hfofDir + " not found");
160     }
161 
162     FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
163     if (familyDirStatuses == null) {
164       throw new FileNotFoundException("No families found in " + hfofDir);
165     }
166 
167     for (FileStatus stat : familyDirStatuses) {
168       if (!stat.isDir()) {
169         LOG.warn("Skipping non-directory " + stat.getPath());
170         continue;
171       }
172       Path familyDir = stat.getPath();
173       // Skip _logs, etc
174       if (familyDir.getName().startsWith("_")) continue;
175       byte[] family = familyDir.getName().getBytes();
176       Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
177       for (Path hfile : hfiles) {
178         if (hfile.getName().startsWith("_")) continue;
179         ret.add(new LoadQueueItem(family, hfile));
180       }
181     }
182   }
183 
184   /**
185    * Perform a bulk load of the given directory into the given
186    * pre-existing table.  This method is not threadsafe.
187    * 
188    * @param hfofDir the directory that was provided as the output path
189    * of a job using HFileOutputFormat
190    * @param table the table to load into
191    * @throws TableNotFoundException if table does not yet exist
192    */
193   public void doBulkLoad(Path hfofDir, final HTable table)
194     throws TableNotFoundException, IOException
195   {
196     final HConnection conn = table.getConnection();
197 
198     if (!conn.isTableAvailable(table.getTableName())) {
199       throw new TableNotFoundException("Table " +
200           Bytes.toStringBinary(table.getTableName()) +
201           "is not currently available.");
202     }
203 
204     // initialize thread pools
205     int nrThreads = cfg.getInt("hbase.loadincremental.threads.max",
206         Runtime.getRuntime().availableProcessors());
207     ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
208     builder.setNameFormat("LoadIncrementalHFiles-%1$d");
209     ExecutorService pool = new ThreadPoolExecutor(nrThreads, nrThreads,
210         60, TimeUnit.SECONDS,
211         new LinkedBlockingQueue<Runnable>(),
212         builder.build());
213     ((ThreadPoolExecutor)pool).allowCoreThreadTimeOut(true);
214 
215     // LQI queue does not need to be threadsafe -- all operations on this queue
216     // happen in this thread
217     Deque<LoadQueueItem> queue = new LinkedList<LoadQueueItem>();
218     try {
219       discoverLoadQueue(queue, hfofDir);
220       int count = 0;
221 
222       if (queue.isEmpty()) {
223         LOG.warn("Bulk load operation did not find any files to load in " +
224             "directory " + hfofDir.toUri() + ".  Does it contain files in " +
225             "subdirectories that correspond to column family names?");
226         return;
227       }
228 
229       //If using secure bulk load
230       //prepare staging directory and token
231       if(useSecure) {
232         //This condition is here for unit testing
233         //Since delegation token doesn't work in mini cluster
234         if(User.isSecurityEnabled()) {
235           FileSystem fs = FileSystem.get(cfg);
236           userToken = fs.getDelegationToken("renewer");
237         }
238         bulkToken = new SecureBulkLoadClient(table).prepareBulkLoad(table.getTableName());
239       }
240 
241       // Assumes that region splits can happen while this occurs.
242       while (!queue.isEmpty()) {
243         // need to reload split keys each iteration.
244         final Pair<byte[][], byte[][]> startEndKeys = table.getStartEndKeys();
245         if (count != 0) {
246           LOG.info("Split occured while grouping HFiles, retry attempt " +
247               + count + " with " + queue.size() + " files remaining to group or split");
248         }
249 
250         int maxRetries = cfg.getInt("hbase.bulkload.retries.number", 0);
251         if (maxRetries != 0 && count >= maxRetries) {
252           LOG.error("Retry attempted " + count +  " times without completing, bailing out");
253           return;
254         }
255         count++;
256 
257         // Using ByteBuffer for byte[] equality semantics
258         Multimap<ByteBuffer, LoadQueueItem> regionGroups = groupOrSplitPhase(table,
259             pool, queue, startEndKeys);
260 
261         bulkLoadPhase(table, conn, pool, queue, regionGroups);
262 
263         // NOTE: The next iteration's split / group could happen in parallel to
264         // atomic bulkloads assuming that there are splits and no merges, and
265         // that we can atomically pull out the groups we want to retry.
266       }
267 
268     } finally {
269       if(useSecure) {
270         if(userToken != null) {
271           try {
272             userToken.cancel(cfg);
273           } catch (Exception e) {
274             LOG.warn("Failed to cancel HDFS delegation token.", e);
275           }
276         }
277         if(bulkToken != null) {
278           new SecureBulkLoadClient(table).cleanupBulkLoad(bulkToken);
279         }
280       }
281       pool.shutdown();
282       if (queue != null && !queue.isEmpty()) {
283         StringBuilder err = new StringBuilder();
284         err.append("-------------------------------------------------\n");
285         err.append("Bulk load aborted with some files not yet loaded:\n");
286         err.append("-------------------------------------------------\n");
287         for (LoadQueueItem q : queue) {
288           err.append("  ").append(q.hfilePath).append('\n');
289         }
290         LOG.error(err);
291       }
292     }
293     
294     if (queue != null && !queue.isEmpty()) {
295         throw new RuntimeException("Bulk load aborted with some files not yet loaded."
296           + "Please check log for more details.");
297     }
298   }
299 
300   /**
301    * This takes the LQI's grouped by likely regions and attempts to bulk load
302    * them.  Any failures are re-queued for another pass with the
303    * groupOrSplitPhase.
304    */
305   protected void bulkLoadPhase(final HTable table, final HConnection conn,
306       ExecutorService pool, Deque<LoadQueueItem> queue,
307       final Multimap<ByteBuffer, LoadQueueItem> regionGroups) throws IOException {
308     // atomically bulk load the groups.
309     Set<Future<List<LoadQueueItem>>> loadingFutures = new HashSet<Future<List<LoadQueueItem>>>();
310     for (Entry<ByteBuffer, ? extends Collection<LoadQueueItem>> e: regionGroups.asMap().entrySet()) {
311       final byte[] first = e.getKey().array();
312       final Collection<LoadQueueItem> lqis =  e.getValue();
313 
314       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
315         public List<LoadQueueItem> call() throws Exception {
316           List<LoadQueueItem> toRetry = tryAtomicRegionLoad(conn, table.getTableName(), first, lqis);
317           return toRetry;
318         }
319       };
320       loadingFutures.add(pool.submit(call));
321     }
322 
323     // get all the results.
324     for (Future<List<LoadQueueItem>> future : loadingFutures) {
325       try {
326         List<LoadQueueItem> toRetry = future.get();
327 
328         // LQIs that are requeued to be regrouped.
329         queue.addAll(toRetry);
330 
331       } catch (ExecutionException e1) {
332         Throwable t = e1.getCause();
333         if (t instanceof IOException) {
334           // At this point something unrecoverable has happened.
335           // TODO Implement bulk load recovery
336           throw new IOException("BulkLoad encountered an unrecoverable problem", t);
337         }
338         LOG.error("Unexpected execution exception during bulk load", e1);
339         throw new IllegalStateException(t);
340       } catch (InterruptedException e1) {
341         LOG.error("Unexpected interrupted exception during bulk load", e1);
342         throw new IllegalStateException(e1);
343       }
344     }
345   }
346 
347   /**
348    * @return A Multimap<startkey, LoadQueueItem> that groups LQI by likely
349    * bulk load region targets.
350    */
351   private Multimap<ByteBuffer, LoadQueueItem> groupOrSplitPhase(final HTable table,
352       ExecutorService pool, Deque<LoadQueueItem> queue,
353       final Pair<byte[][], byte[][]> startEndKeys) throws IOException {
354     // <region start key, LQI> need synchronized only within this scope of this
355     // phase because of the puts that happen in futures.
356     Multimap<ByteBuffer, LoadQueueItem> rgs = HashMultimap.create();
357     final Multimap<ByteBuffer, LoadQueueItem> regionGroups = Multimaps.synchronizedMultimap(rgs);
358 
359     // drain LQIs and figure out bulk load groups
360     Set<Future<List<LoadQueueItem>>> splittingFutures = new HashSet<Future<List<LoadQueueItem>>>();
361     while (!queue.isEmpty()) {
362       final LoadQueueItem item = queue.remove();
363       
364       final Callable<List<LoadQueueItem>> call = new Callable<List<LoadQueueItem>>() {
365         public List<LoadQueueItem> call() throws Exception {
366           List<LoadQueueItem> splits = groupOrSplit(regionGroups, item, table, startEndKeys);
367           return splits;
368         }
369       };
370       splittingFutures.add(pool.submit(call));
371     }
372     // get all the results.  All grouping and splitting must finish before
373     // we can attempt the atomic loads.
374     for (Future<List<LoadQueueItem>> lqis : splittingFutures) {
375       try {
376         List<LoadQueueItem> splits = lqis.get();
377         if (splits != null) {
378           queue.addAll(splits);
379         }
380       } catch (ExecutionException e1) {
381         Throwable t = e1.getCause();
382         if (t instanceof IOException) {
383           LOG.error("IOException during splitting", e1);
384           throw (IOException)t; // would have been thrown if not parallelized,
385         }
386         LOG.error("Unexpected execution exception during splitting", e1);
387         throw new IllegalStateException(t);
388       } catch (InterruptedException e1) {
389         LOG.error("Unexpected interrupted exception during splitting", e1);
390         throw new IllegalStateException(e1);
391       }
392     }
393     return regionGroups;
394   }
395 
396   // unique file name for the table
397   String getUniqueName(byte[] tableName) {
398     String name = Bytes.toStringBinary(tableName) + "," + regionCount.incrementAndGet();
399     return name;
400   }
401 
402   protected List<LoadQueueItem> splitStoreFile(final LoadQueueItem item,
403       final HTable table, byte[] startKey,
404       byte[] splitKey) throws IOException {
405     final Path hfilePath = item.hfilePath;
406 
407     // We use a '_' prefix which is ignored when walking directory trees
408     // above.
409     final Path tmpDir = new Path(item.hfilePath.getParent(), "_tmp");
410 
411     LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
412     "region. Splitting...");
413 
414     String uniqueName = getUniqueName(table.getTableName());
415     HColumnDescriptor familyDesc = table.getTableDescriptor().getFamily(item.family);
416     Path botOut = new Path(tmpDir, uniqueName + ".bottom");
417     Path topOut = new Path(tmpDir, uniqueName + ".top");
418     splitStoreFile(getConf(), hfilePath, familyDesc, splitKey,
419         botOut, topOut);
420 
421     // Add these back at the *front* of the queue, so there's a lower
422     // chance that the region will just split again before we get there.
423     List<LoadQueueItem> lqis = new ArrayList<LoadQueueItem>(2);
424     lqis.add(new LoadQueueItem(item.family, botOut));
425     lqis.add(new LoadQueueItem(item.family, topOut));
426 
427     LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
428     return lqis;
429   }
430 
431   /**
432    * Attempt to assign the given load queue item into its target region group.
433    * If the hfile boundary no longer fits into a region, physically splits
434    * the hfile such that the new bottom half will fit and returns the list of
435    * LQI's corresponding to the resultant hfiles.
436    *
437    * protected for testing
438    */
439   protected List<LoadQueueItem> groupOrSplit(Multimap<ByteBuffer, LoadQueueItem> regionGroups,
440       final LoadQueueItem item, final HTable table,
441       final Pair<byte[][], byte[][]> startEndKeys)
442       throws IOException {
443     final Path hfilePath = item.hfilePath;
444     final FileSystem fs = hfilePath.getFileSystem(getConf());
445     HFile.Reader hfr = HFile.createReader(fs, hfilePath,
446         new CacheConfig(getConf()));
447     final byte[] first, last;
448     try {
449       hfr.loadFileInfo();
450       first = hfr.getFirstRowKey();
451       last = hfr.getLastRowKey();
452     }  finally {
453       hfr.close();
454     }
455 
456     LOG.info("Trying to load hfile=" + hfilePath +
457         " first=" + Bytes.toStringBinary(first) +
458         " last="  + Bytes.toStringBinary(last));
459     if (first == null || last == null) {
460       assert first == null && last == null;
461       // TODO what if this is due to a bad HFile?
462       LOG.info("hfile " + hfilePath + " has no entries, skipping");
463       return null;
464     }
465     if (Bytes.compareTo(first, last) > 0) {
466       throw new IllegalArgumentException(
467       "Invalid range: " + Bytes.toStringBinary(first) +
468       " > " + Bytes.toStringBinary(last));
469     }
470     int idx = Arrays.binarySearch(startEndKeys.getFirst(), first,
471         Bytes.BYTES_COMPARATOR);
472     if (idx < 0) {
473       // not on boundary, returns -(insertion index).  Calculate region it
474       // would be in.
475       idx = -(idx + 1) - 1;
476     }
477     final int indexForCallable = idx;
478     boolean lastKeyInRange =
479       Bytes.compareTo(last, startEndKeys.getSecond()[idx]) < 0 ||
480       Bytes.equals(startEndKeys.getSecond()[idx], HConstants.EMPTY_BYTE_ARRAY);
481     if (!lastKeyInRange) {
482       List<LoadQueueItem> lqis = splitStoreFile(item, table,
483           startEndKeys.getFirst()[indexForCallable],
484           startEndKeys.getSecond()[indexForCallable]);
485       return lqis;
486     }
487 
488     // group regions.
489     regionGroups.put(ByteBuffer.wrap(startEndKeys.getFirst()[idx]), item);
490     return null;
491   }
492 
493   /**
494    * Attempts to do an atomic load of many hfiles into a region.  If it fails,
495    * it returns a list of hfiles that need to be retried.  If it is successful
496    * it will return an empty list.
497    * 
498    * NOTE: To maintain row atomicity guarantees, region server callable should
499    * succeed atomically and fails atomically.
500    * 
501    * Protected for testing.
502    * 
503    * @return empty list if success, list of items to retry on recoverable
504    * failure
505    */
506   protected List<LoadQueueItem> tryAtomicRegionLoad(final HConnection conn,
507       byte[] tableName, final byte[] first, Collection<LoadQueueItem> lqis) throws IOException {
508 
509     final List<Pair<byte[], String>> famPaths =
510       new ArrayList<Pair<byte[], String>>(lqis.size());
511     for (LoadQueueItem lqi : lqis) {
512       famPaths.add(Pair.newPair(lqi.family, lqi.hfilePath.toString()));
513     }
514 
515     final ServerCallable<Boolean> svrCallable = new ServerCallable<Boolean>(conn,
516         tableName, first) {
517       @Override
518       public Boolean call() throws Exception {
519         SecureBulkLoadClient secureClient = null;
520         boolean success = false;
521 
522         try {
523           LOG.debug("Going to connect to server " + location + " for row "
524               + Bytes.toStringBinary(row));
525           byte[] regionName = location.getRegionInfo().getRegionName();
526           if(!useSecure) {
527             success = server.bulkLoadHFiles(famPaths, regionName);
528           } else {
529             HTable table = new HTable(conn.getConfiguration(), tableName);
530             secureClient = new SecureBulkLoadClient(table, location.getRegionInfo().getStartKey());
531             success = secureClient.bulkLoadHFiles(famPaths, userToken, bulkToken);
532           }
533           return success;
534         } finally {
535           //Best effort copying of files that might not have been imported
536           //from the staging directory back to original location
537           //in user directory
538           if(secureClient != null && !success) {
539             FileSystem fs = FileSystem.get(cfg);
540             for(Pair<byte[], String> el : famPaths) {
541               Path hfileStagingPath = null;
542               Path hfileOrigPath = new Path(el.getSecond());
543               try {
544                 hfileStagingPath= new Path(secureClient.getStagingPath(bulkToken, el.getFirst()),
545                     hfileOrigPath.getName());
546                 if(fs.rename(hfileStagingPath, hfileOrigPath)) {
547                   LOG.debug("Moved back file " + hfileOrigPath + " from " +
548                       hfileStagingPath);
549                 } else if(fs.exists(hfileStagingPath)){
550                   LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
551                       hfileStagingPath);
552                 }
553               } catch(Exception ex) {
554                 LOG.debug("Unable to move back file " + hfileOrigPath + " from " +
555                     hfileStagingPath, ex);
556               }
557             }
558           }
559         }
560       }
561 
562     };
563 
564     try {
565       List<LoadQueueItem> toRetry = new ArrayList<LoadQueueItem>();
566       boolean success = svrCallable.withRetries();
567       if (!success) {
568         LOG.warn("Attempt to bulk load region containing "
569             + Bytes.toStringBinary(first) + " into table "
570             + Bytes.toStringBinary(tableName)  + " with files " + lqis
571             + " failed.  This is recoverable and they will be retried.");
572         toRetry.addAll(lqis); // return lqi's to retry
573       }
574       // success
575       return toRetry;
576     } catch (IOException e) {
577       LOG.error("Encountered unrecoverable error from region server", e);
578       throw e;
579     }
580   }
581 
582   /**
583    * Split a storefile into a top and bottom half, maintaining
584    * the metadata, recreating bloom filters, etc.
585    */
586   static void splitStoreFile(
587       Configuration conf, Path inFile,
588       HColumnDescriptor familyDesc, byte[] splitKey,
589       Path bottomOut, Path topOut) throws IOException
590   {
591     // Open reader with no block cache, and not in-memory
592     Reference topReference = new Reference(splitKey, Range.top);
593     Reference bottomReference = new Reference(splitKey, Range.bottom);
594 
595     copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
596     copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
597   }
598 
599   /**
600    * Copy half of an HFile into a new HFile.
601    */
602   private static void copyHFileHalf(
603       Configuration conf, Path inFile, Path outFile, Reference reference,
604       HColumnDescriptor familyDescriptor)
605   throws IOException {
606     FileSystem fs = inFile.getFileSystem(conf);
607     CacheConfig cacheConf = new CacheConfig(conf);
608     HalfStoreFileReader halfReader = null;
609     StoreFile.Writer halfWriter = null;
610     HFileDataBlockEncoder dataBlockEncoder = new HFileDataBlockEncoderImpl(
611         familyDescriptor.getDataBlockEncodingOnDisk(),
612         familyDescriptor.getDataBlockEncoding());
613     try {
614       halfReader = new HalfStoreFileReader(fs, inFile, cacheConf,
615           reference, DataBlockEncoding.NONE);
616       Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
617 
618       int blocksize = familyDescriptor.getBlocksize();
619       Algorithm compression = familyDescriptor.getCompression();
620       BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
621 
622       halfWriter = new StoreFile.WriterBuilder(conf, cacheConf,
623           fs, blocksize)
624               .withFilePath(outFile)
625               .withCompression(compression)
626               .withDataBlockEncoder(dataBlockEncoder)
627               .withBloomType(bloomFilterType)
628               .withChecksumType(Store.getChecksumType(conf))
629               .withBytesPerChecksum(Store.getBytesPerChecksum(conf))
630               .build();
631       HFileScanner scanner = halfReader.getScanner(false, false, false);
632       scanner.seekTo();
633       do {
634         KeyValue kv = scanner.getKeyValue();
635         halfWriter.append(kv);
636       } while (scanner.next());
637 
638       for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
639         if (shouldCopyHFileMetaKey(entry.getKey())) {
640           halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
641         }
642       }
643     } finally {
644       if (halfWriter != null) halfWriter.close();
645       if (halfReader != null) halfReader.close(cacheConf.shouldEvictOnClose());
646     }
647   }
648 
649   private static boolean shouldCopyHFileMetaKey(byte[] key) {
650     return !HFile.isReservedFileInfoKey(key);
651   }
652 
653   private boolean doesTableExist(String tableName) throws Exception {
654     return hbAdmin.tableExists(tableName);
655   }
656   
657   /*
658    * Infers region boundaries for a new table.
659    * Parameter:
660    *   bdryMap is a map between keys to an integer belonging to {+1, -1}
661    *     If a key is a start key of a file, then it maps to +1
662    *     If a key is an end key of a file, then it maps to -1
663    * Algo:
664    * 1) Poll on the keys in order: 
665    *    a) Keep adding the mapped values to these keys (runningSum) 
666    *    b) Each time runningSum reaches 0, add the start Key from when the runningSum had started to a boundary list.
667    * 2) Return the boundary list. 
668    */
669   public static byte[][] inferBoundaries(TreeMap<byte[], Integer> bdryMap) {
670     ArrayList<byte[]> keysArray = new ArrayList<byte[]>();
671     int runningValue = 0;
672     byte[] currStartKey = null;
673     boolean firstBoundary = true;
674     
675     for (Map.Entry<byte[], Integer> item: bdryMap.entrySet()) {
676       if (runningValue == 0) currStartKey = item.getKey();
677       runningValue += item.getValue();
678       if (runningValue == 0) {
679         if (!firstBoundary) keysArray.add(currStartKey);
680         firstBoundary = false;
681       } 
682     }
683     
684     return keysArray.toArray(new byte[0][0]);
685   }
686  
687   /*
688    * If the table is created for the first time, then "completebulkload" reads the files twice.
689    * More modifications necessary if we want to avoid doing it.
690    */
691   private void createTable(String tableName, String dirPath) throws Exception {
692     Path hfofDir = new Path(dirPath);
693     FileSystem fs = hfofDir.getFileSystem(getConf());
694 
695     if (!fs.exists(hfofDir)) {
696       throw new FileNotFoundException("HFileOutputFormat dir " +
697           hfofDir + " not found");
698     }
699 
700     FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
701     if (familyDirStatuses == null) {
702       throw new FileNotFoundException("No families found in " + hfofDir);
703     }
704 
705     HTableDescriptor htd = new HTableDescriptor(tableName);
706     HColumnDescriptor hcd = null;
707 
708     // Add column families
709     // Build a set of keys
710     byte[][] keys = null;
711     TreeMap<byte[], Integer> map = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
712     
713     for (FileStatus stat : familyDirStatuses) {
714       if (!stat.isDir()) {
715         LOG.warn("Skipping non-directory " + stat.getPath());
716         continue;
717       }
718       Path familyDir = stat.getPath();
719       // Skip _logs, etc
720       if (familyDir.getName().startsWith("_")) continue;
721       byte[] family = familyDir.getName().getBytes();
722      
723       hcd = new HColumnDescriptor(family);
724       htd.addFamily(hcd);
725       
726       Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
727       for (Path hfile : hfiles) {
728         if (hfile.getName().startsWith("_")) continue;
729         HFile.Reader reader = HFile.createReader(fs, hfile,
730             new CacheConfig(getConf()));
731         final byte[] first, last;
732         try {
733           if (hcd.getCompressionType() != reader.getCompressionAlgorithm()) {
734             hcd.setCompressionType(reader.getCompressionAlgorithm());
735             LOG.info("Setting compression " + hcd.getCompressionType().name() +
736                      " for family " + hcd.toString());
737           }
738           reader.loadFileInfo();
739           first = reader.getFirstRowKey();
740           last =  reader.getLastRowKey();
741 
742           LOG.info("Trying to figure out region boundaries hfile=" + hfile +
743             " first=" + Bytes.toStringBinary(first) +
744             " last="  + Bytes.toStringBinary(last));
745           
746           // To eventually infer start key-end key boundaries
747           Integer value = map.containsKey(first)?(Integer)map.get(first):0;
748           map.put(first, value+1);
749 
750           value = map.containsKey(last)?(Integer)map.get(last):0;
751           map.put(last, value-1);
752         }  finally {
753           reader.close();
754         }
755       }
756     }
757     
758     keys = LoadIncrementalHFiles.inferBoundaries(map);
759     this.hbAdmin.createTable(htd,keys);
760 
761     LOG.info("Table "+ tableName +" is available!!");
762   }
763 
764   @Override
765   public int run(String[] args) throws Exception {
766     if (args.length != 2) {
767       usage();
768       return -1;
769     }
770 
771     String dirPath   = args[0];
772     String tableName = args[1];
773 
774     boolean tableExists   = this.doesTableExist(tableName);
775     if (!tableExists) this.createTable(tableName,dirPath);
776 
777     Path hfofDir = new Path(dirPath);
778     HTable table = new HTable(this.cfg, tableName);
779 
780     doBulkLoad(hfofDir, table);
781     return 0;
782   }
783 
784   public static void main(String[] args) throws Exception {
785     int ret = ToolRunner.run(new LoadIncrementalHFiles(HBaseConfiguration.create()), args);
786     System.exit(ret);
787   }
788 
789 }