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