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