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.IOException;
23 import java.util.Map;
24 import java.util.TreeMap;
25
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.HBaseConfiguration;
29 import org.apache.hadoop.hbase.KeyValue;
30 import org.apache.hadoop.hbase.client.Put;
31 import org.apache.hadoop.hbase.client.Result;
32 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33 import org.apache.hadoop.hbase.util.Bytes;
34 import org.apache.hadoop.mapreduce.Job;
35 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
36 import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat;
37 import org.apache.hadoop.util.GenericOptionsParser;
38
39
40
41
42 public class Import {
43 final static String NAME = "import";
44 public static final String CF_RENAME_PROP = "HBASE_IMPORTER_RENAME_CFS";
45
46
47
48
49 static class Importer
50 extends TableMapper<ImmutableBytesWritable, Put> {
51 private Map<byte[], byte[]> cfRenameMap;
52
53
54
55
56
57
58
59
60
61 @Override
62 public void map(ImmutableBytesWritable row, Result value,
63 Context context)
64 throws IOException {
65 try {
66 context.write(row, resultToPut(row, value));
67 } catch (InterruptedException e) {
68 e.printStackTrace();
69 }
70 }
71
72 private Put resultToPut(ImmutableBytesWritable key, Result result)
73 throws IOException {
74 Put put = new Put(key.get());
75 for (KeyValue kv : result.raw()) {
76 if(cfRenameMap != null) {
77
78 byte[] newCfName = cfRenameMap.get(kv.getFamily());
79 if(newCfName != null) {
80 kv = new KeyValue(kv.getBuffer(),
81 kv.getRowOffset(),
82 kv.getRowLength(),
83 newCfName,
84 0,
85 newCfName.length,
86 kv.getBuffer(),
87 kv.getQualifierOffset(),
88 kv.getQualifierLength(),
89 kv.getTimestamp(),
90 KeyValue.Type.codeToType(kv.getType()),
91 kv.getBuffer(),
92 kv.getValueOffset(),
93 kv.getValueLength());
94 }
95 }
96 put.add(kv);
97 }
98 return put;
99 }
100
101 @Override
102 public void setup(Context context) {
103
104 cfRenameMap = null;
105 String allMappingsPropVal = context.getConfiguration().get(CF_RENAME_PROP);
106 if(allMappingsPropVal != null) {
107
108 String[] allMappings = allMappingsPropVal.split(",");
109 for (String mapping: allMappings) {
110 if(cfRenameMap == null) {
111 cfRenameMap = new TreeMap<byte[],byte[]>(Bytes.BYTES_COMPARATOR);
112 }
113 String [] srcAndDest = mapping.split(":");
114 if(srcAndDest.length != 2) {
115 continue;
116 }
117 cfRenameMap.put(srcAndDest[0].getBytes(), srcAndDest[1].getBytes());
118 }
119 }
120 }
121 }
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136 static public void configureCfRenaming(Configuration conf,
137 Map<String, String> renameMap) {
138 StringBuilder sb = new StringBuilder();
139 for(Map.Entry<String,String> entry: renameMap.entrySet()) {
140 String sourceCf = entry.getKey();
141 String destCf = entry.getValue();
142
143 if(sourceCf.contains(":") || sourceCf.contains(",") ||
144 destCf.contains(":") || destCf.contains(",")) {
145 throw new IllegalArgumentException("Illegal character in CF names: "
146 + sourceCf + ", " + destCf);
147 }
148
149 if(sb.length() != 0) {
150 sb.append(",");
151 }
152 sb.append(sourceCf + ":" + destCf);
153 }
154 conf.set(CF_RENAME_PROP, sb.toString());
155 }
156
157
158
159
160
161
162
163
164
165
166 public static Job createSubmittableJob(Configuration conf, String[] args)
167 throws IOException {
168 String tableName = args[0];
169 Path inputDir = new Path(args[1]);
170 Job job = new Job(conf, NAME + "_" + tableName);
171 job.setJarByClass(Importer.class);
172 FileInputFormat.setInputPaths(job, inputDir);
173 job.setInputFormatClass(SequenceFileInputFormat.class);
174 job.setMapperClass(Importer.class);
175
176
177 TableMapReduceUtil.initTableReducerJob(tableName, null, job);
178 job.setNumReduceTasks(0);
179 return job;
180 }
181
182
183
184
185 private static void usage(final String errorMsg) {
186 if (errorMsg != null && errorMsg.length() > 0) {
187 System.err.println("ERROR: " + errorMsg);
188 }
189 System.err.println("Usage: Import <tablename> <inputdir>");
190 }
191
192
193
194
195
196
197
198 public static void main(String[] args) throws Exception {
199 Configuration conf = HBaseConfiguration.create();
200 String[] otherArgs = new GenericOptionsParser(conf, args).getRemainingArgs();
201 if (otherArgs.length < 2) {
202 usage("Wrong number of arguments: " + otherArgs.length);
203 System.exit(-1);
204 }
205 Job job = createSubmittableJob(conf, otherArgs);
206 System.exit(job.waitForCompletion(true) ? 0 : 1);
207 }
208 }