1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.mapreduce;
19
20 import java.io.IOException;
21 import java.util.ArrayList;
22 import java.util.List;
23
24 import org.apache.hadoop.hbase.classification.InterfaceAudience;
25 import org.apache.hadoop.hbase.classification.InterfaceStability;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.Cell;
28 import org.apache.hadoop.hbase.KeyValue;
29 import org.apache.hadoop.hbase.Tag;
30 import org.apache.hadoop.hbase.TagType;
31 import org.apache.hadoop.hbase.client.Put;
32 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33 import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
34 import org.apache.hadoop.hbase.security.visibility.CellVisibility;
35 import org.apache.hadoop.hbase.security.visibility.InvalidLabelException;
36 import org.apache.hadoop.hbase.util.Base64;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.apache.hadoop.io.LongWritable;
39 import org.apache.hadoop.io.Text;
40 import org.apache.hadoop.mapreduce.Counter;
41 import org.apache.hadoop.mapreduce.Mapper;
42
43
44
45
46 @InterfaceAudience.Public
47 @InterfaceStability.Stable
48 public class TsvImporterMapper
49 extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
50 {
51
52
53 protected long ts;
54
55
56 private String separator;
57
58
59 private boolean skipBadLines;
60 private Counter badLineCount;
61
62 protected ImportTsv.TsvParser parser;
63
64 protected Configuration conf;
65
66 @InterfaceStability.Unstable
67 protected String cellVisibilityExpr;
68 @InterfaceStability.Unstable
69 protected long ttl;
70
71 protected CellCreator kvCreator;
72
73 private String hfileOutPath;
74
75 public long getTs() {
76 return ts;
77 }
78
79 public boolean getSkipBadLines() {
80 return skipBadLines;
81 }
82
83 public Counter getBadLineCount() {
84 return badLineCount;
85 }
86
87 public void incrementBadLineCount(int count) {
88 this.badLineCount.increment(count);
89 }
90
91
92
93
94
95
96
97
98
99 @Override
100 protected void setup(Context context) {
101 doSetup(context);
102
103 conf = context.getConfiguration();
104 parser = new ImportTsv.TsvParser(conf.get(ImportTsv.COLUMNS_CONF_KEY),
105 separator);
106 if (parser.getRowKeyColumnIndex() == -1) {
107 throw new RuntimeException("No row key column specified");
108 }
109 this.kvCreator = new CellCreator(conf);
110 }
111
112
113
114
115
116 protected void doSetup(Context context) {
117 Configuration conf = context.getConfiguration();
118
119
120
121 separator = conf.get(ImportTsv.SEPARATOR_CONF_KEY);
122 if (separator == null) {
123 separator = ImportTsv.DEFAULT_SEPARATOR;
124 } else {
125 separator = new String(Base64.decode(separator));
126 }
127
128
129 ts = conf.getLong(ImportTsv.TIMESTAMP_CONF_KEY, 0);
130
131 skipBadLines = context.getConfiguration().getBoolean(
132 ImportTsv.SKIP_LINES_CONF_KEY, true);
133 badLineCount = context.getCounter("ImportTsv", "Bad Lines");
134 hfileOutPath = conf.get(ImportTsv.BULK_OUTPUT_CONF_KEY);
135 }
136
137
138
139
140 @Override
141 public void map(LongWritable offset, Text value,
142 Context context)
143 throws IOException {
144 byte[] lineBytes = value.getBytes();
145
146 try {
147 ImportTsv.TsvParser.ParsedLine parsed = parser.parse(
148 lineBytes, value.getLength());
149 ImmutableBytesWritable rowKey =
150 new ImmutableBytesWritable(lineBytes,
151 parsed.getRowKeyOffset(),
152 parsed.getRowKeyLength());
153
154 ts = parsed.getTimestamp(ts);
155 cellVisibilityExpr = parsed.getCellVisibility();
156 ttl = parsed.getCellTTL();
157
158 Put put = new Put(rowKey.copyBytes());
159 for (int i = 0; i < parsed.getColumnCount(); i++) {
160 if (i == parser.getRowKeyColumnIndex() || i == parser.getTimestampKeyColumnIndex()
161 || i == parser.getAttributesKeyColumnIndex() || i == parser.getCellVisibilityColumnIndex()
162 || i == parser.getCellTTLColumnIndex()) {
163 continue;
164 }
165 populatePut(lineBytes, parsed, put, i);
166 }
167 context.write(rowKey, put);
168 } catch (InvalidLabelException badLine) {
169 if (skipBadLines) {
170 System.err.println(
171 "Bad line at offset: " + offset.get() + ":\n" +
172 badLine.getMessage());
173 incrementBadLineCount(1);
174 return;
175 } else {
176 throw new IOException(badLine);
177 }
178 } catch (ImportTsv.TsvParser.BadTsvLineException badLine) {
179 if (skipBadLines) {
180 System.err.println(
181 "Bad line at offset: " + offset.get() + ":\n" +
182 badLine.getMessage());
183 incrementBadLineCount(1);
184 return;
185 } else {
186 throw new IOException(badLine);
187 }
188 } catch (IllegalArgumentException e) {
189 if (skipBadLines) {
190 System.err.println(
191 "Bad line at offset: " + offset.get() + ":\n" +
192 e.getMessage());
193 incrementBadLineCount(1);
194 return;
195 } else {
196 throw new IOException(e);
197 }
198 } catch (InterruptedException e) {
199 e.printStackTrace();
200 }
201 }
202
203 protected void populatePut(byte[] lineBytes, ImportTsv.TsvParser.ParsedLine parsed, Put put,
204 int i) throws BadTsvLineException, IOException {
205 Cell cell = null;
206 if (hfileOutPath == null) {
207 cell = new KeyValue(lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
208 parser.getFamily(i), 0, parser.getFamily(i).length, parser.getQualifier(i), 0,
209 parser.getQualifier(i).length, ts, KeyValue.Type.Put, lineBytes,
210 parsed.getColumnOffset(i), parsed.getColumnLength(i));
211 if (cellVisibilityExpr != null) {
212
213
214 put.setCellVisibility(new CellVisibility(cellVisibilityExpr));
215 }
216 if (ttl > 0) {
217 put.setTTL(ttl);
218 }
219 } else {
220
221
222 List<Tag> tags = new ArrayList<Tag>();
223 if (cellVisibilityExpr != null) {
224 tags.addAll(kvCreator.getVisibilityExpressionResolver()
225 .createVisibilityExpTags(cellVisibilityExpr));
226 }
227
228
229 if (ttl > 0) {
230 tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
231 }
232 cell = this.kvCreator.create(lineBytes, parsed.getRowKeyOffset(), parsed.getRowKeyLength(),
233 parser.getFamily(i), 0, parser.getFamily(i).length, parser.getQualifier(i), 0,
234 parser.getQualifier(i).length, ts, lineBytes, parsed.getColumnOffset(i),
235 parsed.getColumnLength(i), tags);
236 }
237 put.add(cell);
238 }
239 }