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