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