source: proiecte/HadoopJUnit/hadoop-0.20.1/src/test/org/apache/hadoop/mapred/GenericMRLoadGenerator.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 29.3 KB
Line 
1/**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements.  See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership.  The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License.  You may obtain a copy of the License at
9 *
10 *     http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18
19package org.apache.hadoop.mapred;
20
21import java.io.DataInput;
22import java.io.DataOutput;
23import java.io.IOException;
24import java.util.ArrayList;
25import java.util.Date;
26import java.util.Iterator;
27import java.util.Random;
28import java.util.Stack;
29
30import org.apache.hadoop.conf.Configuration;
31import org.apache.hadoop.conf.Configured;
32import org.apache.hadoop.fs.FileStatus;
33import org.apache.hadoop.fs.FileSystem;
34import org.apache.hadoop.fs.Path;
35import org.apache.hadoop.io.LongWritable;
36import org.apache.hadoop.io.SequenceFile;
37import org.apache.hadoop.io.Text;
38import org.apache.hadoop.io.Writable;
39import org.apache.hadoop.io.WritableComparable;
40import org.apache.hadoop.io.WritableUtils;
41import org.apache.hadoop.mapred.lib.NullOutputFormat;
42import org.apache.hadoop.util.GenericOptionsParser;
43import org.apache.hadoop.util.ReflectionUtils;
44import org.apache.hadoop.util.Tool;
45import org.apache.hadoop.util.ToolRunner;
46
47public class GenericMRLoadGenerator extends Configured implements Tool {
48
49  protected static int printUsage() {
50    System.err.println(
51    "Usage: [-m <maps>] [-r <reduces>]\n" +
52    "       [-keepmap <percent>] [-keepred <percent>]\n" +
53    "       [-indir <path>] [-outdir <path]\n" +
54    "       [-inFormat[Indirect] <InputFormat>] [-outFormat <OutputFormat>]\n" +
55    "       [-outKey <WritableComparable>] [-outValue <Writable>]\n");
56    GenericOptionsParser.printGenericCommandUsage(System.err);
57    return -1;
58  }
59
60
61  /**
62   * Configure a job given argv.
63   */
64  public static boolean parseArgs(String[] argv, JobConf job) throws IOException {
65    if (argv.length < 1) {
66      return 0 == printUsage();
67    }
68    for(int i=0; i < argv.length; ++i) {
69      if (argv.length == i + 1) {
70        System.out.println("ERROR: Required parameter missing from " +
71            argv[i]);
72        return 0 == printUsage();
73      }
74      try {
75        if ("-m".equals(argv[i])) {
76          job.setNumMapTasks(Integer.parseInt(argv[++i]));
77        } else if ("-r".equals(argv[i])) {
78          job.setNumReduceTasks(Integer.parseInt(argv[++i]));
79        } else if ("-inFormat".equals(argv[i])) {
80          job.setInputFormat(
81              Class.forName(argv[++i]).asSubclass(InputFormat.class));
82        } else if ("-outFormat".equals(argv[i])) {
83          job.setOutputFormat(
84              Class.forName(argv[++i]).asSubclass(OutputFormat.class));
85        } else if ("-outKey".equals(argv[i])) {
86          job.setOutputKeyClass(
87            Class.forName(argv[++i]).asSubclass(WritableComparable.class));
88        } else if ("-outValue".equals(argv[i])) {
89          job.setOutputValueClass(
90            Class.forName(argv[++i]).asSubclass(Writable.class));
91        } else if ("-keepmap".equals(argv[i])) {
92          job.set("hadoop.sort.map.keep.percent", argv[++i]);
93        } else if ("-keepred".equals(argv[i])) {
94          job.set("hadoop.sort.reduce.keep.percent", argv[++i]);
95        } else if ("-outdir".equals(argv[i])) {
96          FileOutputFormat.setOutputPath(job, new Path(argv[++i]));
97        } else if ("-indir".equals(argv[i])) {
98          FileInputFormat.addInputPaths(job, argv[++i]);
99        } else if ("-inFormatIndirect".equals(argv[i])) {
100          job.setClass("mapred.indirect.input.format",
101              Class.forName(argv[++i]).asSubclass(InputFormat.class),
102              InputFormat.class);
103          job.setInputFormat(IndirectInputFormat.class);
104        } else {
105          System.out.println("Unexpected argument: " + argv[i]);
106          return 0 == printUsage();
107        }
108      } catch (NumberFormatException except) {
109        System.out.println("ERROR: Integer expected instead of " + argv[i]);
110        return 0 == printUsage();
111      } catch (Exception e) {
112        throw (IOException)new IOException().initCause(e);
113      }
114    }
115    return true;
116  }
117
118  public int run(String [] argv) throws Exception {
119    JobConf job = new JobConf(getConf());
120    job.setJarByClass(GenericMRLoadGenerator.class);
121    job.setMapperClass(SampleMapper.class);
122    job.setReducerClass(SampleReducer.class);
123    if (!parseArgs(argv, job)) {
124      return -1;
125    }
126
127    if (null == FileOutputFormat.getOutputPath(job)) {
128      // No output dir? No writes
129      job.setOutputFormat(NullOutputFormat.class);
130    }
131
132    if (0 == FileInputFormat.getInputPaths(job).length) {
133      // No input dir? Generate random data
134      System.err.println("No input path; ignoring InputFormat");
135      confRandom(job);
136    } else if (null != job.getClass("mapred.indirect.input.format", null)) {
137      // specified IndirectInputFormat? Build src list
138      JobClient jClient = new JobClient(job); 
139      Path sysdir = jClient.getSystemDir();
140      Random r = new Random();
141      Path indirInputFile = new Path(sysdir,
142          Integer.toString(r.nextInt(Integer.MAX_VALUE), 36) + "_files");
143      job.set("mapred.indirect.input.file", indirInputFile.toString());
144      SequenceFile.Writer writer = SequenceFile.createWriter(
145          sysdir.getFileSystem(job), job, indirInputFile,
146          LongWritable.class, Text.class,
147          SequenceFile.CompressionType.NONE);
148      try {
149        for (Path p : FileInputFormat.getInputPaths(job)) {
150          FileSystem fs = p.getFileSystem(job);
151          Stack<Path> pathstack = new Stack<Path>();
152          pathstack.push(p);
153          while (!pathstack.empty()) {
154            for (FileStatus stat : fs.listStatus(pathstack.pop())) {
155              if (stat.isDir()) {
156                if (!stat.getPath().getName().startsWith("_")) {
157                  pathstack.push(stat.getPath());
158                }
159              } else {
160                writer.sync();
161                writer.append(new LongWritable(stat.getLen()),
162                    new Text(stat.getPath().toUri().toString()));
163              }
164            }
165          }
166        }
167      } finally {
168        writer.close();
169      }
170    }
171
172    Date startTime = new Date();
173    System.out.println("Job started: " + startTime);
174    JobClient.runJob(job);
175    Date endTime = new Date();
176    System.out.println("Job ended: " + endTime);
177    System.out.println("The job took " +
178                       (endTime.getTime() - startTime.getTime()) /1000 +
179                       " seconds.");
180
181    return 0;
182  }
183
184  /**
185   * Main driver/hook into ToolRunner.
186   */
187  public static void main(String[] argv) throws Exception {
188    int res =
189      ToolRunner.run(new Configuration(), new GenericMRLoadGenerator(), argv);
190    System.exit(res);
191  }
192
193  static class RandomInputFormat implements InputFormat {
194
195    public InputSplit[] getSplits(JobConf conf, int numSplits) {
196      InputSplit[] splits = new InputSplit[numSplits];
197      for (int i = 0; i < numSplits; ++i) {
198        splits[i] = new IndirectInputFormat.IndirectSplit(
199            new Path("ignore" + i), 1);
200      }
201      return splits;
202    }
203
204    public RecordReader<Text,Text> getRecordReader(InputSplit split,
205        JobConf job, Reporter reporter) throws IOException {
206      final IndirectInputFormat.IndirectSplit clSplit =
207        (IndirectInputFormat.IndirectSplit)split;
208      return new RecordReader<Text,Text>() {
209        boolean once = true;
210        public boolean next(Text key, Text value) {
211          if (once) {
212            key.set(clSplit.getPath().toString());
213            once = false;
214            return true;
215          }
216          return false;
217        }
218        public Text createKey() { return new Text(); }
219        public Text createValue() { return new Text(); }
220        public long getPos() { return 0; }
221        public void close() { }
222        public float getProgress() { return 0.0f; }
223      };
224    }
225  }
226
227  static enum Counters { RECORDS_WRITTEN, BYTES_WRITTEN }
228
229  static class RandomMapOutput extends MapReduceBase
230      implements Mapper<Text,Text,Text,Text> {
231    StringBuilder sentence = new StringBuilder();
232    int keymin;
233    int keymax;
234    int valmin;
235    int valmax;
236    long bytesToWrite;
237    Random r = new Random();
238
239    private int generateSentence(Text t, int noWords) {
240      sentence.setLength(0);
241      --noWords;
242      for (int i = 0; i < noWords; ++i) {
243        sentence.append(words[r.nextInt(words.length)]);
244        sentence.append(" ");
245      }
246      if (noWords >= 0) sentence.append(words[r.nextInt(words.length)]);
247      t.set(sentence.toString());
248      return sentence.length();
249    }
250
251    public void configure(JobConf job) {
252      bytesToWrite = job.getLong("test.randomtextwrite.bytes_per_map",
253                                    1*1024*1024*1024);
254      keymin = job.getInt("test.randomtextwrite.min_words_key", 5);
255      keymax = job.getInt("test.randomtextwrite.max_words_key", 10);
256      valmin = job.getInt("test.randomtextwrite.min_words_value", 5);
257      valmax = job.getInt("test.randomtextwrite.max_words_value", 10);
258    }
259
260    public void map(Text key, Text val, OutputCollector<Text,Text> output,
261        Reporter reporter) throws IOException {
262      long acc = 0L;
263      long recs = 0;
264      final int keydiff = keymax - keymin;
265      final int valdiff = valmax - valmin;
266      for (long i = 0L; acc < bytesToWrite; ++i) {
267        int recacc = 0;
268        recacc += generateSentence(key, keymin +
269            (0 == keydiff ? 0 : r.nextInt(keydiff)));
270        recacc += generateSentence(val, valmin +
271            (0 == valdiff ? 0 : r.nextInt(valdiff)));
272        output.collect(key, val);
273        ++recs;
274        acc += recacc;
275        reporter.incrCounter(Counters.BYTES_WRITTEN, recacc);
276        reporter.incrCounter(Counters.RECORDS_WRITTEN, 1);
277        reporter.setStatus(acc + "/" + (bytesToWrite - acc) + " bytes");
278      }
279      reporter.setStatus("Wrote " + recs + " records");
280    }
281
282  }
283
284  /**
285   * When no input dir is specified, generate random data.
286   */
287  protected static void confRandom(JobConf job)
288      throws IOException {
289    // from RandomWriter
290    job.setInputFormat(RandomInputFormat.class);
291    job.setMapperClass(RandomMapOutput.class);
292
293    final ClusterStatus cluster = new JobClient(job).getClusterStatus();
294    int numMapsPerHost = job.getInt("test.randomtextwrite.maps_per_host", 10);
295    long numBytesToWritePerMap =
296      job.getLong("test.randomtextwrite.bytes_per_map", 1*1024*1024*1024);
297    if (numBytesToWritePerMap == 0) {
298      throw new IOException(
299          "Cannot have test.randomtextwrite.bytes_per_map set to 0");
300    }
301    long totalBytesToWrite = job.getLong("test.randomtextwrite.total_bytes",
302         numMapsPerHost * numBytesToWritePerMap * cluster.getTaskTrackers());
303    int numMaps = (int)(totalBytesToWrite / numBytesToWritePerMap);
304    if (numMaps == 0 && totalBytesToWrite > 0) {
305      numMaps = 1;
306      job.setLong("test.randomtextwrite.bytes_per_map", totalBytesToWrite);
307    }
308    job.setNumMapTasks(numMaps);
309  }
310
311
312  // Sampling //
313
314  static abstract class SampleMapReduceBase<K extends WritableComparable,
315                                            V extends Writable>
316      extends MapReduceBase {
317
318    private long total;
319    private long kept = 0;
320    private float keep;
321
322    protected void setKeep(float keep) {
323      this.keep = keep;
324    }
325
326    protected void emit(K key, V val, OutputCollector<K,V> out)
327        throws IOException {
328      ++total;
329      while((float) kept / total < keep) {
330        ++kept;
331        out.collect(key, val);
332      }
333    }
334  }
335
336  public static class SampleMapper<K extends WritableComparable, V extends Writable>
337      extends SampleMapReduceBase<K,V> implements Mapper<K,V,K,V> {
338
339    public void configure(JobConf job) {
340      setKeep(job.getFloat("hadoop.sort.map.keep.percent", (float)100.0) /
341        (float)100.0);
342    }
343
344    public void map(K key, V val,
345                    OutputCollector<K,V> output, Reporter reporter)
346        throws IOException {
347      emit(key, val, output);
348    }
349
350  }
351
352  public static class SampleReducer<K extends WritableComparable, V extends Writable>
353      extends SampleMapReduceBase<K,V> implements Reducer<K,V,K,V> {
354
355    public void configure(JobConf job) {
356      setKeep(job.getFloat("hadoop.sort.reduce.keep.percent", (float)100.0) /
357        (float)100.0);
358    }
359
360    public void reduce(K key, Iterator<V> values,
361                       OutputCollector<K,V> output, Reporter reporter)
362        throws IOException {
363      while (values.hasNext()) {
364        emit(key, values.next(), output);
365      }
366    }
367
368  }
369
370  // Indirect reads //
371
372  /**
373   * Obscures the InputFormat and location information to simulate maps
374   * reading input from arbitrary locations (&quot;indirect&quot; reads).
375   */
376  static class IndirectInputFormat implements InputFormat {
377
378    static class IndirectSplit implements InputSplit {
379      Path file;
380      long len;
381      public IndirectSplit() { }
382      public IndirectSplit(Path file, long len) {
383        this.file = file;
384        this.len = len;
385      }
386      public Path getPath() { return file; }
387      public long getLength() { return len; }
388      public String[] getLocations() throws IOException {
389        return new String[]{};
390      }
391      public void write(DataOutput out) throws IOException {
392        WritableUtils.writeString(out, file.toString());
393        WritableUtils.writeVLong(out, len);
394      }
395      public void readFields(DataInput in) throws IOException {
396        file = new Path(WritableUtils.readString(in));
397        len = WritableUtils.readVLong(in);
398      }
399    }
400
401    public InputSplit[] getSplits(JobConf job, int numSplits)
402        throws IOException {
403
404      Path src = new Path(job.get("mapred.indirect.input.file", null));
405      FileSystem fs = src.getFileSystem(job);
406
407      ArrayList<IndirectSplit> splits = new ArrayList<IndirectSplit>(numSplits);
408      LongWritable key = new LongWritable();
409      Text value = new Text();
410      for (SequenceFile.Reader sl = new SequenceFile.Reader(fs, src, job);
411           sl.next(key, value);) {
412        splits.add(new IndirectSplit(new Path(value.toString()), key.get()));
413      }
414
415      return splits.toArray(new IndirectSplit[splits.size()]);
416    }
417
418    public RecordReader getRecordReader(InputSplit split, JobConf job,
419        Reporter reporter) throws IOException {
420      InputFormat indirIF = (InputFormat)ReflectionUtils.newInstance(
421          job.getClass("mapred.indirect.input.format",
422            SequenceFileInputFormat.class), job);
423      IndirectSplit is = ((IndirectSplit)split);
424      return indirIF.getRecordReader(new FileSplit(is.getPath(), 0,
425            is.getLength(), (String[])null),
426          job, reporter);
427    }
428  }
429
430  /**
431   * A random list of 1000 words from /usr/share/dict/words
432   */
433  private static final String[] words = {
434    "diurnalness", "Homoiousian", "spiranthic", "tetragynian",
435    "silverhead", "ungreat", "lithograph", "exploiter",
436    "physiologian", "by", "hellbender", "Filipendula",
437    "undeterring", "antiscolic", "pentagamist", "hypoid",
438    "cacuminal", "sertularian", "schoolmasterism", "nonuple",
439    "gallybeggar", "phytonic", "swearingly", "nebular",
440    "Confervales", "thermochemically", "characinoid", "cocksuredom",
441    "fallacious", "feasibleness", "debromination", "playfellowship",
442    "tramplike", "testa", "participatingly", "unaccessible",
443    "bromate", "experientialist", "roughcast", "docimastical",
444    "choralcelo", "blightbird", "peptonate", "sombreroed",
445    "unschematized", "antiabolitionist", "besagne", "mastication",
446    "bromic", "sviatonosite", "cattimandoo", "metaphrastical",
447    "endotheliomyoma", "hysterolysis", "unfulminated", "Hester",
448    "oblongly", "blurredness", "authorling", "chasmy",
449    "Scorpaenidae", "toxihaemia", "Dictograph", "Quakerishly",
450    "deaf", "timbermonger", "strammel", "Thraupidae",
451    "seditious", "plerome", "Arneb", "eristically",
452    "serpentinic", "glaumrie", "socioromantic", "apocalypst",
453    "tartrous", "Bassaris", "angiolymphoma", "horsefly",
454    "kenno", "astronomize", "euphemious", "arsenide",
455    "untongued", "parabolicness", "uvanite", "helpless",
456    "gemmeous", "stormy", "templar", "erythrodextrin",
457    "comism", "interfraternal", "preparative", "parastas",
458    "frontoorbital", "Ophiosaurus", "diopside", "serosanguineous",
459    "ununiformly", "karyological", "collegian", "allotropic",
460    "depravity", "amylogenesis", "reformatory", "epidymides",
461    "pleurotropous", "trillium", "dastardliness", "coadvice",
462    "embryotic", "benthonic", "pomiferous", "figureheadship",
463    "Megaluridae", "Harpa", "frenal", "commotion",
464    "abthainry", "cobeliever", "manilla", "spiciferous",
465    "nativeness", "obispo", "monilioid", "biopsic",
466    "valvula", "enterostomy", "planosubulate", "pterostigma",
467    "lifter", "triradiated", "venialness", "tum",
468    "archistome", "tautness", "unswanlike", "antivenin",
469    "Lentibulariaceae", "Triphora", "angiopathy", "anta",
470    "Dawsonia", "becomma", "Yannigan", "winterproof",
471    "antalgol", "harr", "underogating", "ineunt",
472    "cornberry", "flippantness", "scyphostoma", "approbation",
473    "Ghent", "Macraucheniidae", "scabbiness", "unanatomized",
474    "photoelasticity", "eurythermal", "enation", "prepavement",
475    "flushgate", "subsequentially", "Edo", "antihero",
476    "Isokontae", "unforkedness", "porriginous", "daytime",
477    "nonexecutive", "trisilicic", "morphiomania", "paranephros",
478    "botchedly", "impugnation", "Dodecatheon", "obolus",
479    "unburnt", "provedore", "Aktistetae", "superindifference",
480    "Alethea", "Joachimite", "cyanophilous", "chorograph",
481    "brooky", "figured", "periclitation", "quintette",
482    "hondo", "ornithodelphous", "unefficient", "pondside",
483    "bogydom", "laurinoxylon", "Shiah", "unharmed",
484    "cartful", "noncrystallized", "abusiveness", "cromlech",
485    "japanned", "rizzomed", "underskin", "adscendent",
486    "allectory", "gelatinousness", "volcano", "uncompromisingly",
487    "cubit", "idiotize", "unfurbelowed", "undinted",
488    "magnetooptics", "Savitar", "diwata", "ramosopalmate",
489    "Pishquow", "tomorn", "apopenptic", "Haversian",
490    "Hysterocarpus", "ten", "outhue", "Bertat",
491    "mechanist", "asparaginic", "velaric", "tonsure",
492    "bubble", "Pyrales", "regardful", "glyphography",
493    "calabazilla", "shellworker", "stradametrical", "havoc",
494    "theologicopolitical", "sawdust", "diatomaceous", "jajman",
495    "temporomastoid", "Serrifera", "Ochnaceae", "aspersor",
496    "trailmaking", "Bishareen", "digitule", "octogynous",
497    "epididymitis", "smokefarthings", "bacillite", "overcrown",
498    "mangonism", "sirrah", "undecorated", "psychofugal",
499    "bismuthiferous", "rechar", "Lemuridae", "frameable",
500    "thiodiazole", "Scanic", "sportswomanship", "interruptedness",
501    "admissory", "osteopaedion", "tingly", "tomorrowness",
502    "ethnocracy", "trabecular", "vitally", "fossilism",
503    "adz", "metopon", "prefatorial", "expiscate",
504    "diathermacy", "chronist", "nigh", "generalizable",
505    "hysterogen", "aurothiosulphuric", "whitlowwort", "downthrust",
506    "Protestantize", "monander", "Itea", "chronographic",
507    "silicize", "Dunlop", "eer", "componental",
508    "spot", "pamphlet", "antineuritic", "paradisean",
509    "interruptor", "debellator", "overcultured", "Florissant",
510    "hyocholic", "pneumatotherapy", "tailoress", "rave",
511    "unpeople", "Sebastian", "thermanesthesia", "Coniferae",
512    "swacking", "posterishness", "ethmopalatal", "whittle",
513    "analgize", "scabbardless", "naught", "symbiogenetically",
514    "trip", "parodist", "columniform", "trunnel",
515    "yawler", "goodwill", "pseudohalogen", "swangy",
516    "cervisial", "mediateness", "genii", "imprescribable",
517    "pony", "consumptional", "carposporangial", "poleax",
518    "bestill", "subfebrile", "sapphiric", "arrowworm",
519    "qualminess", "ultraobscure", "thorite", "Fouquieria",
520    "Bermudian", "prescriber", "elemicin", "warlike",
521    "semiangle", "rotular", "misthread", "returnability",
522    "seraphism", "precostal", "quarried", "Babylonism",
523    "sangaree", "seelful", "placatory", "pachydermous",
524    "bozal", "galbulus", "spermaphyte", "cumbrousness",
525    "pope", "signifier", "Endomycetaceae", "shallowish",
526    "sequacity", "periarthritis", "bathysphere", "pentosuria",
527    "Dadaism", "spookdom", "Consolamentum", "afterpressure",
528    "mutter", "louse", "ovoviviparous", "corbel",
529    "metastoma", "biventer", "Hydrangea", "hogmace",
530    "seizing", "nonsuppressed", "oratorize", "uncarefully",
531    "benzothiofuran", "penult", "balanocele", "macropterous",
532    "dishpan", "marten", "absvolt", "jirble",
533    "parmelioid", "airfreighter", "acocotl", "archesporial",
534    "hypoplastral", "preoral", "quailberry", "cinque",
535    "terrestrially", "stroking", "limpet", "moodishness",
536    "canicule", "archididascalian", "pompiloid", "overstaid",
537    "introducer", "Italical", "Christianopaganism", "prescriptible",
538    "subofficer", "danseuse", "cloy", "saguran",
539    "frictionlessly", "deindividualization", "Bulanda", "ventricous",
540    "subfoliar", "basto", "scapuloradial", "suspend",
541    "stiffish", "Sphenodontidae", "eternal", "verbid",
542    "mammonish", "upcushion", "barkometer", "concretion",
543    "preagitate", "incomprehensible", "tristich", "visceral",
544    "hemimelus", "patroller", "stentorophonic", "pinulus",
545    "kerykeion", "brutism", "monstership", "merciful",
546    "overinstruct", "defensibly", "bettermost", "splenauxe",
547    "Mormyrus", "unreprimanded", "taver", "ell",
548    "proacquittal", "infestation", "overwoven", "Lincolnlike",
549    "chacona", "Tamil", "classificational", "lebensraum",
550    "reeveland", "intuition", "Whilkut", "focaloid",
551    "Eleusinian", "micromembrane", "byroad", "nonrepetition",
552    "bacterioblast", "brag", "ribaldrous", "phytoma",
553    "counteralliance", "pelvimetry", "pelf", "relaster",
554    "thermoresistant", "aneurism", "molossic", "euphonym",
555    "upswell", "ladhood", "phallaceous", "inertly",
556    "gunshop", "stereotypography", "laryngic", "refasten",
557    "twinling", "oflete", "hepatorrhaphy", "electrotechnics",
558    "cockal", "guitarist", "topsail", "Cimmerianism",
559    "larklike", "Llandovery", "pyrocatechol", "immatchable",
560    "chooser", "metrocratic", "craglike", "quadrennial",
561    "nonpoisonous", "undercolored", "knob", "ultratense",
562    "balladmonger", "slait", "sialadenitis", "bucketer",
563    "magnificently", "unstipulated", "unscourged", "unsupercilious",
564    "packsack", "pansophism", "soorkee", "percent",
565    "subirrigate", "champer", "metapolitics", "spherulitic",
566    "involatile", "metaphonical", "stachyuraceous", "speckedness",
567    "bespin", "proboscidiform", "gul", "squit",
568    "yeelaman", "peristeropode", "opacousness", "shibuichi",
569    "retinize", "yote", "misexposition", "devilwise",
570    "pumpkinification", "vinny", "bonze", "glossing",
571    "decardinalize", "transcortical", "serphoid", "deepmost",
572    "guanajuatite", "wemless", "arval", "lammy",
573    "Effie", "Saponaria", "tetrahedral", "prolificy",
574    "excerpt", "dunkadoo", "Spencerism", "insatiately",
575    "Gilaki", "oratorship", "arduousness", "unbashfulness",
576    "Pithecolobium", "unisexuality", "veterinarian", "detractive",
577    "liquidity", "acidophile", "proauction", "sural",
578    "totaquina", "Vichyite", "uninhabitedness", "allegedly",
579    "Gothish", "manny", "Inger", "flutist",
580    "ticktick", "Ludgatian", "homotransplant", "orthopedical",
581    "diminutively", "monogoneutic", "Kenipsim", "sarcologist",
582    "drome", "stronghearted", "Fameuse", "Swaziland",
583    "alen", "chilblain", "beatable", "agglomeratic",
584    "constitutor", "tendomucoid", "porencephalous", "arteriasis",
585    "boser", "tantivy", "rede", "lineamental",
586    "uncontradictableness", "homeotypical", "masa", "folious",
587    "dosseret", "neurodegenerative", "subtransverse", "Chiasmodontidae",
588    "palaeotheriodont", "unstressedly", "chalcites", "piquantness",
589    "lampyrine", "Aplacentalia", "projecting", "elastivity",
590    "isopelletierin", "bladderwort", "strander", "almud",
591    "iniquitously", "theologal", "bugre", "chargeably",
592    "imperceptivity", "meriquinoidal", "mesophyte", "divinator",
593    "perfunctory", "counterappellant", "synovial", "charioteer",
594    "crystallographical", "comprovincial", "infrastapedial", "pleasurehood",
595    "inventurous", "ultrasystematic", "subangulated", "supraoesophageal",
596    "Vaishnavism", "transude", "chrysochrous", "ungrave",
597    "reconciliable", "uninterpleaded", "erlking", "wherefrom",
598    "aprosopia", "antiadiaphorist", "metoxazine", "incalculable",
599    "umbellic", "predebit", "foursquare", "unimmortal",
600    "nonmanufacture", "slangy", "predisputant", "familist",
601    "preaffiliate", "friarhood", "corelysis", "zoonitic",
602    "halloo", "paunchy", "neuromimesis", "aconitine",
603    "hackneyed", "unfeeble", "cubby", "autoschediastical",
604    "naprapath", "lyrebird", "inexistency", "leucophoenicite",
605    "ferrogoslarite", "reperuse", "uncombable", "tambo",
606    "propodiale", "diplomatize", "Russifier", "clanned",
607    "corona", "michigan", "nonutilitarian", "transcorporeal",
608    "bought", "Cercosporella", "stapedius", "glandularly",
609    "pictorially", "weism", "disilane", "rainproof",
610    "Caphtor", "scrubbed", "oinomancy", "pseudoxanthine",
611    "nonlustrous", "redesertion", "Oryzorictinae", "gala",
612    "Mycogone", "reappreciate", "cyanoguanidine", "seeingness",
613    "breadwinner", "noreast", "furacious", "epauliere",
614    "omniscribent", "Passiflorales", "uninductive", "inductivity",
615    "Orbitolina", "Semecarpus", "migrainoid", "steprelationship",
616    "phlogisticate", "mesymnion", "sloped", "edificator",
617    "beneficent", "culm", "paleornithology", "unurban",
618    "throbless", "amplexifoliate", "sesquiquintile", "sapience",
619    "astucious", "dithery", "boor", "ambitus",
620    "scotching", "uloid", "uncompromisingness", "hoove",
621    "waird", "marshiness", "Jerusalem", "mericarp",
622    "unevoked", "benzoperoxide", "outguess", "pyxie",
623    "hymnic", "euphemize", "mendacity", "erythremia",
624    "rosaniline", "unchatteled", "lienteria", "Bushongo",
625    "dialoguer", "unrepealably", "rivethead", "antideflation",
626    "vinegarish", "manganosiderite", "doubtingness", "ovopyriform",
627    "Cephalodiscus", "Muscicapa", "Animalivora", "angina",
628    "planispheric", "ipomoein", "cuproiodargyrite", "sandbox",
629    "scrat", "Munnopsidae", "shola", "pentafid",
630    "overstudiousness", "times", "nonprofession", "appetible",
631    "valvulotomy", "goladar", "uniarticular", "oxyterpene",
632    "unlapsing", "omega", "trophonema", "seminonflammable",
633    "circumzenithal", "starer", "depthwise", "liberatress",
634    "unleavened", "unrevolting", "groundneedle", "topline",
635    "wandoo", "umangite", "ordinant", "unachievable",
636    "oversand", "snare", "avengeful", "unexplicit",
637    "mustafina", "sonable", "rehabilitative", "eulogization",
638    "papery", "technopsychology", "impressor", "cresylite",
639    "entame", "transudatory", "scotale", "pachydermatoid",
640    "imaginary", "yeat", "slipped", "stewardship",
641    "adatom", "cockstone", "skyshine", "heavenful",
642    "comparability", "exprobratory", "dermorhynchous", "parquet",
643    "cretaceous", "vesperal", "raphis", "undangered",
644    "Glecoma", "engrain", "counteractively", "Zuludom",
645    "orchiocatabasis", "Auriculariales", "warriorwise", "extraorganismal",
646    "overbuilt", "alveolite", "tetchy", "terrificness",
647    "widdle", "unpremonished", "rebilling", "sequestrum",
648    "equiconvex", "heliocentricism", "catabaptist", "okonite",
649    "propheticism", "helminthagogic", "calycular", "giantly",
650    "wingable", "golem", "unprovided", "commandingness",
651    "greave", "haply", "doina", "depressingly",
652    "subdentate", "impairment", "decidable", "neurotrophic",
653    "unpredict", "bicorporeal", "pendulant", "flatman",
654    "intrabred", "toplike", "Prosobranchiata", "farrantly",
655    "toxoplasmosis", "gorilloid", "dipsomaniacal", "aquiline",
656    "atlantite", "ascitic", "perculsive", "prospectiveness",
657    "saponaceous", "centrifugalization", "dinical", "infravaginal",
658    "beadroll", "affaite", "Helvidian", "tickleproof",
659    "abstractionism", "enhedge", "outwealth", "overcontribute",
660    "coldfinch", "gymnastic", "Pincian", "Munychian",
661    "codisjunct", "quad", "coracomandibular", "phoenicochroite",
662    "amender", "selectivity", "putative", "semantician",
663    "lophotrichic", "Spatangoidea", "saccharogenic", "inferent",
664    "Triconodonta", "arrendation", "sheepskin", "taurocolla",
665    "bunghole", "Machiavel", "triakistetrahedral", "dehairer",
666    "prezygapophysial", "cylindric", "pneumonalgia", "sleigher",
667    "emir", "Socraticism", "licitness", "massedly",
668    "instructiveness", "sturdied", "redecrease", "starosta",
669    "evictor", "orgiastic", "squdge", "meloplasty",
670    "Tsonecan", "repealableness", "swoony", "myesthesia",
671    "molecule", "autobiographist", "reciprocation", "refective",
672    "unobservantness", "tricae", "ungouged", "floatability",
673    "Mesua", "fetlocked", "chordacentrum", "sedentariness",
674    "various", "laubanite", "nectopod", "zenick",
675    "sequentially", "analgic", "biodynamics", "posttraumatic",
676    "nummi", "pyroacetic", "bot", "redescend",
677    "dispermy", "undiffusive", "circular", "trillion",
678    "Uraniidae", "ploration", "discipular", "potentness",
679    "sud", "Hu", "Eryon", "plugger",
680    "subdrainage", "jharal", "abscission", "supermarket",
681    "countergabion", "glacierist", "lithotresis", "minniebush",
682    "zanyism", "eucalypteol", "sterilely", "unrealize",
683    "unpatched", "hypochondriacism", "critically", "cheesecutter",
684  };
685}
Note: See TracBrowser for help on using the repository browser.