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