GenericMRLoadGenerator.java 30 KB

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