1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.mapreduce;
21
22 import java.io.FileNotFoundException;
23 import java.io.IOException;
24 import java.util.Deque;
25 import java.util.LinkedList;
26 import java.util.Map;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.conf.Configured;
32 import org.apache.hadoop.fs.FileStatus;
33 import org.apache.hadoop.fs.FileSystem;
34 import org.apache.hadoop.fs.FileUtil;
35 import org.apache.hadoop.fs.Path;
36 import org.apache.hadoop.hbase.HColumnDescriptor;
37 import org.apache.hadoop.hbase.HRegionInfo;
38 import org.apache.hadoop.hbase.KeyValue;
39 import org.apache.hadoop.hbase.TableNotFoundException;
40 import org.apache.hadoop.hbase.client.HConnection;
41 import org.apache.hadoop.hbase.client.HTable;
42 import org.apache.hadoop.hbase.client.ServerCallable;
43 import org.apache.hadoop.hbase.io.HalfStoreFileReader;
44 import org.apache.hadoop.hbase.io.Reference;
45 import org.apache.hadoop.hbase.io.Reference.Range;
46 import org.apache.hadoop.hbase.io.hfile.HFile;
47 import org.apache.hadoop.hbase.io.hfile.HFileScanner;
48 import org.apache.hadoop.hbase.io.hfile.Compression.Algorithm;
49 import org.apache.hadoop.hbase.regionserver.StoreFile;
50 import org.apache.hadoop.hbase.regionserver.StoreFile.BloomType;
51 import org.apache.hadoop.hbase.util.Bytes;
52 import org.apache.hadoop.util.Tool;
53 import org.apache.hadoop.util.ToolRunner;
54
55
56
57
58
59 public class LoadIncrementalHFiles extends Configured implements Tool {
60
61 static Log LOG = LogFactory.getLog(LoadIncrementalHFiles.class);
62
63 public static String NAME = "completebulkload";
64
65 public LoadIncrementalHFiles(Configuration conf) {
66 super(conf);
67 }
68
69 public LoadIncrementalHFiles() {
70 super();
71 }
72
73
74 private void usage() {
75 System.err.println("usage: " + NAME +
76 " /path/to/hfileoutputformat-output " +
77 "tablename");
78 }
79
80
81
82
83
84
85
86
87
88 private static class LoadQueueItem {
89 final byte[] family;
90 final Path hfilePath;
91
92 public LoadQueueItem(byte[] family, Path hfilePath) {
93 this.family = family;
94 this.hfilePath = hfilePath;
95 }
96 }
97
98
99
100
101
102 private Deque<LoadQueueItem> discoverLoadQueue(Path hfofDir)
103 throws IOException {
104 FileSystem fs = hfofDir.getFileSystem(getConf());
105
106 if (!fs.exists(hfofDir)) {
107 throw new FileNotFoundException("HFileOutputFormat dir " +
108 hfofDir + " not found");
109 }
110
111 FileStatus[] familyDirStatuses = fs.listStatus(hfofDir);
112 if (familyDirStatuses == null) {
113 throw new FileNotFoundException("No families found in " + hfofDir);
114 }
115
116 Deque<LoadQueueItem> ret = new LinkedList<LoadQueueItem>();
117 for (FileStatus stat : familyDirStatuses) {
118 if (!stat.isDir()) {
119 LOG.warn("Skipping non-directory " + stat.getPath());
120 continue;
121 }
122 Path familyDir = stat.getPath();
123
124 if (familyDir.getName().startsWith("_")) continue;
125 byte[] family = familyDir.getName().getBytes();
126 Path[] hfiles = FileUtil.stat2Paths(fs.listStatus(familyDir));
127 for (Path hfile : hfiles) {
128 if (hfile.getName().startsWith("_")) continue;
129 ret.add(new LoadQueueItem(family, hfile));
130 }
131 }
132 return ret;
133 }
134
135
136
137
138
139
140
141
142
143 public void doBulkLoad(Path hfofDir, HTable table)
144 throws TableNotFoundException, IOException
145 {
146 HConnection conn = table.getConnection();
147
148 if (!conn.isTableAvailable(table.getTableName())) {
149 throw new TableNotFoundException("Table " +
150 Bytes.toStringBinary(table.getTableName()) +
151 "is not currently available.");
152 }
153
154 Deque<LoadQueueItem> queue = null;
155 try {
156 queue = discoverLoadQueue(hfofDir);
157 while (!queue.isEmpty()) {
158 LoadQueueItem item = queue.remove();
159 tryLoad(item, conn, table.getTableName(), queue);
160 }
161 } finally {
162 if (queue != null && !queue.isEmpty()) {
163 StringBuilder err = new StringBuilder();
164 err.append("-------------------------------------------------\n");
165 err.append("Bulk load aborted with some files not yet loaded:\n");
166 err.append("-------------------------------------------------\n");
167 for (LoadQueueItem q : queue) {
168 err.append(" ").append(q.hfilePath).append('\n');
169 }
170 LOG.error(err);
171 }
172 }
173 }
174
175
176
177
178
179
180
181 private void tryLoad(final LoadQueueItem item,
182 HConnection conn, final byte[] table,
183 final Deque<LoadQueueItem> queue)
184 throws IOException {
185 final Path hfilePath = item.hfilePath;
186 final FileSystem fs = hfilePath.getFileSystem(getConf());
187 HFile.Reader hfr = new HFile.Reader(fs, hfilePath, null, false);
188 final byte[] first, last;
189 try {
190 hfr.loadFileInfo();
191 first = hfr.getFirstRowKey();
192 last = hfr.getLastRowKey();
193 } finally {
194 hfr.close();
195 }
196
197 LOG.info("Trying to load hfile=" + hfilePath +
198 " first=" + Bytes.toStringBinary(first) +
199 " last=" + Bytes.toStringBinary(last));
200 if (first == null || last == null) {
201 assert first == null && last == null;
202 LOG.info("hfile " + hfilePath + " has no entries, skipping");
203 return;
204 }
205
206
207
208 final Path tmpDir = new Path(item.hfilePath.getParent(), "_tmp");
209
210 conn.getRegionServerWithRetries(
211 new ServerCallable<Void>(conn, table, first) {
212 @Override
213 public Void call() throws Exception {
214 LOG.debug("Going to connect to server " + location +
215 "for row " + Bytes.toStringBinary(row));
216 HRegionInfo hri = location.getRegionInfo();
217 if (!hri.containsRange(first, last)) {
218 LOG.info("HFile at " + hfilePath + " no longer fits inside a single " +
219 "region. Splitting...");
220
221 HColumnDescriptor familyDesc = hri.getTableDesc().getFamily(item.family);
222 Path botOut = new Path(tmpDir, hri.getEncodedName() + ".bottom");
223 Path topOut = new Path(tmpDir, hri.getEncodedName() + ".top");
224 splitStoreFile(getConf(), hfilePath, familyDesc, hri.getEndKey(),
225 botOut, topOut);
226
227
228
229 queue.addFirst(new LoadQueueItem(item.family, botOut));
230 queue.addFirst(new LoadQueueItem(item.family, topOut));
231 LOG.info("Successfully split into new HFiles " + botOut + " and " + topOut);
232 return null;
233 }
234
235 byte[] regionName = location.getRegionInfo().getRegionName();
236 server.bulkLoadHFile(hfilePath.toString(), regionName, item.family);
237 return null;
238 }
239 });
240 }
241
242
243
244
245
246 static void splitStoreFile(
247 Configuration conf, Path inFile,
248 HColumnDescriptor familyDesc, byte[] splitKey,
249 Path bottomOut, Path topOut) throws IOException
250 {
251
252 Reference topReference = new Reference(splitKey, Range.top);
253 Reference bottomReference = new Reference(splitKey, Range.bottom);
254
255 copyHFileHalf(conf, inFile, topOut, topReference, familyDesc);
256 copyHFileHalf(conf, inFile, bottomOut, bottomReference, familyDesc);
257 }
258
259
260
261
262 private static void copyHFileHalf(
263 Configuration conf, Path inFile, Path outFile, Reference reference,
264 HColumnDescriptor familyDescriptor)
265 throws IOException {
266 FileSystem fs = inFile.getFileSystem(conf);
267 HalfStoreFileReader halfReader = null;
268 StoreFile.Writer halfWriter = null;
269 try {
270 halfReader = new HalfStoreFileReader(fs, inFile, null, reference);
271 Map<byte[], byte[]> fileInfo = halfReader.loadFileInfo();
272
273 int blocksize = familyDescriptor.getBlocksize();
274 Algorithm compression = familyDescriptor.getCompression();
275 BloomType bloomFilterType = familyDescriptor.getBloomFilterType();
276
277 halfWriter = new StoreFile.Writer(
278 fs, outFile, blocksize, compression, conf, KeyValue.COMPARATOR,
279 bloomFilterType, 0);
280 HFileScanner scanner = halfReader.getScanner(false, false);
281 scanner.seekTo();
282 do {
283 KeyValue kv = scanner.getKeyValue();
284 halfWriter.append(kv);
285 } while (scanner.next());
286
287 for (Map.Entry<byte[],byte[]> entry : fileInfo.entrySet()) {
288 if (shouldCopyHFileMetaKey(entry.getKey())) {
289 halfWriter.appendFileInfo(entry.getKey(), entry.getValue());
290 }
291 }
292 } finally {
293 if (halfWriter != null) halfWriter.close();
294 if (halfReader != null) halfReader.close();
295 }
296 }
297
298 private static boolean shouldCopyHFileMetaKey(byte[] key) {
299 return !HFile.isReservedFileInfoKey(key);
300 }
301
302
303 @Override
304 public int run(String[] args) throws Exception {
305 if (args.length != 2) {
306 usage();
307 return -1;
308 }
309
310 Path hfofDir = new Path(args[0]);
311 HTable table = new HTable(args[1]);
312
313 doBulkLoad(hfofDir, table);
314 return 0;
315 }
316
317 public static void main(String[] args) throws Exception {
318 ToolRunner.run(new LoadIncrementalHFiles(), args);
319 }
320
321 }