1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.mapreduce.hadoopbackport;
20
21 import java.io.IOException;
22 import java.util.ArrayList;
23 import java.util.Arrays;
24 import java.util.List;
25 import java.util.Random;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.conf.Configured;
32 import org.apache.hadoop.fs.FileSystem;
33 import org.apache.hadoop.fs.Path;
34 import org.apache.hadoop.io.NullWritable;
35 import org.apache.hadoop.io.RawComparator;
36 import org.apache.hadoop.io.SequenceFile;
37 import org.apache.hadoop.io.WritableComparable;
38 import org.apache.hadoop.mapreduce.InputFormat;
39 import org.apache.hadoop.mapreduce.InputSplit;
40 import org.apache.hadoop.mapreduce.Job;
41 import org.apache.hadoop.mapreduce.RecordReader;
42 import org.apache.hadoop.mapreduce.TaskAttemptContext;
43 import org.apache.hadoop.mapreduce.TaskAttemptID;
44 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
45 import org.apache.hadoop.util.ReflectionUtils;
46 import org.apache.hadoop.util.Tool;
47 import org.apache.hadoop.util.ToolRunner;
48
49
50
51
52
53
54
55
56
57 public class InputSampler<K,V> extends Configured implements Tool {
58
59 private static final Log LOG = LogFactory.getLog(InputSampler.class);
60
61 static int printUsage() {
62 System.out.println("sampler -r <reduces>\n" +
63 " [-inFormat <input format class>]\n" +
64 " [-keyClass <map input & output key class>]\n" +
65 " [-splitRandom <double pcnt> <numSamples> <maxsplits> | " +
66 "// Sample from random splits at random (general)\n" +
67 " -splitSample <numSamples> <maxsplits> | " +
68 " // Sample from first records in splits (random data)\n"+
69 " -splitInterval <double pcnt> <maxsplits>]" +
70 " // Sample from splits at intervals (sorted data)");
71 System.out.println("Default sampler: -splitRandom 0.1 10000 10");
72 ToolRunner.printGenericCommandUsage(System.out);
73 return -1;
74 }
75
76 public InputSampler(Configuration conf) {
77 setConf(conf);
78 }
79
80
81
82
83
84 public interface Sampler<K,V> {
85
86
87
88
89 K[] getSample(InputFormat<K,V> inf, Job job)
90 throws IOException, InterruptedException;
91 }
92
93
94
95
96
97 public static class SplitSampler<K,V> implements Sampler<K,V> {
98
99 private final int numSamples;
100 private final int maxSplitsSampled;
101
102
103
104
105
106
107
108 public SplitSampler(int numSamples) {
109 this(numSamples, Integer.MAX_VALUE);
110 }
111
112
113
114
115
116
117
118 public SplitSampler(int numSamples, int maxSplitsSampled) {
119 this.numSamples = numSamples;
120 this.maxSplitsSampled = maxSplitsSampled;
121 }
122
123
124
125
126 @SuppressWarnings("unchecked")
127 public K[] getSample(InputFormat<K,V> inf, Job job)
128 throws IOException, InterruptedException {
129 List<InputSplit> splits = inf.getSplits(job);
130 ArrayList<K> samples = new ArrayList<K>(numSamples);
131 int splitsToSample = Math.min(maxSplitsSampled, splits.size());
132 int splitStep = splits.size() / splitsToSample;
133 int samplesPerSplit = numSamples / splitsToSample;
134 long records = 0;
135 for (int i = 0; i < splitsToSample; ++i) {
136 RecordReader<K,V> reader = inf.createRecordReader(
137 splits.get(i * splitStep),
138 new TaskAttemptContext(job.getConfiguration(),
139 new TaskAttemptID()));
140 while (reader.nextKeyValue()) {
141 samples.add(reader.getCurrentKey());
142 ++records;
143 if ((i+1) * samplesPerSplit <= records) {
144 break;
145 }
146 }
147 reader.close();
148 }
149 return (K[])samples.toArray();
150 }
151 }
152
153
154
155
156
157
158 public static class RandomSampler<K,V> implements Sampler<K,V> {
159 private double freq;
160 private final int numSamples;
161 private final int maxSplitsSampled;
162
163
164
165
166
167
168
169
170 public RandomSampler(double freq, int numSamples) {
171 this(freq, numSamples, Integer.MAX_VALUE);
172 }
173
174
175
176
177
178
179
180
181 public RandomSampler(double freq, int numSamples, int maxSplitsSampled) {
182 this.freq = freq;
183 this.numSamples = numSamples;
184 this.maxSplitsSampled = maxSplitsSampled;
185 }
186
187
188
189
190
191
192
193 @SuppressWarnings("unchecked")
194 public K[] getSample(InputFormat<K,V> inf, Job job)
195 throws IOException, InterruptedException {
196 List<InputSplit> splits = inf.getSplits(job);
197 ArrayList<K> samples = new ArrayList<K>(numSamples);
198 int splitsToSample = Math.min(maxSplitsSampled, splits.size());
199
200 Random r = new Random();
201 long seed = r.nextLong();
202 r.setSeed(seed);
203 LOG.debug("seed: " + seed);
204
205 for (int i = 0; i < splits.size(); ++i) {
206 InputSplit tmp = splits.get(i);
207 int j = r.nextInt(splits.size());
208 splits.set(i, splits.get(j));
209 splits.set(j, tmp);
210 }
211
212
213
214 for (int i = 0; i < splitsToSample ||
215 (i < splits.size() && samples.size() < numSamples); ++i) {
216 RecordReader<K,V> reader = inf.createRecordReader(splits.get(i),
217 new TaskAttemptContext(job.getConfiguration(),
218 new TaskAttemptID()));
219 while (reader.nextKeyValue()) {
220 if (r.nextDouble() <= freq) {
221 if (samples.size() < numSamples) {
222 samples.add(reader.getCurrentKey());
223 } else {
224
225
226
227
228 int ind = r.nextInt(numSamples);
229 if (ind != numSamples) {
230 samples.set(ind, reader.getCurrentKey());
231 }
232 freq *= (numSamples - 1) / (double) numSamples;
233 }
234 }
235 }
236 reader.close();
237 }
238 return (K[])samples.toArray();
239 }
240 }
241
242
243
244
245
246 public static class IntervalSampler<K,V> implements Sampler<K,V> {
247 private final double freq;
248 private final int maxSplitsSampled;
249
250
251
252
253
254 public IntervalSampler(double freq) {
255 this(freq, Integer.MAX_VALUE);
256 }
257
258
259
260
261
262
263
264 public IntervalSampler(double freq, int maxSplitsSampled) {
265 this.freq = freq;
266 this.maxSplitsSampled = maxSplitsSampled;
267 }
268
269
270
271
272
273
274 @SuppressWarnings("unchecked")
275 public K[] getSample(InputFormat<K,V> inf, Job job)
276 throws IOException, InterruptedException {
277 List<InputSplit> splits = inf.getSplits(job);
278 ArrayList<K> samples = new ArrayList<K>();
279 int splitsToSample = Math.min(maxSplitsSampled, splits.size());
280 int splitStep = splits.size() / splitsToSample;
281 long records = 0;
282 long kept = 0;
283 for (int i = 0; i < splitsToSample; ++i) {
284 RecordReader<K,V> reader = inf.createRecordReader(
285 splits.get(i * splitStep),
286 new TaskAttemptContext(job.getConfiguration(),
287 new TaskAttemptID()));
288 while (reader.nextKeyValue()) {
289 ++records;
290 if ((double) kept / records < freq) {
291 ++kept;
292 samples.add(reader.getCurrentKey());
293 }
294 }
295 reader.close();
296 }
297 return (K[])samples.toArray();
298 }
299 }
300
301
302
303
304
305
306
307 @SuppressWarnings("unchecked")
308 public static <K,V> void writePartitionFile(Job job, Sampler<K,V> sampler)
309 throws IOException, ClassNotFoundException, InterruptedException {
310 Configuration conf = job.getConfiguration();
311 final InputFormat inf =
312 ReflectionUtils.newInstance(job.getInputFormatClass(), conf);
313 int numPartitions = job.getNumReduceTasks();
314 K[] samples = sampler.getSample(inf, job);
315 LOG.info("Using " + samples.length + " samples");
316 RawComparator<K> comparator =
317 (RawComparator<K>) job.getSortComparator();
318 Arrays.sort(samples, comparator);
319 Path dst = new Path(TotalOrderPartitioner.getPartitionFile(conf));
320 FileSystem fs = dst.getFileSystem(conf);
321 if (fs.exists(dst)) {
322 fs.delete(dst, false);
323 }
324 SequenceFile.Writer writer = SequenceFile.createWriter(fs,
325 conf, dst, job.getMapOutputKeyClass(), NullWritable.class);
326 NullWritable nullValue = NullWritable.get();
327 float stepSize = samples.length / (float) numPartitions;
328 int last = -1;
329 for(int i = 1; i < numPartitions; ++i) {
330 int k = Math.round(stepSize * i);
331 while (last >= k && comparator.compare(samples[last], samples[k]) == 0) {
332 ++k;
333 }
334 writer.append(samples[k], nullValue);
335 last = k;
336 }
337 writer.close();
338 }
339
340
341
342
343
344 public int run(String[] args) throws Exception {
345 Job job = new Job(getConf());
346 ArrayList<String> otherArgs = new ArrayList<String>();
347 Sampler<K,V> sampler = null;
348 for(int i=0; i < args.length; ++i) {
349 try {
350 if ("-r".equals(args[i])) {
351 job.setNumReduceTasks(Integer.parseInt(args[++i]));
352 } else if ("-inFormat".equals(args[i])) {
353 job.setInputFormatClass(
354 Class.forName(args[++i]).asSubclass(InputFormat.class));
355 } else if ("-keyClass".equals(args[i])) {
356 job.setMapOutputKeyClass(
357 Class.forName(args[++i]).asSubclass(WritableComparable.class));
358 } else if ("-splitSample".equals(args[i])) {
359 int numSamples = Integer.parseInt(args[++i]);
360 int maxSplits = Integer.parseInt(args[++i]);
361 if (0 >= maxSplits) maxSplits = Integer.MAX_VALUE;
362 sampler = new SplitSampler<K,V>(numSamples, maxSplits);
363 } else if ("-splitRandom".equals(args[i])) {
364 double pcnt = Double.parseDouble(args[++i]);
365 int numSamples = Integer.parseInt(args[++i]);
366 int maxSplits = Integer.parseInt(args[++i]);
367 if (0 >= maxSplits) maxSplits = Integer.MAX_VALUE;
368 sampler = new RandomSampler<K,V>(pcnt, numSamples, maxSplits);
369 } else if ("-splitInterval".equals(args[i])) {
370 double pcnt = Double.parseDouble(args[++i]);
371 int maxSplits = Integer.parseInt(args[++i]);
372 if (0 >= maxSplits) maxSplits = Integer.MAX_VALUE;
373 sampler = new IntervalSampler<K,V>(pcnt, maxSplits);
374 } else {
375 otherArgs.add(args[i]);
376 }
377 } catch (NumberFormatException except) {
378 System.out.println("ERROR: Integer expected instead of " + args[i]);
379 return printUsage();
380 } catch (ArrayIndexOutOfBoundsException except) {
381 System.out.println("ERROR: Required parameter missing from " +
382 args[i-1]);
383 return printUsage();
384 }
385 }
386 if (job.getNumReduceTasks() <= 1) {
387 System.err.println("Sampler requires more than one reducer");
388 return printUsage();
389 }
390 if (otherArgs.size() < 2) {
391 System.out.println("ERROR: Wrong number of parameters: ");
392 return printUsage();
393 }
394 if (null == sampler) {
395 sampler = new RandomSampler<K,V>(0.1, 10000, 10);
396 }
397
398 Path outf = new Path(otherArgs.remove(otherArgs.size() - 1));
399 TotalOrderPartitioner.setPartitionFile(getConf(), outf);
400 for (String s : otherArgs) {
401 FileInputFormat.addInputPath(job, new Path(s));
402 }
403 InputSampler.<K,V>writePartitionFile(job, sampler);
404
405 return 0;
406 }
407
408 public static void main(String[] args) throws Exception {
409 InputSampler<?,?> sampler = new InputSampler(new Configuration());
410 int res = ToolRunner.run(sampler, args);
411 System.exit(res);
412 }
413 }