diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 12c4dea136b..0654aefe90e 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -82,6 +82,14 @@ API Changes
* LUCENE-5454: Add RandomAccessOrds, an optional extension of SortedSetDocValues
that supports random access to the ordinals in a document. (Robert Muir)
+* LUCENE-5468: Move offline Sort (from suggest module) to OfflineSort. (Robert Muir)
+
+Optimizations
+
+* LUCENE-5468: HunspellStemFilter uses 10 to 100x less RAM. It also loads
+ all known openoffice dictionaries without error, and supports an additional
+ longestOnly option for a less aggressive approach. (Robert Muir)
+
Bug fixes
* LUCENE-5450: Fix getField() NPE issues with SpanOr/SpanNear when they have an
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
new file mode 100644
index 00000000000..e62e662ddf2
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Dictionary.java
@@ -0,0 +1,821 @@
+package org.apache.lucene.analysis.hunspell;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesReader;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
+import org.apache.lucene.util.UnicodeUtil;
+import org.apache.lucene.util.fst.Builder;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.IntSequenceOutputs;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+
+import java.io.BufferedInputStream;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.LineNumberReader;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.regex.Pattern;
+
+/**
+ * In-memory structure for the dictionary (.dic) and affix (.aff)
+ * data of a hunspell dictionary.
+ */
+public class Dictionary {
+
+ static final char[] NOFLAGS = new char[0];
+
+ private static final String ALIAS_KEY = "AF";
+ private static final String PREFIX_KEY = "PFX";
+ private static final String SUFFIX_KEY = "SFX";
+ private static final String FLAG_KEY = "FLAG";
+
+ private static final String NUM_FLAG_TYPE = "num";
+ private static final String UTF8_FLAG_TYPE = "UTF-8";
+ private static final String LONG_FLAG_TYPE = "long";
+
+ private static final String PREFIX_CONDITION_REGEX_PATTERN = "%s.*";
+ private static final String SUFFIX_CONDITION_REGEX_PATTERN = ".*%s";
+
+ FST prefixes;
+ FST suffixes;
+
+ // all Patterns used by prefixes and suffixes. these are typically re-used across
+ // many affix stripping rules. so these are deduplicated, to save RAM.
+ // TODO: maybe don't use Pattern for the condition check...
+ // TODO: when we cut over Affix to FST, just store integer index to this.
+ ArrayList patterns = new ArrayList<>();
+
+ // the entries in the .dic file, mapping to their set of flags.
+ // the fst output is the ordinal for flagLookup
+ FST words;
+ // the list of unique flagsets (wordforms). theoretically huge, but practically
+ // small (e.g. for polish this is 756), otherwise humans wouldn't be able to deal with it either.
+ BytesRefHash flagLookup = new BytesRefHash();
+
+ // the list of unique strip affixes.
+ BytesRefHash stripLookup = new BytesRefHash();
+
+ // 8 bytes per affix
+ byte[] affixData = new byte[64];
+ private int currentAffix = 0;
+
+ private FlagParsingStrategy flagParsingStrategy = new SimpleFlagParsingStrategy(); // Default flag parsing strategy
+
+ private String[] aliases;
+ private int aliasCount = 0;
+
+ private final File tempDir = OfflineSorter.defaultTempDir(); // TODO: make this configurable?
+
+ boolean ignoreCase;
+
+ /**
+ * Creates a new Dictionary containing the information read from the provided InputStreams to hunspell affix
+ * and dictionary files.
+ * You have to close the provided InputStreams yourself.
+ *
+ * @param affix InputStream for reading the hunspell affix file (won't be closed).
+ * @param dictionary InputStream for reading the hunspell dictionary file (won't be closed).
+ * @throws IOException Can be thrown while reading from the InputStreams
+ * @throws ParseException Can be thrown if the content of the files does not meet expected formats
+ */
+ public Dictionary(InputStream affix, InputStream dictionary) throws IOException, ParseException {
+ this(affix, Collections.singletonList(dictionary), false);
+ }
+
+ /**
+ * Creates a new Dictionary containing the information read from the provided InputStreams to hunspell affix
+ * and dictionary files.
+ * You have to close the provided InputStreams yourself.
+ *
+ * @param affix InputStream for reading the hunspell affix file (won't be closed).
+ * @param dictionaries InputStream for reading the hunspell dictionary files (won't be closed).
+ * @throws IOException Can be thrown while reading from the InputStreams
+ * @throws ParseException Can be thrown if the content of the files does not meet expected formats
+ */
+ public Dictionary(InputStream affix, List dictionaries, boolean ignoreCase) throws IOException, ParseException {
+ this.ignoreCase = ignoreCase;
+ BufferedInputStream buffered = new BufferedInputStream(affix, 8192);
+ buffered.mark(8192);
+ String encoding = getDictionaryEncoding(affix);
+ buffered.reset();
+ CharsetDecoder decoder = getJavaEncoding(encoding);
+ readAffixFile(buffered, decoder);
+ flagLookup.add(new BytesRef()); // no flags -> ord 0
+ stripLookup.add(new BytesRef()); // no strip -> ord 0
+ PositiveIntOutputs o = PositiveIntOutputs.getSingleton();
+ Builder b = new Builder(FST.INPUT_TYPE.BYTE4, o);
+ readDictionaryFiles(dictionaries, decoder, b);
+ words = b.finish();
+ }
+
+ /**
+ * Looks up words that match the String created from the given char array, offset and length
+ *
+ * @param word Char array to generate the String from
+ * @param offset Offset in the char array that the String starts at
+ * @param length Length from the offset that the String is
+ * @return List of HunspellWords that match the generated String, or {@code null} if none are found
+ */
+ char[] lookupWord(char word[], int offset, int length, BytesRef scratch) {
+ Integer ord = null;
+ try {
+ ord = lookupOrd(word, offset, length);
+ } catch (IOException ex) { /* bogus */ }
+ if (ord == null) {
+ return null;
+ }
+ return decodeFlags(flagLookup.get(ord, scratch));
+ }
+
+ Integer lookupOrd(char word[], int offset, int length) throws IOException {
+ final FST.BytesReader bytesReader = words.getBytesReader();
+ final FST.Arc arc = words.getFirstArc(new FST.Arc());
+ // Accumulate output as we go
+ final Long NO_OUTPUT = words.outputs.getNoOutput();
+ Long output = NO_OUTPUT;
+
+ int l = offset + length;
+ for (int i = offset, cp = 0; i < l; i += Character.charCount(cp)) {
+ cp = Character.codePointAt(word, i, l);
+ if (words.findTargetArc(cp, arc, arc, bytesReader) == null) {
+ return null;
+ } else if (arc.output != NO_OUTPUT) {
+ output = words.outputs.add(output, arc.output);
+ }
+ }
+ if (words.findTargetArc(FST.END_LABEL, arc, arc, bytesReader) == null) {
+ return null;
+ } else if (arc.output != NO_OUTPUT) {
+ return words.outputs.add(output, arc.output).intValue();
+ } else {
+ return output.intValue();
+ }
+ }
+
+ /**
+ * Looks up HunspellAffix prefixes that have an append that matches the String created from the given char array, offset and length
+ *
+ * @param word Char array to generate the String from
+ * @param offset Offset in the char array that the String starts at
+ * @param length Length from the offset that the String is
+ * @return List of HunspellAffix prefixes with an append that matches the String, or {@code null} if none are found
+ */
+ IntsRef lookupPrefix(char word[], int offset, int length) {
+ return lookupAffix(prefixes, word, offset, length);
+ }
+
+ /**
+ * Looks up HunspellAffix suffixes that have an append that matches the String created from the given char array, offset and length
+ *
+ * @param word Char array to generate the String from
+ * @param offset Offset in the char array that the String starts at
+ * @param length Length from the offset that the String is
+ * @return List of HunspellAffix suffixes with an append that matches the String, or {@code null} if none are found
+ */
+ IntsRef lookupSuffix(char word[], int offset, int length) {
+ return lookupAffix(suffixes, word, offset, length);
+ }
+
+ // TODO: this is pretty stupid, considering how the stemming algorithm works
+ // we can speed it up to be significantly faster!
+ IntsRef lookupAffix(FST fst, char word[], int offset, int length) {
+ if (fst == null) {
+ return null;
+ }
+ final FST.BytesReader bytesReader = fst.getBytesReader();
+ final FST.Arc arc = fst.getFirstArc(new FST.Arc());
+ // Accumulate output as we go
+ final IntsRef NO_OUTPUT = fst.outputs.getNoOutput();
+ IntsRef output = NO_OUTPUT;
+
+ int l = offset + length;
+ try {
+ for (int i = offset, cp = 0; i < l; i += Character.charCount(cp)) {
+ cp = Character.codePointAt(word, i, l);
+ if (fst.findTargetArc(cp, arc, arc, bytesReader) == null) {
+ return null;
+ } else if (arc.output != NO_OUTPUT) {
+ output = fst.outputs.add(output, arc.output);
+ }
+ }
+ if (fst.findTargetArc(FST.END_LABEL, arc, arc, bytesReader) == null) {
+ return null;
+ } else if (arc.output != NO_OUTPUT) {
+ return fst.outputs.add(output, arc.output);
+ } else {
+ return output;
+ }
+ } catch (IOException bogus) {
+ throw new RuntimeException(bogus);
+ }
+ }
+
+ /**
+ * Reads the affix file through the provided InputStream, building up the prefix and suffix maps
+ *
+ * @param affixStream InputStream to read the content of the affix file from
+ * @param decoder CharsetDecoder to decode the content of the file
+ * @throws IOException Can be thrown while reading from the InputStream
+ */
+ private void readAffixFile(InputStream affixStream, CharsetDecoder decoder) throws IOException, ParseException {
+ TreeMap> prefixes = new TreeMap<>();
+ TreeMap> suffixes = new TreeMap<>();
+ Map seenPatterns = new HashMap<>();
+
+ LineNumberReader reader = new LineNumberReader(new InputStreamReader(affixStream, decoder));
+ String line = null;
+ while ((line = reader.readLine()) != null) {
+ if (line.startsWith(ALIAS_KEY)) {
+ parseAlias(line);
+ } else if (line.startsWith(PREFIX_KEY)) {
+ parseAffix(prefixes, line, reader, PREFIX_CONDITION_REGEX_PATTERN, seenPatterns);
+ } else if (line.startsWith(SUFFIX_KEY)) {
+ parseAffix(suffixes, line, reader, SUFFIX_CONDITION_REGEX_PATTERN, seenPatterns);
+ } else if (line.startsWith(FLAG_KEY)) {
+ // Assume that the FLAG line comes before any prefix or suffixes
+ // Store the strategy so it can be used when parsing the dic file
+ flagParsingStrategy = getFlagParsingStrategy(line);
+ }
+ }
+
+ this.prefixes = affixFST(prefixes);
+ this.suffixes = affixFST(suffixes);
+ }
+
+ private FST affixFST(TreeMap> affixes) throws IOException {
+ IntSequenceOutputs outputs = IntSequenceOutputs.getSingleton();
+ Builder builder = new Builder<>(FST.INPUT_TYPE.BYTE4, outputs);
+
+ IntsRef scratch = new IntsRef();
+ for (Map.Entry> entry : affixes.entrySet()) {
+ Util.toUTF32(entry.getKey(), scratch);
+ List entries = entry.getValue();
+ IntsRef output = new IntsRef(entries.size());
+ for (Character c : entries) {
+ output.ints[output.length++] = c;
+ }
+ builder.add(scratch, output);
+ }
+ return builder.finish();
+ }
+
+ /**
+ * Parses a specific affix rule putting the result into the provided affix map
+ *
+ * @param affixes Map where the result of the parsing will be put
+ * @param header Header line of the affix rule
+ * @param reader BufferedReader to read the content of the rule from
+ * @param conditionPattern {@link String#format(String, Object...)} pattern to be used to generate the condition regex
+ * pattern
+ * @param seenPatterns map from condition -> index of patterns, for deduplication.
+ * @throws IOException Can be thrown while reading the rule
+ */
+ private void parseAffix(TreeMap> affixes,
+ String header,
+ LineNumberReader reader,
+ String conditionPattern,
+ Map seenPatterns) throws IOException, ParseException {
+
+ BytesRef scratch = new BytesRef();
+ String args[] = header.split("\\s+");
+
+ boolean crossProduct = args[2].equals("Y");
+
+ int numLines = Integer.parseInt(args[3]);
+ affixData = ArrayUtil.grow(affixData, (currentAffix << 3) + (numLines << 3));
+ ByteArrayDataOutput affixWriter = new ByteArrayDataOutput(affixData, currentAffix << 3, numLines << 3);
+
+ for (int i = 0; i < numLines; i++) {
+ if (currentAffix > Short.MAX_VALUE) {
+ throw new UnsupportedOperationException("Too many affixes, please report this to dev@lucene.apache.org");
+ }
+ assert affixWriter.getPosition() == currentAffix << 3;
+ String line = reader.readLine();
+ String ruleArgs[] = line.split("\\s+");
+
+ if (ruleArgs.length < 5) {
+ throw new ParseException("The affix file contains a rule with less than five elements", reader.getLineNumber());
+ }
+
+ char flag = flagParsingStrategy.parseFlag(ruleArgs[1]);
+ String strip = ruleArgs[2].equals("0") ? "" : ruleArgs[2];
+ String affixArg = ruleArgs[3];
+ char appendFlags[] = null;
+
+ int flagSep = affixArg.lastIndexOf('/');
+ if (flagSep != -1) {
+ String flagPart = affixArg.substring(flagSep + 1);
+ affixArg = affixArg.substring(0, flagSep);
+
+ if (aliasCount > 0) {
+ flagPart = getAliasValue(Integer.parseInt(flagPart));
+ }
+
+ appendFlags = flagParsingStrategy.parseFlags(flagPart);
+ Arrays.sort(appendFlags);
+ }
+
+ String condition = ruleArgs[4];
+ // at least the gascon affix file has this issue
+ if (condition.startsWith("[") && !condition.endsWith("]")) {
+ condition = condition + "]";
+ }
+ // "dash hasn't got special meaning" (we must escape it)
+ if (condition.indexOf('-') >= 0) {
+ condition = condition.replace("-", "\\-");
+ }
+
+ String regex = String.format(Locale.ROOT, conditionPattern, condition);
+
+ // deduplicate patterns
+ Integer patternIndex = seenPatterns.get(regex);
+ if (patternIndex == null) {
+ patternIndex = patterns.size();
+ seenPatterns.put(regex, patternIndex);
+ Pattern pattern = Pattern.compile(regex);
+ patterns.add(pattern);
+ }
+
+ scratch.copyChars(strip);
+ int stripOrd = stripLookup.add(scratch);
+ if (stripOrd < 0) {
+ // already exists in our hash
+ stripOrd = (-stripOrd)-1;
+ }
+
+ if (appendFlags == null) {
+ appendFlags = NOFLAGS;
+ }
+
+ final int hashCode = encodeFlagsWithHash(scratch, appendFlags);
+ int appendFlagsOrd = flagLookup.add(scratch, hashCode);
+ if (appendFlagsOrd < 0) {
+ // already exists in our hash
+ appendFlagsOrd = (-appendFlagsOrd)-1;
+ } else if (appendFlagsOrd > Short.MAX_VALUE) {
+ // this limit is probably flexible, but its a good sanity check too
+ throw new UnsupportedOperationException("Too many unique flags, please report this to dev@lucene.apache.org");
+ }
+
+ affixWriter.writeShort((short)flag);
+ affixWriter.writeShort((short)stripOrd);
+ // encode crossProduct into patternIndex
+ int patternOrd = patternIndex.intValue() << 1 | (crossProduct ? 1 : 0);
+ affixWriter.writeShort((short)patternOrd);
+ affixWriter.writeShort((short)appendFlagsOrd);
+
+ List list = affixes.get(affixArg);
+ if (list == null) {
+ list = new ArrayList();
+ affixes.put(affixArg, list);
+ }
+
+ list.add((char)currentAffix);
+ currentAffix++;
+ }
+ }
+
+ /**
+ * Parses the encoding specified in the affix file readable through the provided InputStream
+ *
+ * @param affix InputStream for reading the affix file
+ * @return Encoding specified in the affix file
+ * @throws IOException Can be thrown while reading from the InputStream
+ * @throws ParseException Thrown if the first non-empty non-comment line read from the file does not adhere to the format {@code SET }
+ */
+ private String getDictionaryEncoding(InputStream affix) throws IOException, ParseException {
+ final StringBuilder encoding = new StringBuilder();
+ for (;;) {
+ encoding.setLength(0);
+ int ch;
+ while ((ch = affix.read()) >= 0) {
+ if (ch == '\n') {
+ break;
+ }
+ if (ch != '\r') {
+ encoding.append((char)ch);
+ }
+ }
+ if (
+ encoding.length() == 0 || encoding.charAt(0) == '#' ||
+ // this test only at the end as ineffective but would allow lines only containing spaces:
+ encoding.toString().trim().length() == 0
+ ) {
+ if (ch < 0) {
+ throw new ParseException("Unexpected end of affix file.", 0);
+ }
+ continue;
+ }
+ if (encoding.length() > 4 && "SET ".equals(encoding.substring(0, 4))) {
+ // cleanup the encoding string, too (whitespace)
+ return encoding.substring(4).trim();
+ }
+ }
+ }
+
+ static final Map CHARSET_ALIASES;
+ static {
+ Map m = new HashMap<>();
+ m.put("microsoft-cp1251", "windows-1251");
+ m.put("TIS620-2533", "TIS-620");
+ CHARSET_ALIASES = Collections.unmodifiableMap(m);
+ }
+
+ /**
+ * Retrieves the CharsetDecoder for the given encoding. Note, This isn't perfect as I think ISCII-DEVANAGARI and
+ * MICROSOFT-CP1251 etc are allowed...
+ *
+ * @param encoding Encoding to retrieve the CharsetDecoder for
+ * @return CharSetDecoder for the given encoding
+ */
+ private CharsetDecoder getJavaEncoding(String encoding) {
+ if ("ISO8859-14".equals(encoding)) {
+ return new ISO8859_14Decoder();
+ }
+ String canon = CHARSET_ALIASES.get(encoding);
+ if (canon != null) {
+ encoding = canon;
+ }
+ Charset charset = Charset.forName(encoding);
+ return charset.newDecoder().onMalformedInput(CodingErrorAction.REPLACE);
+ }
+
+ /**
+ * Determines the appropriate {@link FlagParsingStrategy} based on the FLAG definition line taken from the affix file
+ *
+ * @param flagLine Line containing the flag information
+ * @return FlagParsingStrategy that handles parsing flags in the way specified in the FLAG definition
+ */
+ private FlagParsingStrategy getFlagParsingStrategy(String flagLine) {
+ String flagType = flagLine.substring(5);
+
+ if (NUM_FLAG_TYPE.equals(flagType)) {
+ return new NumFlagParsingStrategy();
+ } else if (UTF8_FLAG_TYPE.equals(flagType)) {
+ return new SimpleFlagParsingStrategy();
+ } else if (LONG_FLAG_TYPE.equals(flagType)) {
+ return new DoubleASCIIFlagParsingStrategy();
+ }
+
+ throw new IllegalArgumentException("Unknown flag type: " + flagType);
+ }
+
+ /**
+ * Reads the dictionary file through the provided InputStreams, building up the words map
+ *
+ * @param dictionaries InputStreams to read the dictionary file through
+ * @param decoder CharsetDecoder used to decode the contents of the file
+ * @throws IOException Can be thrown while reading from the file
+ */
+ private void readDictionaryFiles(List dictionaries, CharsetDecoder decoder, Builder words) throws IOException {
+ BytesRef flagsScratch = new BytesRef();
+ IntsRef scratchInts = new IntsRef();
+
+ File unsorted = File.createTempFile("unsorted", "dat", tempDir);
+ try (ByteSequencesWriter writer = new ByteSequencesWriter(unsorted)) {
+ for (InputStream dictionary : dictionaries) {
+ BufferedReader lines = new BufferedReader(new InputStreamReader(dictionary, decoder));
+ String line = lines.readLine(); // first line is number of entries (approximately, sometimes)
+
+ while ((line = lines.readLine()) != null) {
+ if (ignoreCase) {
+ int flagSep = line.lastIndexOf('/');
+ if (flagSep == -1) {
+ writer.write(line.toLowerCase(Locale.ROOT).getBytes(IOUtils.CHARSET_UTF_8));
+ } else {
+ StringBuilder sb = new StringBuilder();
+ sb.append(line.substring(0, flagSep).toLowerCase(Locale.ROOT));
+ if (flagSep < line.length()) {
+ sb.append(line.substring(flagSep, line.length()));
+ }
+ writer.write(sb.toString().getBytes(IOUtils.CHARSET_UTF_8));
+ }
+ } else {
+ writer.write(line.getBytes(IOUtils.CHARSET_UTF_8));
+ }
+ }
+ }
+ }
+ File sorted = File.createTempFile("sorted", "dat", tempDir);
+
+ OfflineSorter sorter = new OfflineSorter(new Comparator() {
+ BytesRef scratch1 = new BytesRef();
+ BytesRef scratch2 = new BytesRef();
+
+ @Override
+ public int compare(BytesRef o1, BytesRef o2) {
+ scratch1.bytes = o1.bytes;
+ scratch1.offset = o1.offset;
+ scratch1.length = o1.length;
+
+ for (int i = scratch1.length - 1; i >= 0; i--) {
+ if (scratch1.bytes[scratch1.offset + i] == '/') {
+ scratch1.length = i;
+ break;
+ }
+ }
+
+ scratch2.bytes = o2.bytes;
+ scratch2.offset = o2.offset;
+ scratch2.length = o2.length;
+
+ for (int i = scratch2.length - 1; i >= 0; i--) {
+ if (scratch2.bytes[scratch2.offset + i] == '/') {
+ scratch2.length = i;
+ break;
+ }
+ }
+
+ return scratch1.compareTo(scratch2);
+ }
+ });
+ sorter.sort(unsorted, sorted);
+ unsorted.delete();
+
+ ByteSequencesReader reader = new ByteSequencesReader(sorted);
+ BytesRef scratchLine = new BytesRef();
+
+ // TODO: the flags themselves can be double-chars (long) or also numeric
+ // either way the trick is to encode them as char... but they must be parsed differently
+
+ BytesRef currentEntry = new BytesRef();
+ char currentFlags[] = new char[0];
+
+ String line;
+ while (reader.read(scratchLine)) {
+ line = scratchLine.utf8ToString();
+ String entry;
+ char wordForm[];
+
+ int flagSep = line.lastIndexOf('/');
+ if (flagSep == -1) {
+ wordForm = NOFLAGS;
+ entry = line;
+ } else {
+ // note, there can be comments (morph description) after a flag.
+ // we should really look for any whitespace
+ int end = line.indexOf('\t', flagSep);
+ if (end == -1)
+ end = line.length();
+
+ String flagPart = line.substring(flagSep + 1, end);
+ if (aliasCount > 0) {
+ flagPart = getAliasValue(Integer.parseInt(flagPart));
+ }
+
+ wordForm = flagParsingStrategy.parseFlags(flagPart);
+ Arrays.sort(wordForm);
+ entry = line.substring(0, flagSep);
+ }
+
+ BytesRef scratch = new BytesRef(entry);
+ int cmp = scratch.compareTo(currentEntry);
+ if (cmp < 0) {
+ throw new IllegalArgumentException("out of order: " + scratch.utf8ToString() + " < " + currentEntry.utf8ToString());
+ } else if (cmp == 0) {
+ currentFlags = merge(currentFlags, wordForm);
+ } else {
+ final int hashCode = encodeFlagsWithHash(flagsScratch, currentFlags);
+ int ord = flagLookup.add(flagsScratch, hashCode);
+ if (ord < 0) {
+ // already exists in our hash
+ ord = (-ord)-1;
+ }
+ UnicodeUtil.UTF8toUTF32(currentEntry, scratchInts);
+ words.add(scratchInts, (long)ord);
+ currentEntry = scratch;
+ currentFlags = wordForm;
+ }
+ }
+
+ final int hashCode = encodeFlagsWithHash(flagsScratch, currentFlags);
+ int ord = flagLookup.add(flagsScratch, hashCode);
+ if (ord < 0) {
+ // already exists in our hash
+ ord = (-ord)-1;
+ }
+ UnicodeUtil.UTF8toUTF32(currentEntry, scratchInts);
+ words.add(scratchInts, (long)ord);
+
+ reader.close();
+ sorted.delete();
+ }
+
+ static char[] decodeFlags(BytesRef b) {
+ int len = b.length >>> 1;
+ char flags[] = new char[len];
+ int upto = 0;
+ int end = b.offset + b.length;
+ for (int i = b.offset; i < end; i += 2) {
+ flags[upto++] = (char)((b.bytes[i] << 8) | (b.bytes[i+1] & 0xff));
+ }
+ return flags;
+ }
+
+ static int encodeFlagsWithHash(BytesRef b, char flags[]) {
+ int hash = 0;
+ int len = flags.length << 1;
+ b.grow(len);
+ b.length = len;
+ int upto = b.offset;
+ for (int i = 0; i < flags.length; i++) {
+ int flag = flags[i];
+ hash = 31*hash + (b.bytes[upto++] = (byte) ((flag >> 8) & 0xff));
+ hash = 31*hash + (b.bytes[upto++] = (byte) (flag & 0xff));
+ }
+ return hash;
+ }
+
+ private void parseAlias(String line) {
+ String ruleArgs[] = line.split("\\s+");
+ if (aliases == null) {
+ //first line should be the aliases count
+ final int count = Integer.parseInt(ruleArgs[1]);
+ aliases = new String[count];
+ } else {
+ aliases[aliasCount++] = ruleArgs[1];
+ }
+ }
+
+ private String getAliasValue(int id) {
+ try {
+ return aliases[id - 1];
+ } catch (IndexOutOfBoundsException ex) {
+ throw new IllegalArgumentException("Bad flag alias number:" + id, ex);
+ }
+ }
+
+ /**
+ * Abstraction of the process of parsing flags taken from the affix and dic files
+ */
+ private static abstract class FlagParsingStrategy {
+
+ /**
+ * Parses the given String into a single flag
+ *
+ * @param rawFlag String to parse into a flag
+ * @return Parsed flag
+ */
+ char parseFlag(String rawFlag) {
+ return parseFlags(rawFlag)[0];
+ }
+
+ /**
+ * Parses the given String into multiple flags
+ *
+ * @param rawFlags String to parse into flags
+ * @return Parsed flags
+ */
+ abstract char[] parseFlags(String rawFlags);
+ }
+
+ /**
+ * Simple implementation of {@link FlagParsingStrategy} that treats the chars in each String as a individual flags.
+ * Can be used with both the ASCII and UTF-8 flag types.
+ */
+ private static class SimpleFlagParsingStrategy extends FlagParsingStrategy {
+ @Override
+ public char[] parseFlags(String rawFlags) {
+ return rawFlags.toCharArray();
+ }
+ }
+
+ /**
+ * Implementation of {@link FlagParsingStrategy} that assumes each flag is encoded in its numerical form. In the case
+ * of multiple flags, each number is separated by a comma.
+ */
+ private static class NumFlagParsingStrategy extends FlagParsingStrategy {
+ @Override
+ public char[] parseFlags(String rawFlags) {
+ String[] rawFlagParts = rawFlags.trim().split(",");
+ char[] flags = new char[rawFlagParts.length];
+ int upto = 0;
+
+ for (int i = 0; i < rawFlagParts.length; i++) {
+ // note, removing the trailing X/leading I for nepali... what is the rule here?!
+ String replacement = rawFlagParts[i].replaceAll("[^0-9]", "");
+ // note, ignoring empty flags (this happens in danish, for example)
+ if (replacement.isEmpty()) {
+ continue;
+ }
+ flags[upto++] = (char) Integer.parseInt(replacement);
+ }
+
+ if (upto < flags.length) {
+ flags = Arrays.copyOf(flags, upto);
+ }
+ return flags;
+ }
+ }
+
+ /**
+ * Implementation of {@link FlagParsingStrategy} that assumes each flag is encoded as two ASCII characters whose codes
+ * must be combined into a single character.
+ *
+ * TODO (rmuir) test
+ */
+ private static class DoubleASCIIFlagParsingStrategy extends FlagParsingStrategy {
+
+ @Override
+ public char[] parseFlags(String rawFlags) {
+ if (rawFlags.length() == 0) {
+ return new char[0];
+ }
+
+ StringBuilder builder = new StringBuilder();
+ for (int i = 0; i < rawFlags.length(); i+=2) {
+ char cookedFlag = (char) ((int) rawFlags.charAt(i) + (int) rawFlags.charAt(i + 1));
+ builder.append(cookedFlag);
+ }
+
+ char flags[] = new char[builder.length()];
+ builder.getChars(0, builder.length(), flags, 0);
+ return flags;
+ }
+ }
+
+ static boolean hasFlag(char flags[], char flag) {
+ return Arrays.binarySearch(flags, flag) >= 0;
+ }
+
+ static char[] merge(char[] flags1, char[] flags2) {
+ char merged[] = new char[flags1.length + flags2.length];
+ int i1 = 0, i2 = 0;
+ int last = -1;
+ int upto = 0;
+
+ while (i1 < flags1.length && i2 < flags2.length) {
+ final char next;
+ if (flags1[i1] <= flags2[i2]) {
+ next = flags1[i1++];
+ } else {
+ next = flags2[i2++];
+ }
+ if (next != last) {
+ merged[upto++] = next;
+ last = next;
+ }
+ }
+
+ while (i1 < flags1.length) {
+ char next = flags1[i1++];
+ if (next != last) {
+ merged[upto++] = next;
+ last = next;
+ }
+ }
+
+ while (i2 < flags2.length) {
+ char next = flags2[i2++];
+ if (next != last) {
+ merged[upto++] = next;
+ last = next;
+ }
+ }
+
+ if (merged.length != upto) {
+ merged = Arrays.copyOf(merged, upto);
+ }
+
+ return merged;
+ }
+}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellAffix.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellAffix.java
deleted file mode 100644
index 97376c0b15e..00000000000
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellAffix.java
+++ /dev/null
@@ -1,157 +0,0 @@
-package org.apache.lucene.analysis.hunspell;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.util.regex.Pattern;
-
-/**
- * Wrapper class representing a hunspell affix
- */
-public class HunspellAffix {
-
- private String append; // the affix itself, what is appended
- private char appendFlags[]; // continuation class flags
- private String strip;
-
- private String condition;
- private Pattern conditionPattern;
-
- private char flag;
-
- private boolean crossProduct;
-
- /**
- * Checks whether the given text matches the conditional pattern on this affix
- *
- * @param text Text to check if it matches the affix's conditional pattern
- * @return {@code true} if the text meets the condition, {@code false} otherwise
- */
- public boolean checkCondition(CharSequence text) {
- return conditionPattern.matcher(text).matches();
- }
-
- /**
- * Returns the append defined for the affix
- *
- * @return Defined append
- */
- public String getAppend() {
- return append;
- }
-
- /**
- * Sets the append defined for the affix
- *
- * @param append Defined append for the affix
- */
- public void setAppend(String append) {
- this.append = append;
- }
-
- /**
- * Returns the flags defined for the affix append
- *
- * @return Flags defined for the affix append
- */
- public char[] getAppendFlags() {
- return appendFlags;
- }
-
- /**
- * Sets the flags defined for the affix append
- *
- * @param appendFlags Flags defined for the affix append
- */
- public void setAppendFlags(char[] appendFlags) {
- this.appendFlags = appendFlags;
- }
-
- /**
- * Returns the stripping characters defined for the affix
- *
- * @return Stripping characters defined for the affix
- */
- public String getStrip() {
- return strip;
- }
-
- /**
- * Sets the stripping characters defined for the affix
- *
- * @param strip Stripping characters defined for the affix
- */
- public void setStrip(String strip) {
- this.strip = strip;
- }
-
- /**
- * Returns the condition that must be met before the affix can be applied
- *
- * @return Condition that must be met before the affix can be applied
- */
- public String getCondition() {
- return condition;
- }
-
- /**
- * Sets the condition that must be met before the affix can be applied
- *
- * @param condition Condition to be met before affix application
- * @param pattern Condition as a regular expression pattern
- */
- public void setCondition(String condition, String pattern) {
- this.condition = condition;
- this.conditionPattern = Pattern.compile(pattern);
- }
-
- /**
- * Returns the affix flag
- *
- * @return Affix flag
- */
- public char getFlag() {
- return flag;
- }
-
- /**
- * Sets the affix flag
- *
- * @param flag Affix flag
- */
- public void setFlag(char flag) {
- this.flag = flag;
- }
-
- /**
- * Returns whether the affix is defined as cross product
- *
- * @return {@code true} if the affix is cross product, {@code false} otherwise
- */
- public boolean isCrossProduct() {
- return crossProduct;
- }
-
- /**
- * Sets whether the affix is defined as cross product
- *
- * @param crossProduct Whether the affix is defined as cross product
- */
- public void setCrossProduct(boolean crossProduct) {
- this.crossProduct = crossProduct;
- }
-}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellDictionary.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellDictionary.java
deleted file mode 100644
index ccb53f57d29..00000000000
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellDictionary.java
+++ /dev/null
@@ -1,507 +0,0 @@
-package org.apache.lucene.analysis.hunspell;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.lucene.analysis.util.CharArrayMap;
-import org.apache.lucene.util.Version;
-
-import java.io.*;
-import java.nio.charset.Charset;
-import java.nio.charset.CharsetDecoder;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Locale;
-
-/**
- * In-memory structure for the dictionary (.dic) and affix (.aff)
- * data of a hunspell dictionary.
- */
-public class HunspellDictionary {
-
- static final HunspellWord NOFLAGS = new HunspellWord();
-
- private static final String ALIAS_KEY = "AF";
- private static final String PREFIX_KEY = "PFX";
- private static final String SUFFIX_KEY = "SFX";
- private static final String FLAG_KEY = "FLAG";
-
- private static final String NUM_FLAG_TYPE = "num";
- private static final String UTF8_FLAG_TYPE = "UTF-8";
- private static final String LONG_FLAG_TYPE = "long";
-
- private static final String PREFIX_CONDITION_REGEX_PATTERN = "%s.*";
- private static final String SUFFIX_CONDITION_REGEX_PATTERN = ".*%s";
-
- private static final boolean IGNORE_CASE_DEFAULT = false;
- private static final boolean STRICT_AFFIX_PARSING_DEFAULT = true;
-
- private CharArrayMap> words;
- private CharArrayMap> prefixes;
- private CharArrayMap> suffixes;
-
- private FlagParsingStrategy flagParsingStrategy = new SimpleFlagParsingStrategy(); // Default flag parsing strategy
- private boolean ignoreCase = IGNORE_CASE_DEFAULT;
-
- private final Version version;
-
- private String[] aliases;
- private int aliasCount = 0;
-
- /**
- * Creates a new HunspellDictionary containing the information read from the provided InputStreams to hunspell affix
- * and dictionary files.
- * You have to close the provided InputStreams yourself.
- *
- * @param affix InputStream for reading the hunspell affix file (won't be closed).
- * @param dictionary InputStream for reading the hunspell dictionary file (won't be closed).
- * @param version Lucene Version
- * @throws IOException Can be thrown while reading from the InputStreams
- * @throws ParseException Can be thrown if the content of the files does not meet expected formats
- */
- public HunspellDictionary(InputStream affix, InputStream dictionary, Version version) throws IOException, ParseException {
- this(affix, Arrays.asList(dictionary), version, IGNORE_CASE_DEFAULT);
- }
-
- /**
- * Creates a new HunspellDictionary containing the information read from the provided InputStreams to hunspell affix
- * and dictionary files.
- * You have to close the provided InputStreams yourself.
- *
- * @param affix InputStream for reading the hunspell affix file (won't be closed).
- * @param dictionary InputStream for reading the hunspell dictionary file (won't be closed).
- * @param version Lucene Version
- * @param ignoreCase If true, dictionary matching will be case insensitive
- * @throws IOException Can be thrown while reading from the InputStreams
- * @throws ParseException Can be thrown if the content of the files does not meet expected formats
- */
- public HunspellDictionary(InputStream affix, InputStream dictionary, Version version, boolean ignoreCase) throws IOException, ParseException {
- this(affix, Arrays.asList(dictionary), version, ignoreCase);
- }
-
- /**
- * Creates a new HunspellDictionary containing the information read from the provided InputStreams to hunspell affix
- * and dictionary files.
- * You have to close the provided InputStreams yourself.
- *
- * @param affix InputStream for reading the hunspell affix file (won't be closed).
- * @param dictionaries InputStreams for reading the hunspell dictionary file (won't be closed).
- * @param version Lucene Version
- * @param ignoreCase If true, dictionary matching will be case insensitive
- * @throws IOException Can be thrown while reading from the InputStreams
- * @throws ParseException Can be thrown if the content of the files does not meet expected formats
- */
- public HunspellDictionary(InputStream affix, List dictionaries, Version version, boolean ignoreCase) throws IOException, ParseException {
- this(affix, dictionaries, version, ignoreCase, STRICT_AFFIX_PARSING_DEFAULT);
- }
-
- /**
- * Creates a new HunspellDictionary containing the information read from the provided InputStreams to hunspell affix
- * and dictionary files.
- * You have to close the provided InputStreams yourself.
- *
- * @param affix InputStream for reading the hunspell affix file (won't be closed).
- * @param dictionaries InputStreams for reading the hunspell dictionary file (won't be closed).
- * @param version Lucene Version
- * @param ignoreCase If true, dictionary matching will be case insensitive
- * @param strictAffixParsing Affix strict parsing enabled or not (an error while reading a rule causes exception or is ignored)
- * @throws IOException Can be thrown while reading from the InputStreams
- * @throws ParseException Can be thrown if the content of the files does not meet expected formats
- */
- public HunspellDictionary(InputStream affix, List dictionaries, Version version, boolean ignoreCase, boolean strictAffixParsing) throws IOException, ParseException {
- this.version = version;
- this.ignoreCase = ignoreCase;
- String encoding = getDictionaryEncoding(affix);
- CharsetDecoder decoder = getJavaEncoding(encoding);
- readAffixFile(affix, decoder, strictAffixParsing);
- words = new CharArrayMap>(version, 65535 /* guess */, this.ignoreCase);
- for (InputStream dictionary : dictionaries) {
- readDictionaryFile(dictionary, decoder);
- }
- }
-
- /**
- * Looks up HunspellWords that match the String created from the given char array, offset and length
- *
- * @param word Char array to generate the String from
- * @param offset Offset in the char array that the String starts at
- * @param length Length from the offset that the String is
- * @return List of HunspellWords that match the generated String, or {@code null} if none are found
- */
- public List lookupWord(char word[], int offset, int length) {
- return words.get(word, offset, length);
- }
-
- /**
- * Looks up HunspellAffix prefixes that have an append that matches the String created from the given char array, offset and length
- *
- * @param word Char array to generate the String from
- * @param offset Offset in the char array that the String starts at
- * @param length Length from the offset that the String is
- * @return List of HunspellAffix prefixes with an append that matches the String, or {@code null} if none are found
- */
- public List lookupPrefix(char word[], int offset, int length) {
- return prefixes.get(word, offset, length);
- }
-
- /**
- * Looks up HunspellAffix suffixes that have an append that matches the String created from the given char array, offset and length
- *
- * @param word Char array to generate the String from
- * @param offset Offset in the char array that the String starts at
- * @param length Length from the offset that the String is
- * @return List of HunspellAffix suffixes with an append that matches the String, or {@code null} if none are found
- */
- public List lookupSuffix(char word[], int offset, int length) {
- return suffixes.get(word, offset, length);
- }
-
- /**
- * Reads the affix file through the provided InputStream, building up the prefix and suffix maps
- *
- * @param affixStream InputStream to read the content of the affix file from
- * @param decoder CharsetDecoder to decode the content of the file
- * @throws IOException Can be thrown while reading from the InputStream
- */
- private void readAffixFile(InputStream affixStream, CharsetDecoder decoder, boolean strict) throws IOException, ParseException {
- prefixes = new CharArrayMap>(version, 8, ignoreCase);
- suffixes = new CharArrayMap>(version, 8, ignoreCase);
-
- LineNumberReader reader = new LineNumberReader(new InputStreamReader(affixStream, decoder));
- String line = null;
- while ((line = reader.readLine()) != null) {
- if (line.startsWith(ALIAS_KEY)) {
- parseAlias(line);
- } else if (line.startsWith(PREFIX_KEY)) {
- parseAffix(prefixes, line, reader, PREFIX_CONDITION_REGEX_PATTERN, strict);
- } else if (line.startsWith(SUFFIX_KEY)) {
- parseAffix(suffixes, line, reader, SUFFIX_CONDITION_REGEX_PATTERN, strict);
- } else if (line.startsWith(FLAG_KEY)) {
- // Assume that the FLAG line comes before any prefix or suffixes
- // Store the strategy so it can be used when parsing the dic file
- flagParsingStrategy = getFlagParsingStrategy(line);
- }
- }
- }
-
- /**
- * Parses a specific affix rule putting the result into the provided affix map
- *
- * @param affixes Map where the result of the parsing will be put
- * @param header Header line of the affix rule
- * @param reader BufferedReader to read the content of the rule from
- * @param conditionPattern {@link String#format(String, Object...)} pattern to be used to generate the condition regex
- * pattern
- * @throws IOException Can be thrown while reading the rule
- */
- private void parseAffix(CharArrayMap> affixes,
- String header,
- LineNumberReader reader,
- String conditionPattern,
- boolean strict) throws IOException, ParseException {
- String args[] = header.split("\\s+");
-
- boolean crossProduct = args[2].equals("Y");
-
- int numLines = Integer.parseInt(args[3]);
- for (int i = 0; i < numLines; i++) {
- String line = reader.readLine();
- String ruleArgs[] = line.split("\\s+");
-
- if (ruleArgs.length < 5) {
- if (strict) {
- throw new ParseException("The affix file contains a rule with less than five elements", reader.getLineNumber());
- }
- continue;
- }
-
- HunspellAffix affix = new HunspellAffix();
-
- affix.setFlag(flagParsingStrategy.parseFlag(ruleArgs[1]));
- affix.setStrip(ruleArgs[2].equals("0") ? "" : ruleArgs[2]);
-
- String affixArg = ruleArgs[3];
-
- int flagSep = affixArg.lastIndexOf('/');
- if (flagSep != -1) {
- String flagPart = affixArg.substring(flagSep + 1);
-
- if (aliasCount > 0) {
- flagPart = getAliasValue(Integer.parseInt(flagPart));
- }
-
- char appendFlags[] = flagParsingStrategy.parseFlags(flagPart);
- Arrays.sort(appendFlags);
- affix.setAppendFlags(appendFlags);
- affix.setAppend(affixArg.substring(0, flagSep));
- } else {
- affix.setAppend(affixArg);
- }
-
- String condition = ruleArgs[4];
- affix.setCondition(condition, String.format(Locale.ROOT, conditionPattern, condition));
- affix.setCrossProduct(crossProduct);
-
- List list = affixes.get(affix.getAppend());
- if (list == null) {
- list = new ArrayList();
- affixes.put(affix.getAppend(), list);
- }
-
- list.add(affix);
- }
- }
-
- /**
- * Parses the encoding specified in the affix file readable through the provided InputStream
- *
- * @param affix InputStream for reading the affix file
- * @return Encoding specified in the affix file
- * @throws IOException Can be thrown while reading from the InputStream
- * @throws ParseException Thrown if the first non-empty non-comment line read from the file does not adhere to the format {@code SET }
- */
- private String getDictionaryEncoding(InputStream affix) throws IOException, ParseException {
- final StringBuilder encoding = new StringBuilder();
- for (;;) {
- encoding.setLength(0);
- int ch;
- while ((ch = affix.read()) >= 0) {
- if (ch == '\n') {
- break;
- }
- if (ch != '\r') {
- encoding.append((char)ch);
- }
- }
- if (
- encoding.length() == 0 || encoding.charAt(0) == '#' ||
- // this test only at the end as ineffective but would allow lines only containing spaces:
- encoding.toString().trim().length() == 0
- ) {
- if (ch < 0) {
- throw new ParseException("Unexpected end of affix file.", 0);
- }
- continue;
- }
- if ("SET ".equals(encoding.substring(0, 4))) {
- // cleanup the encoding string, too (whitespace)
- return encoding.substring(4).trim();
- }
- throw new ParseException("The first non-comment line in the affix file must "+
- "be a 'SET charset', was: '" + encoding +"'", 0);
- }
- }
-
- /**
- * Retrieves the CharsetDecoder for the given encoding. Note, This isn't perfect as I think ISCII-DEVANAGARI and
- * MICROSOFT-CP1251 etc are allowed...
- *
- * @param encoding Encoding to retrieve the CharsetDecoder for
- * @return CharSetDecoder for the given encoding
- */
- private CharsetDecoder getJavaEncoding(String encoding) {
- Charset charset = Charset.forName(encoding);
- return charset.newDecoder();
- }
-
- /**
- * Determines the appropriate {@link FlagParsingStrategy} based on the FLAG definition line taken from the affix file
- *
- * @param flagLine Line containing the flag information
- * @return FlagParsingStrategy that handles parsing flags in the way specified in the FLAG definition
- */
- private FlagParsingStrategy getFlagParsingStrategy(String flagLine) {
- String flagType = flagLine.substring(5);
-
- if (NUM_FLAG_TYPE.equals(flagType)) {
- return new NumFlagParsingStrategy();
- } else if (UTF8_FLAG_TYPE.equals(flagType)) {
- return new SimpleFlagParsingStrategy();
- } else if (LONG_FLAG_TYPE.equals(flagType)) {
- return new DoubleASCIIFlagParsingStrategy();
- }
-
- throw new IllegalArgumentException("Unknown flag type: " + flagType);
- }
-
- /**
- * Reads the dictionary file through the provided InputStream, building up the words map
- *
- * @param dictionary InputStream to read the dictionary file through
- * @param decoder CharsetDecoder used to decode the contents of the file
- * @throws IOException Can be thrown while reading from the file
- */
- private void readDictionaryFile(InputStream dictionary, CharsetDecoder decoder) throws IOException {
- BufferedReader reader = new BufferedReader(new InputStreamReader(dictionary, decoder));
- // TODO: don't create millions of strings.
- String line = reader.readLine(); // first line is number of entries
- int numEntries = Integer.parseInt(line);
-
- // TODO: the flags themselves can be double-chars (long) or also numeric
- // either way the trick is to encode them as char... but they must be parsed differently
- while ((line = reader.readLine()) != null) {
- String entry;
- HunspellWord wordForm;
-
- int flagSep = line.lastIndexOf('/');
- if (flagSep == -1) {
- wordForm = NOFLAGS;
- entry = line;
- } else {
- // note, there can be comments (morph description) after a flag.
- // we should really look for any whitespace
- int end = line.indexOf('\t', flagSep);
- if (end == -1)
- end = line.length();
-
- String flagPart = line.substring(flagSep + 1, end);
- if (aliasCount > 0) {
- flagPart = getAliasValue(Integer.parseInt(flagPart));
- }
-
- wordForm = new HunspellWord(flagParsingStrategy.parseFlags(flagPart));
- Arrays.sort(wordForm.getFlags());
- entry = line.substring(0, flagSep);
- }
- if(ignoreCase) {
- entry = entry.toLowerCase(Locale.ROOT);
- }
-
- List entries = new ArrayList();
- entries.add(wordForm);
- words.put(entry, entries);
- }
- }
-
- public Version getVersion() {
- return version;
- }
-
- private void parseAlias(String line) {
- String ruleArgs[] = line.split("\\s+");
- if (aliases == null) {
- //first line should be the aliases count
- final int count = Integer.parseInt(ruleArgs[1]);
- aliases = new String[count];
- } else {
- aliases[aliasCount++] = ruleArgs[1];
- }
- }
-
- private String getAliasValue(int id) {
- try {
- return aliases[id - 1];
- } catch (IndexOutOfBoundsException ex) {
- throw new IllegalArgumentException("Bad flag alias number:" + id, ex);
- }
- }
-
- /**
- * Abstraction of the process of parsing flags taken from the affix and dic files
- */
- private static abstract class FlagParsingStrategy {
-
- /**
- * Parses the given String into a single flag
- *
- * @param rawFlag String to parse into a flag
- * @return Parsed flag
- */
- char parseFlag(String rawFlag) {
- return parseFlags(rawFlag)[0];
- }
-
- /**
- * Parses the given String into multiple flags
- *
- * @param rawFlags String to parse into flags
- * @return Parsed flags
- */
- abstract char[] parseFlags(String rawFlags);
- }
-
- /**
- * Simple implementation of {@link FlagParsingStrategy} that treats the chars in each String as a individual flags.
- * Can be used with both the ASCII and UTF-8 flag types.
- */
- private static class SimpleFlagParsingStrategy extends FlagParsingStrategy {
- /**
- * {@inheritDoc}
- */
- @Override
- public char[] parseFlags(String rawFlags) {
- return rawFlags.toCharArray();
- }
- }
-
- /**
- * Implementation of {@link FlagParsingStrategy} that assumes each flag is encoded in its numerical form. In the case
- * of multiple flags, each number is separated by a comma.
- */
- private static class NumFlagParsingStrategy extends FlagParsingStrategy {
- /**
- * {@inheritDoc}
- */
- @Override
- public char[] parseFlags(String rawFlags) {
- String[] rawFlagParts = rawFlags.trim().split(",");
- char[] flags = new char[rawFlagParts.length];
-
- for (int i = 0; i < rawFlagParts.length; i++) {
- // note, removing the trailing X/leading I for nepali... what is the rule here?!
- flags[i] = (char) Integer.parseInt(rawFlagParts[i].replaceAll("[^0-9]", ""));
- }
-
- return flags;
- }
- }
-
- /**
- * Implementation of {@link FlagParsingStrategy} that assumes each flag is encoded as two ASCII characters whose codes
- * must be combined into a single character.
- *
- * TODO (rmuir) test
- */
- private static class DoubleASCIIFlagParsingStrategy extends FlagParsingStrategy {
-
- /**
- * {@inheritDoc}
- */
- @Override
- public char[] parseFlags(String rawFlags) {
- if (rawFlags.length() == 0) {
- return new char[0];
- }
-
- StringBuilder builder = new StringBuilder();
- for (int i = 0; i < rawFlags.length(); i+=2) {
- char cookedFlag = (char) ((int) rawFlags.charAt(i) + (int) rawFlags.charAt(i + 1));
- builder.append(cookedFlag);
- }
-
- char flags[] = new char[builder.length()];
- builder.getChars(0, builder.length(), flags, 0);
- return flags;
- }
- }
-
- public boolean isIgnoreCase() {
- return ignoreCase;
- }
-}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemFilter.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemFilter.java
index 4ff0a741ad8..87de53aee63 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemFilter.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemFilter.java
@@ -18,14 +18,16 @@ package org.apache.lucene.analysis.hunspell;
*/
import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
import java.util.List;
import org.apache.lucene.analysis.TokenFilter;
import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.hunspell.HunspellStemmer.Stem;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.KeywordAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.util.CharsRef;
/**
* TokenFilter that uses hunspell affix rules and words to stem tokens. Since hunspell supports a word having multiple
@@ -41,71 +43,83 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
* {@link org.apache.lucene.analysis.miscellaneous.KeywordRepeatFilterFactory}
*
*
- *
+ * @lucene.experimental
*/
public final class HunspellStemFilter extends TokenFilter {
private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
private final KeywordAttribute keywordAtt = addAttribute(KeywordAttribute.class);
- private final HunspellStemmer stemmer;
+ private final Stemmer stemmer;
- private List buffer;
+ private List buffer;
private State savedState;
private final boolean dedup;
+ private final boolean longestOnly;
/** Create a {@link HunspellStemFilter} which deduplicates stems and has a maximum
* recursion level of 2.
- * @see #HunspellStemFilter(TokenStream, HunspellDictionary, int) */
- public HunspellStemFilter(TokenStream input, HunspellDictionary dictionary) {
+ * @see #HunspellStemFilter(TokenStream, Dictionary, int) */
+ public HunspellStemFilter(TokenStream input, Dictionary dictionary) {
this(input, dictionary, 2);
}
/**
* Creates a new HunspellStemFilter that will stem tokens from the given TokenStream using affix rules in the provided
- * HunspellDictionary
+ * Dictionary
*
* @param input TokenStream whose tokens will be stemmed
* @param dictionary HunspellDictionary containing the affix rules and words that will be used to stem the tokens
* @param recursionCap maximum level of recursion stemmer can go into, defaults to 2
*/
- public HunspellStemFilter(TokenStream input, HunspellDictionary dictionary, int recursionCap) {
+ public HunspellStemFilter(TokenStream input, Dictionary dictionary, int recursionCap) {
this(input, dictionary, true, recursionCap);
}
/** Create a {@link HunspellStemFilter} which has a maximum recursion level of 2.
- * @see #HunspellStemFilter(TokenStream, HunspellDictionary, boolean, int) */
- public HunspellStemFilter(TokenStream input, HunspellDictionary dictionary, boolean dedup) {
+ * @see #HunspellStemFilter(TokenStream, Dictionary, boolean, int) */
+ public HunspellStemFilter(TokenStream input, Dictionary dictionary, boolean dedup) {
this(input, dictionary, dedup, 2);
}
-
+
/**
* Creates a new HunspellStemFilter that will stem tokens from the given TokenStream using affix rules in the provided
- * HunspellDictionary
+ * Dictionary
*
* @param input TokenStream whose tokens will be stemmed
* @param dictionary HunspellDictionary containing the affix rules and words that will be used to stem the tokens
* @param dedup true if only unique terms should be output.
* @param recursionCap maximum level of recursion stemmer can go into, defaults to 2
*/
- public HunspellStemFilter(TokenStream input, HunspellDictionary dictionary, boolean dedup, int recursionCap) {
- super(input);
- this.dedup = dedup;
- this.stemmer = new HunspellStemmer(dictionary, recursionCap);
+ public HunspellStemFilter(TokenStream input, Dictionary dictionary, boolean dedup, int recursionCap) {
+ this(input, dictionary, dedup, recursionCap, false);
}
/**
- * {@inheritDoc}
+ * Creates a new HunspellStemFilter that will stem tokens from the given TokenStream using affix rules in the provided
+ * Dictionary
+ *
+ * @param input TokenStream whose tokens will be stemmed
+ * @param dictionary HunspellDictionary containing the affix rules and words that will be used to stem the tokens
+ * @param dedup true if only unique terms should be output.
+ * @param recursionCap maximum level of recursion stemmer can go into, defaults to 2
+ * @param longestOnly true if only the longest term should be output.
*/
+ public HunspellStemFilter(TokenStream input, Dictionary dictionary, boolean dedup, int recursionCap, boolean longestOnly) {
+ super(input);
+ this.dedup = dedup && longestOnly == false; // don't waste time deduping if longestOnly is set
+ this.stemmer = new Stemmer(dictionary, recursionCap);
+ this.longestOnly = longestOnly;
+ }
+
@Override
public boolean incrementToken() throws IOException {
if (buffer != null && !buffer.isEmpty()) {
- Stem nextStem = buffer.remove(0);
+ CharsRef nextStem = buffer.remove(0);
restoreState(savedState);
posIncAtt.setPositionIncrement(0);
- termAtt.copyBuffer(nextStem.getStem(), 0, nextStem.getStemLength());
- termAtt.setLength(nextStem.getStemLength());
+ termAtt.setEmpty().append(nextStem);
return true;
}
@@ -122,24 +136,41 @@ public final class HunspellStemFilter extends TokenFilter {
if (buffer.isEmpty()) { // we do not know this word, return it unchanged
return true;
}
+
+ if (longestOnly && buffer.size() > 1) {
+ Collections.sort(buffer, lengthComparator);
+ }
- Stem stem = buffer.remove(0);
- termAtt.copyBuffer(stem.getStem(), 0, stem.getStemLength());
- termAtt.setLength(stem.getStemLength());
+ CharsRef stem = buffer.remove(0);
+ termAtt.setEmpty().append(stem);
- if (!buffer.isEmpty()) {
- savedState = captureState();
+ if (longestOnly) {
+ buffer.clear();
+ } else {
+ if (!buffer.isEmpty()) {
+ savedState = captureState();
+ }
}
return true;
}
- /**
- * {@inheritDoc}
- */
@Override
public void reset() throws IOException {
super.reset();
buffer = null;
}
+
+ static final Comparator lengthComparator = new Comparator() {
+ @Override
+ public int compare(CharsRef o1, CharsRef o2) {
+ int cmp = Integer.compare(o2.length, o1.length);
+ if (cmp == 0) {
+ // tie break on text
+ return o2.compareTo(o1);
+ } else {
+ return cmp;
+ }
+ }
+ };
}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemFilterFactory.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemFilterFactory.java
index 63e621c2ab9..e632b489d51 100644
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemFilterFactory.java
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemFilterFactory.java
@@ -31,89 +31,75 @@ import org.apache.lucene.analysis.util.TokenFilterFactory;
import org.apache.lucene.util.IOUtils;
/**
- * TokenFilterFactory that creates instances of {@link org.apache.lucene.analysis.hunspell.HunspellStemFilter}.
- * Example config for British English including a custom dictionary, case insensitive matching:
+ * TokenFilterFactory that creates instances of {@link HunspellStemFilter}.
+ * Example config for British English:
*
* <filter class="solr.HunspellStemFilterFactory"
- * dictionary="en_GB.dic,my_custom.dic"
- * affix="en_GB.aff"
- * ignoreCase="true" />
+ * dictionary="en_GB.dic,my_custom.dic"
+ * affix="en_GB.aff"
+ * ignoreCase="false"
+ * longestOnly="false" />
* Both parameters dictionary and affix are mandatory.
- *
- * The parameter ignoreCase (true/false) controls whether matching is case sensitive or not. Default false.
- *
- * The parameter strictAffixParsing (true/false) controls whether the affix parsing is strict or not. Default true.
- * If strict an error while reading an affix rule causes a ParseException, otherwise is ignored.
- *
* Dictionaries for many languages are available through the OpenOffice project.
*
* See http://wiki.apache.org/solr/Hunspell
+ * @lucene.experimental
*/
public class HunspellStemFilterFactory extends TokenFilterFactory implements ResourceLoaderAware {
- private static final String PARAM_DICTIONARY = "dictionary";
- private static final String PARAM_AFFIX = "affix";
- private static final String PARAM_IGNORE_CASE = "ignoreCase";
- private static final String PARAM_STRICT_AFFIX_PARSING = "strictAffixParsing";
+ private static final String PARAM_DICTIONARY = "dictionary";
+ private static final String PARAM_AFFIX = "affix";
private static final String PARAM_RECURSION_CAP = "recursionCap";
+ private static final String PARAM_IGNORE_CASE = "ignoreCase";
+ private static final String PARAM_LONGEST_ONLY = "longestOnly";
- private final String dictionaryArg;
+ private final String dictionaryFiles;
private final String affixFile;
private final boolean ignoreCase;
- private final boolean strictAffixParsing;
- private HunspellDictionary dictionary;
+ private final boolean longestOnly;
+ private Dictionary dictionary;
private int recursionCap;
/** Creates a new HunspellStemFilterFactory */
public HunspellStemFilterFactory(Map args) {
super(args);
- assureMatchVersion();
- dictionaryArg = require(args, PARAM_DICTIONARY);
+ dictionaryFiles = require(args, PARAM_DICTIONARY);
affixFile = get(args, PARAM_AFFIX);
ignoreCase = getBoolean(args, PARAM_IGNORE_CASE, false);
- strictAffixParsing = getBoolean(args, PARAM_STRICT_AFFIX_PARSING, true);
recursionCap = getInt(args, PARAM_RECURSION_CAP, 2);
+ longestOnly = getBoolean(args, PARAM_LONGEST_ONLY, false);
+ // this isnt necessary: we properly load all dictionaries.
+ // but recognize and ignore for back compat
+ getBoolean(args, "strictAffixParsing", true);
if (!args.isEmpty()) {
throw new IllegalArgumentException("Unknown parameters: " + args);
}
}
- /**
- * Loads the hunspell dictionary and affix files defined in the configuration
- *
- * @param loader ResourceLoader used to load the files
- */
@Override
public void inform(ResourceLoader loader) throws IOException {
- String dictionaryFiles[] = dictionaryArg.split(",");
+ String dicts[] = dictionaryFiles.split(",");
InputStream affix = null;
List dictionaries = new ArrayList();
try {
dictionaries = new ArrayList();
- for (String file : dictionaryFiles) {
+ for (String file : dicts) {
dictionaries.add(loader.openResource(file));
}
affix = loader.openResource(affixFile);
- this.dictionary = new HunspellDictionary(affix, dictionaries, luceneMatchVersion, ignoreCase, strictAffixParsing);
+ this.dictionary = new Dictionary(affix, dictionaries, ignoreCase);
} catch (ParseException e) {
- throw new IOException("Unable to load hunspell data! [dictionary=" + dictionaryArg + ",affix=" + affixFile + "]", e);
+ throw new IOException("Unable to load hunspell data! [dictionary=" + dictionaries + ",affix=" + affixFile + "]", e);
} finally {
IOUtils.closeWhileHandlingException(affix);
IOUtils.closeWhileHandlingException(dictionaries);
}
}
- /**
- * Creates an instance of {@link org.apache.lucene.analysis.hunspell.HunspellStemFilter} that will filter the given
- * TokenStream
- *
- * @param tokenStream TokenStream that will be filtered
- * @return HunspellStemFilter that filters the TokenStream
- */
@Override
public TokenStream create(TokenStream tokenStream) {
- return new HunspellStemFilter(tokenStream, dictionary, recursionCap);
+ return new HunspellStemFilter(tokenStream, dictionary, true, recursionCap, longestOnly);
}
}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemmer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemmer.java
deleted file mode 100644
index ae2948284d6..00000000000
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellStemmer.java
+++ /dev/null
@@ -1,392 +0,0 @@
-package org.apache.lucene.analysis.hunspell;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.lucene.analysis.util.CharArraySet;
-import org.apache.lucene.analysis.util.CharacterUtils;
-import org.apache.lucene.util.Version;
-
-/**
- * HunspellStemmer uses the affix rules declared in the HunspellDictionary to generate one or more stems for a word. It
- * conforms to the algorithm in the original hunspell algorithm, including recursive suffix stripping.
- */
-public class HunspellStemmer {
- private final int recursionCap;
- private final HunspellDictionary dictionary;
- private final StringBuilder segment = new StringBuilder();
- private CharacterUtils charUtils = CharacterUtils.getInstance(Version.LUCENE_CURRENT);
-
- /**
- * Constructs a new HunspellStemmer which will use the provided HunspellDictionary to create its stems. Uses the
- * default recursion cap of 2
(based on Hunspell documentation).
- *
- * @param dictionary HunspellDictionary that will be used to create the stems
- */
- public HunspellStemmer(HunspellDictionary dictionary) {
- this(dictionary, 2);
- }
-
- /**
- * Constructs a new HunspellStemmer which will use the provided HunspellDictionary to create its stems
- *
- * @param dictionary HunspellDictionary that will be used to create the stems
- * @param recursionCap maximum level of recursion stemmer can go into
- */
- public HunspellStemmer(HunspellDictionary dictionary, int recursionCap) {
- this.dictionary = dictionary;
- this.recursionCap = recursionCap;
- }
-
- /**
- * Find the stem(s) of the provided word
- *
- * @param word Word to find the stems for
- * @return List of stems for the word
- */
- public List stem(String word) {
- return stem(word.toCharArray(), word.length());
- }
-
- /**
- * Find the stem(s) of the provided word
- *
- * @param word Word to find the stems for
- * @return List of stems for the word
- */
- public List stem(char word[], int length) {
- List stems = new ArrayList();
- if (dictionary.lookupWord(word, 0, length) != null) {
- stems.add(new Stem(word, length));
- }
- stems.addAll(stem(word, length, null, 0));
- return stems;
- }
-
- /**
- * Find the unique stem(s) of the provided word
- *
- * @param word Word to find the stems for
- * @return List of stems for the word
- */
- public List uniqueStems(char word[], int length) {
- List stems = new ArrayList();
- CharArraySet terms = new CharArraySet(dictionary.getVersion(), 8, dictionary.isIgnoreCase());
- if (dictionary.lookupWord(word, 0, length) != null) {
- stems.add(new Stem(word, length));
- terms.add(word);
- }
- List otherStems = stem(word, length, null, 0);
- for (Stem s : otherStems) {
- if (!terms.contains(s.stem)) {
- stems.add(s);
- terms.add(s.stem);
- }
- }
- return stems;
- }
-
- // ================================================= Helper Methods ================================================
-
- /**
- * Generates a list of stems for the provided word
- *
- * @param word Word to generate the stems for
- * @param flags Flags from a previous stemming step that need to be cross-checked with any affixes in this recursive step
- * @param recursionDepth Level of recursion this stemming step is at
- * @return List of stems, pr an empty if no stems are found
- */
- private List stem(char word[], int length, char[] flags, int recursionDepth) {
- List stems = new ArrayList();
-
- for (int i = 0; i < length; i++) {
- List suffixes = dictionary.lookupSuffix(word, i, length - i);
- if (suffixes == null) {
- continue;
- }
-
- for (HunspellAffix suffix : suffixes) {
- if (hasCrossCheckedFlag(suffix.getFlag(), flags)) {
- int deAffixedLength = length - suffix.getAppend().length();
- // TODO: can we do this in-place?
- String strippedWord = new StringBuilder().append(word, 0, deAffixedLength).append(suffix.getStrip()).toString();
-
- List stemList = applyAffix(strippedWord.toCharArray(), strippedWord.length(), suffix, recursionDepth);
- for (Stem stem : stemList) {
- stem.addSuffix(suffix);
- }
-
- stems.addAll(stemList);
- }
- }
- }
-
- for (int i = length - 1; i >= 0; i--) {
- List prefixes = dictionary.lookupPrefix(word, 0, i);
- if (prefixes == null) {
- continue;
- }
-
- for (HunspellAffix prefix : prefixes) {
- if (hasCrossCheckedFlag(prefix.getFlag(), flags)) {
- int deAffixedStart = prefix.getAppend().length();
- int deAffixedLength = length - deAffixedStart;
-
- String strippedWord = new StringBuilder().append(prefix.getStrip())
- .append(word, deAffixedStart, deAffixedLength)
- .toString();
-
- List stemList = applyAffix(strippedWord.toCharArray(), strippedWord.length(), prefix, recursionDepth);
- for (Stem stem : stemList) {
- stem.addPrefix(prefix);
- }
-
- stems.addAll(stemList);
- }
- }
- }
-
- return stems;
- }
-
- /**
- * Applies the affix rule to the given word, producing a list of stems if any are found
- *
- * @param strippedWord Word the affix has been removed and the strip added
- * @param affix HunspellAffix representing the affix rule itself
- * @param recursionDepth Level of recursion this stemming step is at
- * @return List of stems for the word, or an empty list if none are found
- */
- @SuppressWarnings("unchecked")
- public List applyAffix(char strippedWord[], int length, HunspellAffix affix, int recursionDepth) {
- if(dictionary.isIgnoreCase()) {
- charUtils.toLowerCase(strippedWord, 0, strippedWord.length);
- }
- segment.setLength(0);
- segment.append(strippedWord, 0, length);
- if (!affix.checkCondition(segment)) {
- return Collections.EMPTY_LIST;
- }
-
- List stems = new ArrayList();
-
- List words = dictionary.lookupWord(strippedWord, 0, length);
- if (words != null) {
- for (HunspellWord hunspellWord : words) {
- if (hunspellWord.hasFlag(affix.getFlag())) {
- stems.add(new Stem(strippedWord, length));
- }
- }
- }
-
- if (affix.isCrossProduct() && recursionDepth < recursionCap) {
- stems.addAll(stem(strippedWord, length, affix.getAppendFlags(), ++recursionDepth));
- }
-
- return stems;
- }
-
- /**
- * Checks if the given flag cross checks with the given array of flags
- *
- * @param flag Flag to cross check with the array of flags
- * @param flags Array of flags to cross check against. Can be {@code null}
- * @return {@code true} if the flag is found in the array or the array is {@code null}, {@code false} otherwise
- */
- private boolean hasCrossCheckedFlag(char flag, char[] flags) {
- return flags == null || Arrays.binarySearch(flags, flag) >= 0;
- }
-
- /**
- * Stem represents all information known about a stem of a word. This includes the stem, and the prefixes and suffixes
- * that were used to change the word into the stem.
- */
- public static class Stem {
-
- private final List prefixes = new ArrayList();
- private final List suffixes = new ArrayList();
- private final char stem[];
- private final int stemLength;
-
- /**
- * Creates a new Stem wrapping the given word stem
- *
- * @param stem Stem of a word
- */
- public Stem(char stem[], int stemLength) {
- this.stem = stem;
- this.stemLength = stemLength;
- }
-
- /**
- * Adds a prefix to the list of prefixes used to generate this stem. Because it is assumed that prefixes are added
- * depth first, the prefix is added to the front of the list
- *
- * @param prefix Prefix to add to the list of prefixes for this stem
- */
- public void addPrefix(HunspellAffix prefix) {
- prefixes.add(0, prefix);
- }
-
- /**
- * Adds a suffix to the list of suffixes used to generate this stem. Because it is assumed that suffixes are added
- * depth first, the suffix is added to the end of the list
- *
- * @param suffix Suffix to add to the list of suffixes for this stem
- */
- public void addSuffix(HunspellAffix suffix) {
- suffixes.add(suffix);
- }
-
- /**
- * Returns the list of prefixes used to generate the stem
- *
- * @return List of prefixes used to generate the stem or an empty list if no prefixes were required
- */
- public List getPrefixes() {
- return prefixes;
- }
-
- /**
- * Returns the list of suffixes used to generate the stem
- *
- * @return List of suffixes used to generate the stem or an empty list if no suffixes were required
- */
- public List getSuffixes() {
- return suffixes;
- }
-
- /**
- * Returns the actual word stem itself
- *
- * @return Word stem itself
- */
- public char[] getStem() {
- return stem;
- }
-
- /**
- * @return the stemLength
- */
- public int getStemLength() {
- return stemLength;
- }
-
- public String getStemString() {
- return new String(stem, 0, stemLength);
- }
-
- }
-
-
- // ================================================= Entry Point ===================================================
-
- /*
- * HunspellStemmer entry point. Accepts two arguments: location of affix file and location of dic file
- *
- * @param args Program arguments. Should contain location of affix file and location of dic file
- * @throws IOException Can be thrown while reading from the files
- * @throws ParseException Can be thrown while parsing the files
- public static void main(String[] args) throws IOException, ParseException {
- boolean ignoreCase = false;
- int offset = 0;
-
- if (args.length < 2) {
- System.out.println("usage: HunspellStemmer [-i] ");
- System.exit(1);
- }
-
- if(args[offset].equals("-i")) {
- ignoreCase = true;
- System.out.println("Ignoring case. All stems will be returned lowercased");
- offset++;
- }
-
- InputStream affixInputStream = new FileInputStream(args[offset++]);
- InputStream dicInputStream = new FileInputStream(args[offset++]);
-
- // :Post-Release-Update-Version.LUCENE_XY:
- HunspellDictionary dictionary = new HunspellDictionary(affixInputStream, dicInputStream, Version.LUCENE_50, ignoreCase);
-
- affixInputStream.close();
- dicInputStream.close();
-
- HunspellStemmer stemmer = new HunspellStemmer(dictionary);
-
- Scanner scanner = new Scanner(System.in, Charset.defaultCharset().name());
-
- System.out.print("> ");
- while (scanner.hasNextLine()) {
- String word = scanner.nextLine();
-
- if ("exit".equals(word)) {
- break;
- }
-
- printStemResults(word, stemmer.stem(word.toCharArray(), word.length()));
-
- System.out.print("> ");
- }
- }
-
- * Prints the results of the stemming of a word
- *
- * @param originalWord Word that has been stemmed
- * @param stems Stems of the word
- private static void printStemResults(String originalWord, List stems) {
- StringBuilder builder = new StringBuilder().append("stem(").append(originalWord).append(")").append("\n");
-
- for (Stem stem : stems) {
- builder.append("- ").append(stem.getStem()).append(": ");
-
- for (HunspellAffix prefix : stem.getPrefixes()) {
- builder.append(prefix.getAppend()).append("+");
-
- if (hasText(prefix.getStrip())) {
- builder.append(prefix.getStrip()).append("-");
- }
- }
-
- builder.append(stem.getStem());
-
- for (HunspellAffix suffix : stem.getSuffixes()) {
- if (hasText(suffix.getStrip())) {
- builder.append("-").append(suffix.getStrip());
- }
-
- builder.append("+").append(suffix.getAppend());
- }
- builder.append("\n");
- }
-
- System.out.println(builder);
- }
-
- * Simple utility to check if the given String has any text
- *
- * @param str String to check if it has any text
- * @return {@code true} if the String has text, {@code false} otherwise
- private static boolean hasText(String str) {
- return str != null && str.length() > 0;
- }
- */
-}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellWord.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellWord.java
deleted file mode 100644
index fe216d30dc8..00000000000
--- a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/HunspellWord.java
+++ /dev/null
@@ -1,63 +0,0 @@
-package org.apache.lucene.analysis.hunspell;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.util.Arrays;
-
-/**
- * A dictionary (.dic) entry with its associated flags.
- */
-public class HunspellWord {
-
- private final char flags[]; // sorted, can we represent more concisely?
-
- /**
- * Creates a new HunspellWord with no associated flags
- */
- public HunspellWord() {
- flags = null;
- }
-
- /**
- * Constructs a new HunspellWord with the given flags
- *
- * @param flags Flags to associate with the word
- */
- public HunspellWord(char[] flags) {
- this.flags = flags;
- }
-
- /**
- * Checks whether the word has the given flag associated with it
- *
- * @param flag Flag to check whether it is associated with the word
- * @return {@code true} if the flag is associated, {@code false} otherwise
- */
- public boolean hasFlag(char flag) {
- return flags != null && Arrays.binarySearch(flags, flag) >= 0;
- }
-
- /**
- * Returns the flags associated with the word
- *
- * @return Flags associated with the word
- */
- public char[] getFlags() {
- return flags;
- }
-}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ISO8859_14Decoder.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ISO8859_14Decoder.java
new file mode 100644
index 00000000000..2d87947ab3d
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/ISO8859_14Decoder.java
@@ -0,0 +1,60 @@
+package org.apache.lucene.analysis.hunspell;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CoderResult;
+
+import org.apache.lucene.util.IOUtils;
+
+// many hunspell dictionaries use this encoding, yet java does not have it?!?!
+final class ISO8859_14Decoder extends CharsetDecoder {
+
+ static final char TABLE[] = new char[] {
+ 0x00A0, 0x1E02, 0x1E03, 0x00A3, 0x010A, 0x010B, 0x1E0A, 0x00A7,
+ 0x1E80, 0x00A9, 0x1E82, 0x1E0B, 0x1EF2, 0x00AD, 0x00AE, 0x0178,
+ 0x1E1E, 0x1E1F, 0x0120, 0x0121, 0x1E40, 0x1E41, 0x00B6, 0x1E56,
+ 0x1E81, 0x1E57, 0x1E83, 0x1E60, 0x1EF3, 0x1E84, 0x1E85, 0x1E61,
+ 0x00C0, 0x00C1, 0x00C2, 0x00C3, 0x00C4, 0x00C5, 0x00C6, 0x00C7,
+ 0x00C8, 0x00C9, 0x00CA, 0x00CB, 0x00CC, 0x00CD, 0x00CE, 0x00CF,
+ 0x0174, 0x00D1, 0x00D2, 0x00D3, 0x00D4, 0x00D5, 0x00D6, 0x1E6A,
+ 0x00D8, 0x00D9, 0x00DA, 0x00DB, 0x00DC, 0x00DD, 0x0176, 0x00DF,
+ 0x00E0, 0x00E1, 0x00E2, 0x00E3, 0x00E4, 0x00E5, 0x00E6, 0x00E7,
+ 0x00E8, 0x00E9, 0x00EA, 0x00EB, 0x00EC, 0x00ED, 0x00EE, 0x00EF,
+ 0x0175, 0x00F1, 0x00F2, 0x00F3, 0x00F4, 0x00F5, 0x00F6, 0x1E6B,
+ 0x00F8, 0x00F9, 0x00FA, 0x00FB, 0x00FC, 0x00FD, 0x0177, 0x00FF
+ };
+
+ ISO8859_14Decoder() {
+ super(IOUtils.CHARSET_UTF_8, 1f, 1f);
+ }
+
+ @Override
+ protected CoderResult decodeLoop(ByteBuffer in, CharBuffer out) {
+ while (in.hasRemaining() && out.hasRemaining()) {
+ char ch = (char) (in.get() & 0xff);
+ if (ch >= 0xA0) {
+ ch = TABLE[ch - 0xA0];
+ }
+ out.put(ch);
+ }
+ return in.hasRemaining() ? CoderResult.OVERFLOW : CoderResult.UNDERFLOW;
+ }
+}
diff --git a/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Stemmer.java b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Stemmer.java
new file mode 100644
index 00000000000..18e6588ce7a
--- /dev/null
+++ b/lucene/analysis/common/src/java/org/apache/lucene/analysis/hunspell/Stemmer.java
@@ -0,0 +1,238 @@
+package org.apache.lucene.analysis.hunspell;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.regex.Pattern;
+
+import org.apache.lucene.analysis.util.CharArraySet;
+import org.apache.lucene.analysis.util.CharacterUtils;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.Version;
+
+/**
+ * Stemmer uses the affix rules declared in the Dictionary to generate one or more stems for a word. It
+ * conforms to the algorithm in the original hunspell algorithm, including recursive suffix stripping.
+ */
+final class Stemmer {
+ private final int recursionCap;
+ private final Dictionary dictionary;
+ private final BytesRef scratch = new BytesRef();
+ private final StringBuilder segment = new StringBuilder();
+ private final ByteArrayDataInput affixReader;
+ private final CharacterUtils charUtils = CharacterUtils.getInstance(Version.LUCENE_CURRENT);
+
+ /**
+ * Constructs a new Stemmer which will use the provided Dictionary to create its stems. Uses the
+ * default recursion cap of 2
(based on Hunspell documentation).
+ *
+ * @param dictionary Dictionary that will be used to create the stems
+ */
+ public Stemmer(Dictionary dictionary) {
+ this(dictionary, 2);
+ }
+
+ /**
+ * Constructs a new Stemmer which will use the provided Dictionary to create its stems.
+ *
+ * @param dictionary Dictionary that will be used to create the stems
+ * @param recursionCap maximum level of recursion stemmer can go into
+ */
+ public Stemmer(Dictionary dictionary, int recursionCap) {
+ this.dictionary = dictionary;
+ this.affixReader = new ByteArrayDataInput(dictionary.affixData);
+ this.recursionCap = recursionCap;
+ }
+
+ /**
+ * Find the stem(s) of the provided word.
+ *
+ * @param word Word to find the stems for
+ * @return List of stems for the word
+ */
+ public List stem(String word) {
+ return stem(word.toCharArray(), word.length());
+ }
+
+ /**
+ * Find the stem(s) of the provided word
+ *
+ * @param word Word to find the stems for
+ * @return List of stems for the word
+ */
+ public List stem(char word[], int length) {
+ if (dictionary.ignoreCase) {
+ charUtils.toLowerCase(word, 0, length);
+ }
+ List stems = new ArrayList();
+ if (dictionary.lookupWord(word, 0, length, scratch) != null) {
+ stems.add(new CharsRef(word, 0, length));
+ }
+ stems.addAll(stem(word, length, Dictionary.NOFLAGS, 0));
+ return stems;
+ }
+
+ /**
+ * Find the unique stem(s) of the provided word
+ *
+ * @param word Word to find the stems for
+ * @return List of stems for the word
+ */
+ public List uniqueStems(char word[], int length) {
+ List stems = stem(word, length);
+ if (stems.size() < 2) {
+ return stems;
+ }
+ CharArraySet terms = new CharArraySet(Version.LUCENE_CURRENT, 8, dictionary.ignoreCase);
+ List deduped = new ArrayList<>();
+ for (CharsRef s : stems) {
+ if (!terms.contains(s)) {
+ deduped.add(s);
+ terms.add(s);
+ }
+ }
+ return deduped;
+ }
+
+ // ================================================= Helper Methods ================================================
+
+ /**
+ * Generates a list of stems for the provided word
+ *
+ * @param word Word to generate the stems for
+ * @param flags Flags from a previous stemming step that need to be cross-checked with any affixes in this recursive step
+ * @param recursionDepth Level of recursion this stemming step is at
+ * @return List of stems, or empty list if no stems are found
+ */
+ private List stem(char word[], int length, char[] flags, int recursionDepth) {
+ // TODO: allow this stuff to be reused by tokenfilter
+ List stems = new ArrayList();
+
+ for (int i = 0; i < length; i++) {
+ IntsRef suffixes = dictionary.lookupSuffix(word, i, length - i);
+ if (suffixes == null) {
+ continue;
+ }
+
+ for (int j = 0; j < suffixes.length; j++) {
+ int suffix = suffixes.ints[suffixes.offset + j];
+ affixReader.setPosition(8 * suffix);
+ char flag = (char) (affixReader.readShort() & 0xffff);
+ if (hasCrossCheckedFlag(flag, flags)) {
+ int appendLength = length - i;
+ int deAffixedLength = length - appendLength;
+ // TODO: can we do this in-place?
+ char stripOrd = (char) (affixReader.readShort() & 0xffff);
+ dictionary.stripLookup.get(stripOrd, scratch);
+ String strippedWord = new StringBuilder().append(word, 0, deAffixedLength).append(scratch.utf8ToString()).toString();
+
+ List stemList = applyAffix(strippedWord.toCharArray(), strippedWord.length(), suffix, recursionDepth);
+
+ stems.addAll(stemList);
+ }
+ }
+ }
+
+ for (int i = length - 1; i >= 0; i--) {
+ IntsRef prefixes = dictionary.lookupPrefix(word, 0, i);
+ if (prefixes == null) {
+ continue;
+ }
+
+ for (int j = 0; j < prefixes.length; j++) {
+ int prefix = prefixes.ints[prefixes.offset + j];
+ affixReader.setPosition(8 * prefix);
+ char flag = (char) (affixReader.readShort() & 0xffff);
+ if (hasCrossCheckedFlag(flag, flags)) {
+ int deAffixedStart = i;
+ int deAffixedLength = length - deAffixedStart;
+ char stripOrd = (char) (affixReader.readShort() & 0xffff);
+
+ dictionary.stripLookup.get(stripOrd, scratch);
+ String strippedWord = new StringBuilder().append(scratch.utf8ToString())
+ .append(word, deAffixedStart, deAffixedLength)
+ .toString();
+
+ List stemList = applyAffix(strippedWord.toCharArray(), strippedWord.length(), prefix, recursionDepth);
+
+ stems.addAll(stemList);
+ }
+ }
+ }
+
+ return stems;
+ }
+
+ /**
+ * Applies the affix rule to the given word, producing a list of stems if any are found
+ *
+ * @param strippedWord Word the affix has been removed and the strip added
+ * @param affix HunspellAffix representing the affix rule itself
+ * @param recursionDepth Level of recursion this stemming step is at
+ * @return List of stems for the word, or an empty list if none are found
+ */
+ List applyAffix(char strippedWord[], int length, int affix, int recursionDepth) {
+ segment.setLength(0);
+ segment.append(strippedWord, 0, length);
+
+ affixReader.setPosition(8 * affix);
+ char flag = (char) (affixReader.readShort() & 0xffff);
+ affixReader.skipBytes(2); // strip
+ int condition = (char) (affixReader.readShort() & 0xffff);
+ boolean crossProduct = (condition & 1) == 1;
+ condition >>>= 1;
+ char append = (char) (affixReader.readShort() & 0xffff);
+
+ Pattern pattern = dictionary.patterns.get(condition);
+ if (!pattern.matcher(segment).matches()) {
+ return Collections.emptyList();
+ }
+
+ List stems = new ArrayList();
+
+ char wordFlags[] = dictionary.lookupWord(strippedWord, 0, length, scratch);
+ if (wordFlags != null && Dictionary.hasFlag(wordFlags, flag)) {
+ stems.add(new CharsRef(strippedWord, 0, length));
+ }
+
+ if (crossProduct && recursionDepth < recursionCap) {
+ dictionary.flagLookup.get(append, scratch);
+ char appendFlags[] = Dictionary.decodeFlags(scratch);
+ stems.addAll(stem(strippedWord, length, appendFlags, ++recursionDepth));
+ }
+
+ return stems;
+ }
+
+ /**
+ * Checks if the given flag cross checks with the given array of flags
+ *
+ * @param flag Flag to cross check with the array of flags
+ * @param flags Array of flags to cross check against. Can be {@code null}
+ * @return {@code true} if the flag is found in the array or the array is {@code null}, {@code false} otherwise
+ */
+ private boolean hasCrossCheckedFlag(char flag, char[] flags) {
+ return flags.length == 0 || Arrays.binarySearch(flags, flag) >= 0;
+ }
+}
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
index bca5e1ede50..617e7523b69 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/core/TestRandomChains.java
@@ -62,8 +62,8 @@ import org.apache.lucene.analysis.commongrams.CommonGramsQueryFilter;
import org.apache.lucene.analysis.compound.HyphenationCompoundWordTokenFilter;
import org.apache.lucene.analysis.compound.TestCompoundWordTokenFilter;
import org.apache.lucene.analysis.compound.hyphenation.HyphenationTree;
-import org.apache.lucene.analysis.hunspell.HunspellDictionary;
-import org.apache.lucene.analysis.hunspell.HunspellDictionaryTest;
+import org.apache.lucene.analysis.hunspell.Dictionary;
+import org.apache.lucene.analysis.hunspell.TestHunspellStemFilter;
import org.apache.lucene.analysis.miscellaneous.HyphenatedWordsFilter;
import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
import org.apache.lucene.analysis.miscellaneous.LimitTokenPositionFilter;
@@ -406,13 +406,13 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
return new IdentityEncoder(); // the other encoders will throw exceptions if tokens arent numbers?
}
});
- put(HunspellDictionary.class, new ArgProducer() {
+ put(Dictionary.class, new ArgProducer() {
@Override public Object create(Random random) {
// TODO: make nastier
- InputStream affixStream = HunspellDictionaryTest.class.getResourceAsStream("test.aff");
- InputStream dictStream = HunspellDictionaryTest.class.getResourceAsStream("test.dic");
+ InputStream affixStream = TestHunspellStemFilter.class.getResourceAsStream("simple.aff");
+ InputStream dictStream = TestHunspellStemFilter.class.getResourceAsStream("simple.dic");
try {
- return new HunspellDictionary(affixStream, dictStream, TEST_VERSION_CURRENT);
+ return new Dictionary(affixStream, dictStream);
} catch (Exception ex) {
Rethrow.rethrow(ex);
return null; // unreachable code
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellDictionaryTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellDictionaryTest.java
deleted file mode 100644
index fd8f9211727..00000000000
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellDictionaryTest.java
+++ /dev/null
@@ -1,201 +0,0 @@
-package org.apache.lucene.analysis.hunspell;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.text.ParseException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.apache.lucene.util.LuceneTestCase;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class HunspellDictionaryTest extends LuceneTestCase {
-
- private class CloseCheckInputStream extends InputStream {
- private InputStream delegate;
-
- private boolean closed = false;
-
- public CloseCheckInputStream(InputStream delegate) {
- super();
- this.delegate = delegate;
- }
-
- @Override
- public int read() throws IOException {
- return delegate.read();
- }
-
- @Override
- public int hashCode() {
- return delegate.hashCode();
- }
-
- @Override
- public int read(byte[] b) throws IOException {
- return delegate.read(b);
- }
-
- @Override
- public boolean equals(Object obj) {
- return delegate.equals(obj);
- }
-
- @Override
- public int read(byte[] b, int off, int len) throws IOException {
- return delegate.read(b, off, len);
- }
-
- @Override
- public long skip(long n) throws IOException {
- return delegate.skip(n);
- }
-
- @Override
- public String toString() {
- return delegate.toString();
- }
-
- @Override
- public int available() throws IOException {
- return delegate.available();
- }
-
- @Override
- public void close() throws IOException {
- this.closed = true;
- delegate.close();
- }
-
- @Override
- public void mark(int readlimit) {
- delegate.mark(readlimit);
- }
-
- @Override
- public void reset() throws IOException {
- delegate.reset();
- }
-
- @Override
- public boolean markSupported() {
- return delegate.markSupported();
- }
-
- public boolean isClosed() {
- return this.closed;
- }
-
- }
-
- @Test
- public void testResourceCleanup() throws IOException, ParseException {
- CloseCheckInputStream affixStream = new CloseCheckInputStream(getClass().getResourceAsStream("testCompressed.aff"));
- CloseCheckInputStream dictStream = new CloseCheckInputStream(getClass().getResourceAsStream("testCompressed.dic"));
-
- new HunspellDictionary(affixStream, dictStream, TEST_VERSION_CURRENT);
-
- assertFalse(affixStream.isClosed());
- assertFalse(dictStream.isClosed());
-
- affixStream.close();
- dictStream.close();
-
- assertTrue(affixStream.isClosed());
- assertTrue(dictStream.isClosed());
- }
-
- @Test
- public void testHunspellDictionary_loadDicAff() throws IOException, ParseException {
- InputStream affixStream = getClass().getResourceAsStream("test.aff");
- InputStream dictStream = getClass().getResourceAsStream("test.dic");
-
- HunspellDictionary dictionary = new HunspellDictionary(affixStream, dictStream, TEST_VERSION_CURRENT);
- assertEquals(3, dictionary.lookupSuffix(new char[]{'e'}, 0, 1).size());
- assertEquals(1, dictionary.lookupPrefix(new char[]{'s'}, 0, 1).size());
- assertEquals(1, dictionary.lookupWord(new char[]{'o', 'l', 'r'}, 0, 3).size());
- assertEquals("Wrong number of flags for lucen", 1, dictionary.lookupWord(new char[]{'l', 'u', 'c', 'e', 'n'}, 0, 5).get(0).getFlags().length);
-
- affixStream.close();
- dictStream.close();
- }
-
- @Test
- public void testHunspellDictionary_multipleDictWithOverride() throws IOException, ParseException {
- InputStream affixStream = getClass().getResourceAsStream("test.aff");
- List dictStreams = new ArrayList();
- dictStreams.add(getClass().getResourceAsStream("test.dic"));
- dictStreams.add(getClass().getResourceAsStream("testOverride.dic"));
-
- HunspellDictionary dictionary = new HunspellDictionary(affixStream, dictStreams, TEST_VERSION_CURRENT, false);
- assertEquals("Wrong number of flags for lucen", 3, dictionary.lookupWord(new char[]{'l', 'u', 'c', 'e', 'n'}, 0, 5).get(0).getFlags().length);
- assertEquals("Wrong number of flags for bar", 1, dictionary.lookupWord(new char[]{'b', 'a', 'r'}, 0, 3).get(0).getFlags().length);
-
- affixStream.close();
- for(InputStream dstream : dictStreams) {
- dstream.close();
- }
- }
-
- @Test
- public void testCompressedHunspellDictionary_loadDicAff() throws IOException, ParseException {
- InputStream affixStream = getClass().getResourceAsStream("testCompressed.aff");
- InputStream dictStream = getClass().getResourceAsStream("testCompressed.dic");
-
- HunspellDictionary dictionary = new HunspellDictionary(affixStream, dictStream, TEST_VERSION_CURRENT);
- assertEquals(3, dictionary.lookupSuffix(new char[]{'e'}, 0, 1).size());
- assertEquals(1, dictionary.lookupPrefix(new char[]{'s'}, 0, 1).size());
- assertEquals(1, dictionary.lookupWord(new char[]{'o', 'l', 'r'}, 0, 3).size());
-
- affixStream.close();
- dictStream.close();
- }
-
- @Test
- public void testHunspellDictionary_loadDicWrongAff() throws IOException, ParseException {
- InputStream affixStream = getClass().getResourceAsStream("testWrongAffixRule.aff");
- InputStream dictStream = getClass().getResourceAsStream("test.dic");
-
- HunspellDictionary dictionary = new HunspellDictionary(affixStream, Arrays.asList(dictStream), TEST_VERSION_CURRENT, false, false);
- assertEquals(3, dictionary.lookupSuffix(new char[]{'e'}, 0, 1).size());
- assertEquals(1, dictionary.lookupPrefix(new char[]{'s'}, 0, 1).size());
- assertEquals(1, dictionary.lookupWord(new char[]{'o', 'l', 'r'}, 0, 3).size());
- //strict parsing disabled: malformed rule is not loaded
- assertNull(dictionary.lookupPrefix(new char[]{'a'}, 0, 1));
- affixStream.close();
- dictStream.close();
-
- affixStream = getClass().getResourceAsStream("testWrongAffixRule.aff");
- dictStream = getClass().getResourceAsStream("test.dic");
- //strict parsing enabled: malformed rule causes ParseException
- try {
- dictionary = new HunspellDictionary(affixStream, Arrays.asList(dictStream), TEST_VERSION_CURRENT, false, true);
- Assert.fail();
- } catch(ParseException e) {
- Assert.assertEquals("The affix file contains a rule with less than five elements", e.getMessage());
- Assert.assertEquals(23, e.getErrorOffset());
- }
-
- affixStream.close();
- dictStream.close();
- }
-}
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemmerTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemmerTest.java
deleted file mode 100644
index 66a9410c27a..00000000000
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemmerTest.java
+++ /dev/null
@@ -1,137 +0,0 @@
-package org.apache.lucene.analysis.hunspell;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.Version;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.text.ParseException;
-import java.util.List;
-
-import static junit.framework.Assert.assertEquals;
-
-public class HunspellStemmerTest extends LuceneTestCase {
-
- private static HunspellStemmer stemmer;
-
- @BeforeClass
- public static void beforeClass() throws IOException, ParseException {
- createStemmer(true);
- }
-
- @AfterClass
- public static void afterClass() {
- stemmer = null;
- }
-
- @Test
- public void testStem_simpleSuffix() {
- List stems = stemmer.stem("lucene");
-
- assertEquals(2, stems.size());
- assertEquals("lucene", stems.get(0).getStemString());
- assertEquals("lucen", stems.get(1).getStemString());
-
- stems = stemmer.stem("mahoute");
- assertEquals(1, stems.size());
- assertEquals("mahout", stems.get(0).getStemString());
- }
-
- @Test
- public void testStem_simplePrefix() {
- List stems = stemmer.stem("solr");
-
- assertEquals(1, stems.size());
- assertEquals("olr", stems.get(0).getStemString());
- }
-
- @Test
- public void testStem_recursiveSuffix() {
- List stems = stemmer.stem("abcd");
-
- assertEquals(1, stems.size());
- assertEquals("ab", stems.get(0).getStemString());
- }
-
- @Test
- public void testStem_ignoreCase() throws IOException, ParseException {
- List stems;
- createStemmer(true);
-
- stems = stemmer.stem("apache");
- assertEquals(1, stems.size());
- assertEquals("apach", stems.get(0).getStemString());
-
- stems = stemmer.stem("APACHE");
- assertEquals(1, stems.size());
- assertEquals("apach", stems.get(0).getStemString());
-
- stems = stemmer.stem("Apache");
- assertEquals(1, stems.size());
- assertEquals("apach", stems.get(0).getStemString());
-
- stems = stemmer.stem("foos");
- assertEquals(1, stems.size());
- assertEquals("foo", stems.get(0).getStemString());
-
- stems = stemmer.stem("mood");
- assertEquals(1, stems.size());
- assertEquals("moo", stems.get(0).getStemString());
-
- stems = stemmer.stem("Foos");
- assertEquals(1, stems.size());
- assertEquals("foo", stems.get(0).getStemString());
-
- // The "Foo" rule gets overridden by the "foo" rule, and we don't merge
- stems = stemmer.stem("Food");
- assertEquals(0, stems.size());
-
- stems = stemmer.stem("Mood");
- assertEquals(1, stems.size());
- assertEquals("moo", stems.get(0).getStemString());
- }
-
- @Test
- public void testStem_caseSensitive() throws IOException, ParseException {
- createStemmer(false);
- List stems = stemmer.stem("apache");
- assertEquals(0, stems.size());
-
- stems = stemmer.stem("Apache");
- assertEquals(1, stems.size());
- assertEquals("Apach", stems.get(0).getStemString());
- }
-
-
- private static void createStemmer(boolean ignoreCase) throws IOException, ParseException {
- InputStream affixStream = HunspellStemmerTest.class.getResourceAsStream("test.aff");
- InputStream dictStream = HunspellStemmerTest.class.getResourceAsStream("test.dic");
-
- HunspellDictionary dictionary = new HunspellDictionary(affixStream, dictStream, TEST_VERSION_CURRENT, ignoreCase);
- stemmer = new HunspellStemmer(dictionary);
-
- affixStream.close();
- dictStream.close();
- }
-
-}
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
new file mode 100644
index 00000000000..3322eb109a6
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestAllDictionaries.java
@@ -0,0 +1,203 @@
+package org.apache.lucene.analysis.hunspell;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.File;
+import java.io.InputStream;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipFile;
+
+import org.apache.lucene.analysis.hunspell.Dictionary;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.junit.Ignore;
+
+/**
+ * Can be retrieved via:
+ * wget --mirror -np http://archive.services.openoffice.org/pub/mirror/OpenOffice.org/contrib/dictionaries/
+ * Note some of the files differ only in case. This may be a problem on your operating system!
+ */
+@Ignore("enable manually")
+public class TestAllDictionaries extends LuceneTestCase {
+
+ // set this to the location of where you downloaded all the files
+ static final File DICTIONARY_HOME =
+ new File("/data/archive.services.openoffice.org/pub/mirror/OpenOffice.org/contrib/dictionaries");
+
+ final String tests[] = {
+ /* zip file */ /* dictionary */ /* affix */
+ "af_ZA.zip", "af_ZA.dic", "af_ZA.aff",
+ "ak_GH.zip", "ak_GH.dic", "ak_GH.aff",
+ "bg_BG.zip", "bg_BG.dic", "bg_BG.aff",
+ "ca_ANY.zip", "catalan.dic", "catalan.aff",
+ "ca_ES.zip", "ca_ES.dic", "ca_ES.aff",
+ "cop_EG.zip", "cop_EG.dic", "cop_EG.aff",
+ "cs_CZ.zip", "cs_CZ.dic", "cs_CZ.aff",
+ "cy_GB.zip", "cy_GB.dic", "cy_GB.aff",
+ "da_DK.zip", "da_DK.dic", "da_DK.aff",
+ "de_AT.zip", "de_AT.dic", "de_AT.aff",
+ "de_CH.zip", "de_CH.dic", "de_CH.aff",
+ "de_DE.zip", "de_DE.dic", "de_DE.aff",
+ "de_DE_comb.zip", "de_DE_comb.dic", "de_DE_comb.aff",
+ "de_DE_frami.zip", "de_DE_frami.dic", "de_DE_frami.aff",
+ "de_DE_neu.zip", "de_DE_neu.dic", "de_DE_neu.aff",
+ "el_GR.zip", "el_GR.dic", "el_GR.aff",
+ "en_AU.zip", "en_AU.dic", "en_AU.aff",
+ "en_CA.zip", "en_CA.dic", "en_CA.aff",
+ "en_GB-oed.zip", "en_GB-oed.dic", "en_GB-oed.aff",
+ "en_GB.zip", "en_GB.dic", "en_GB.aff",
+ "en_NZ.zip", "en_NZ.dic", "en_NZ.aff",
+ "eo.zip", "eo_l3.dic", "eo_l3.aff",
+ "eo_EO.zip", "eo_EO.dic", "eo_EO.aff",
+ "es_AR.zip", "es_AR.dic", "es_AR.aff",
+ "es_BO.zip", "es_BO.dic", "es_BO.aff",
+ "es_CL.zip", "es_CL.dic", "es_CL.aff",
+ "es_CO.zip", "es_CO.dic", "es_CO.aff",
+ "es_CR.zip", "es_CR.dic", "es_CR.aff",
+ "es_CU.zip", "es_CU.dic", "es_CU.aff",
+ "es_DO.zip", "es_DO.dic", "es_DO.aff",
+ "es_EC.zip", "es_EC.dic", "es_EC.aff",
+ "es_ES.zip", "es_ES.dic", "es_ES.aff",
+ "es_GT.zip", "es_GT.dic", "es_GT.aff",
+ "es_HN.zip", "es_HN.dic", "es_HN.aff",
+ "es_MX.zip", "es_MX.dic", "es_MX.aff",
+ "es_NEW.zip", "es_NEW.dic", "es_NEW.aff",
+ "es_NI.zip", "es_NI.dic", "es_NI.aff",
+ "es_PA.zip", "es_PA.dic", "es_PA.aff",
+ "es_PE.zip", "es_PE.dic", "es_PE.aff",
+ "es_PR.zip", "es_PR.dic", "es_PR.aff",
+ "es_PY.zip", "es_PY.dic", "es_PY.aff",
+ "es_SV.zip", "es_SV.dic", "es_SV.aff",
+ "es_UY.zip", "es_UY.dic", "es_UY.aff",
+ "es_VE.zip", "es_VE.dic", "es_VE.aff",
+ "et_EE.zip", "et_EE.dic", "et_EE.aff",
+ "fo_FO.zip", "fo_FO.dic", "fo_FO.aff",
+ "fr_FR-1990_1-3-2.zip", "fr_FR-1990.dic", "fr_FR-1990.aff",
+ "fr_FR-classique_1-3-2.zip", "fr_FR-classique.dic", "fr_FR-classique.aff",
+ "fr_FR_1-3-2.zip", "fr_FR.dic", "fr_FR.aff",
+ "fy_NL.zip", "fy_NL.dic", "fy_NL.aff",
+ "ga_IE.zip", "ga_IE.dic", "ga_IE.aff",
+ "gd_GB.zip", "gd_GB.dic", "gd_GB.aff",
+ "gl_ES.zip", "gl_ES.dic", "gl_ES.aff",
+ "gsc_FR.zip", "gsc_FR.dic", "gsc_FR.aff",
+ "gu_IN.zip", "gu_IN.dic", "gu_IN.aff",
+ "he_IL.zip", "he_IL.dic", "he_IL.aff",
+ "hi_IN.zip", "hi_IN.dic", "hi_IN.aff",
+ "hil_PH.zip", "hil_PH.dic", "hil_PH.aff",
+ "hr_HR.zip", "hr_HR.dic", "hr_HR.aff",
+ "hu_HU.zip", "hu_HU.dic", "hu_HU.aff",
+ "hu_HU_comb.zip", "hu_HU.dic", "hu_HU.aff",
+ "ia.zip", "ia.dic", "ia.aff",
+ "id_ID.zip", "id_ID.dic", "id_ID.aff",
+ "it_IT.zip", "it_IT.dic", "it_IT.aff",
+ "ku_TR.zip", "ku_TR.dic", "ku_TR.aff",
+ "la.zip", "la.dic", "la.aff",
+ "lt_LT.zip", "lt_LT.dic", "lt_LT.aff",
+ "lv_LV.zip", "lv_LV.dic", "lv_LV.aff",
+ "mg_MG.zip", "mg_MG.dic", "mg_MG.aff",
+ "mi_NZ.zip", "mi_NZ.dic", "mi_NZ.aff",
+ "mk_MK.zip", "mk_MK.dic", "mk_MK.aff",
+ "mos_BF.zip", "mos_BF.dic", "mos_BF.aff",
+ "mr_IN.zip", "mr_IN.dic", "mr_IN.aff",
+ "ms_MY.zip", "ms_MY.dic", "ms_MY.aff",
+ "nb_NO.zip", "nb_NO.dic", "nb_NO.aff",
+ "ne_NP.zip", "ne_NP.dic", "ne_NP.aff",
+ "nl_NL.zip", "nl_NL.dic", "nl_NL.aff",
+ "nl_med.zip", "nl_med.dic", "nl_med.aff",
+ "nn_NO.zip", "nn_NO.dic", "nn_NO.aff",
+ "nr_ZA.zip", "nr_ZA.dic", "nr_ZA.aff",
+ "ns_ZA.zip", "ns_ZA.dic", "ns_ZA.aff",
+ "ny_MW.zip", "ny_MW.dic", "ny_MW.aff",
+ "oc_FR.zip", "oc_FR.dic", "oc_FR.aff",
+ "pl_PL.zip", "pl_PL.dic", "pl_PL.aff",
+ "pt_BR.zip", "pt_BR.dic", "pt_BR.aff",
+ "pt_PT.zip", "pt_PT.dic", "pt_PT.aff",
+ "ro_RO.zip", "ro_RO.dic", "ro_RO.aff",
+ "ru_RU.zip", "ru_RU.dic", "ru_RU.aff",
+ "ru_RU_ye.zip", "ru_RU_ie.dic", "ru_RU_ie.aff",
+ "ru_RU_yo.zip", "ru_RU_yo.dic", "ru_RU_yo.aff",
+ "rw_RW.zip", "rw_RW.dic", "rw_RW.aff",
+ "sk_SK.zip", "sk_SK.dic", "sk_SK.aff",
+ "sl_SI.zip", "sl_SI.dic", "sl_SI.aff",
+ "sq_AL.zip", "sq_AL.dic", "sq_AL.aff",
+ "ss_ZA.zip", "ss_ZA.dic", "ss_ZA.aff",
+ "st_ZA.zip", "st_ZA.dic", "st_ZA.aff",
+ "sv_SE.zip", "sv_SE.dic", "sv_SE.aff",
+ "sw_KE.zip", "sw_KE.dic", "sw_KE.aff",
+ "tet_ID.zip", "tet_ID.dic", "tet_ID.aff",
+ "th_TH.zip", "th_TH.dic", "th_TH.aff",
+ "tl_PH.zip", "tl_PH.dic", "tl_PH.aff",
+ "tn_ZA.zip", "tn_ZA.dic", "tn_ZA.aff",
+ "ts_ZA.zip", "ts_ZA.dic", "ts_ZA.aff",
+ "uk_UA.zip", "uk_UA.dic", "uk_UA.aff",
+ "ve_ZA.zip", "ve_ZA.dic", "ve_ZA.aff",
+ "vi_VN.zip", "vi_VN.dic", "vi_VN.aff",
+ "xh_ZA.zip", "xh_ZA.dic", "xh_ZA.aff",
+ "zu_ZA.zip", "zu_ZA.dic", "zu_ZA.aff",
+ };
+
+ public void test() throws Exception {
+ for (int i = 0; i < tests.length; i += 3) {
+ File f = new File(DICTIONARY_HOME, tests[i]);
+ assert f.exists();
+
+ try (ZipFile zip = new ZipFile(f, IOUtils.CHARSET_UTF_8)) {
+ ZipEntry dicEntry = zip.getEntry(tests[i+1]);
+ assert dicEntry != null;
+ ZipEntry affEntry = zip.getEntry(tests[i+2]);
+ assert affEntry != null;
+
+ try (InputStream dictionary = zip.getInputStream(dicEntry);
+ InputStream affix = zip.getInputStream(affEntry)) {
+ Dictionary dic = new Dictionary(affix, dictionary);
+ System.out.println(tests[i] + "\t" + RamUsageEstimator.humanSizeOf(dic) + "\t(" +
+ "words=" + RamUsageEstimator.humanSizeOf(dic.words) + ", " +
+ "flags=" + RamUsageEstimator.humanSizeOf(dic.flagLookup) + ", " +
+ "strips=" + RamUsageEstimator.humanSizeOf(dic.stripLookup) + ", " +
+ "conditions=" + RamUsageEstimator.humanSizeOf(dic.patterns) + ", " +
+ "affixData=" + RamUsageEstimator.humanSizeOf(dic.affixData) + ", " +
+ "prefixes=" + RamUsageEstimator.humanSizeOf(dic.prefixes) + ", " +
+ "suffixes=" + RamUsageEstimator.humanSizeOf(dic.suffixes) + ")");
+ }
+ }
+ }
+ }
+
+ public void testOneDictionary() throws Exception {
+ String toTest = "hu_HU.zip";
+ for (int i = 0; i < tests.length; i++) {
+ if (tests[i].equals(toTest)) {
+ File f = new File(DICTIONARY_HOME, tests[i]);
+ assert f.exists();
+
+ try (ZipFile zip = new ZipFile(f, IOUtils.CHARSET_UTF_8)) {
+ ZipEntry dicEntry = zip.getEntry(tests[i+1]);
+ assert dicEntry != null;
+ ZipEntry affEntry = zip.getEntry(tests[i+2]);
+ assert affEntry != null;
+
+ try (InputStream dictionary = zip.getInputStream(dicEntry);
+ InputStream affix = zip.getInputStream(affEntry)) {
+ new Dictionary(affix, dictionary);
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestCaseInsensitive.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestCaseInsensitive.java
new file mode 100644
index 00000000000..64bdb41e8c7
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestCaseInsensitive.java
@@ -0,0 +1,110 @@
+package org.apache.lucene.analysis.hunspell;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.analysis.hunspell.Dictionary;
+import org.apache.lucene.analysis.hunspell.Stemmer;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public class TestCaseInsensitive extends LuceneTestCase {
+ private static Stemmer stemmer;
+
+ @BeforeClass
+ public static void beforeClass() throws Exception {
+ try (InputStream affixStream = TestCaseInsensitive.class.getResourceAsStream("simple.aff");
+ InputStream dictStream = TestCaseInsensitive.class.getResourceAsStream("mixedcase.dic")) {
+ Dictionary dictionary = new Dictionary(affixStream, Collections.singletonList(dictStream), true);
+ stemmer = new Stemmer(dictionary);
+ }
+ }
+
+ @AfterClass
+ public static void afterClass() {
+ stemmer = null;
+ }
+
+ public void testCaseInsensitivity() {
+ assertStemsTo("lucene", "lucene", "lucen");
+ assertStemsTo("LuCeNe", "lucene", "lucen");
+ assertStemsTo("mahoute", "mahout");
+ assertStemsTo("MaHoUte", "mahout");
+ }
+
+ public void testSimplePrefix() {
+ assertStemsTo("solr", "olr");
+ }
+
+ public void testRecursiveSuffix() {
+ assertStemsTo("abcd", "ab");
+ }
+
+ // all forms unmunched from dictionary
+ public void testAllStems() {
+ assertStemsTo("ab", "ab");
+ assertStemsTo("abc", "ab");
+ assertStemsTo("apach", "apach");
+ assertStemsTo("apache", "apach");
+ assertStemsTo("foo", "foo");
+ assertStemsTo("food", "foo");
+ assertStemsTo("foos", "foo");
+ assertStemsTo("lucen", "lucen");
+ assertStemsTo("lucene", "lucen", "lucene");
+ assertStemsTo("mahout", "mahout");
+ assertStemsTo("mahoute", "mahout");
+ assertStemsTo("moo", "moo");
+ assertStemsTo("mood", "moo");
+ assertStemsTo("olr", "olr");
+ assertStemsTo("solr", "olr");
+ }
+
+ // some bogus stuff that should not stem (empty lists)!
+ public void testBogusStems() {
+ assertStemsTo("abs");
+ assertStemsTo("abe");
+ assertStemsTo("sab");
+ assertStemsTo("sapach");
+ assertStemsTo("sapache");
+ assertStemsTo("apachee");
+ assertStemsTo("sfoo");
+ assertStemsTo("sfoos");
+ assertStemsTo("fooss");
+ assertStemsTo("lucenee");
+ assertStemsTo("solre");
+ }
+
+ private void assertStemsTo(String s, String... expected) {
+ Arrays.sort(expected);
+
+ List stems = stemmer.stem(s);
+ String actual[] = new String[stems.size()];
+ for (int i = 0; i < actual.length; i++) {
+ actual[i] = stems.get(i).toString();
+ }
+ Arrays.sort(actual);
+
+ assertArrayEquals(expected, actual);
+ }
+}
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
new file mode 100644
index 00000000000..6cbe931d376
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestDictionary.java
@@ -0,0 +1,110 @@
+package org.apache.lucene.analysis.hunspell;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.text.ParseException;
+
+import org.apache.lucene.analysis.hunspell.Dictionary;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestDictionary extends LuceneTestCase {
+
+ public void testSimpleDictionary() throws Exception {
+ InputStream affixStream = getClass().getResourceAsStream("simple.aff");
+ InputStream dictStream = getClass().getResourceAsStream("simple.dic");
+
+ Dictionary dictionary = new Dictionary(affixStream, dictStream);
+ assertEquals(3, dictionary.lookupSuffix(new char[]{'e'}, 0, 1).length);
+ assertEquals(1, dictionary.lookupPrefix(new char[]{'s'}, 0, 1).length);
+ char flags[] = dictionary.lookupWord(new char[]{'o', 'l', 'r'}, 0, 3, new BytesRef());
+ assertNotNull(flags);
+ assertEquals(1, flags.length);
+ assertEquals("Wrong number of flags for lucen", 1, dictionary.lookupWord(new char[]{'l', 'u', 'c', 'e', 'n'}, 0, 5, new BytesRef()).length);
+
+ affixStream.close();
+ dictStream.close();
+ }
+
+ public void testCompressedDictionary() throws Exception {
+ InputStream affixStream = getClass().getResourceAsStream("compressed.aff");
+ InputStream dictStream = getClass().getResourceAsStream("compressed.dic");
+
+ Dictionary dictionary = new Dictionary(affixStream, dictStream);
+ assertEquals(3, dictionary.lookupSuffix(new char[]{'e'}, 0, 1).length);
+ assertEquals(1, dictionary.lookupPrefix(new char[]{'s'}, 0, 1).length);
+ assertEquals(1, dictionary.lookupWord(new char[]{'o', 'l', 'r'}, 0, 3, new BytesRef()).length);
+
+ affixStream.close();
+ dictStream.close();
+ }
+
+ // malformed rule causes ParseException
+ public void testInvalidData() throws Exception {
+ InputStream affixStream = getClass().getResourceAsStream("broken.aff");
+ InputStream dictStream = getClass().getResourceAsStream("simple.dic");
+
+ try {
+ new Dictionary(affixStream, dictStream);
+ fail("didn't get expected exception");
+ } catch (ParseException expected) {
+ assertEquals("The affix file contains a rule with less than five elements", expected.getMessage());
+ assertEquals(23, expected.getErrorOffset());
+ }
+
+ affixStream.close();
+ dictStream.close();
+ }
+
+ private class CloseCheckInputStream extends FilterInputStream {
+ private boolean closed = false;
+
+ public CloseCheckInputStream(InputStream delegate) {
+ super(delegate);
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.closed = true;
+ super.close();
+ }
+
+ public boolean isClosed() {
+ return this.closed;
+ }
+ }
+
+ public void testResourceCleanup() throws Exception {
+ CloseCheckInputStream affixStream = new CloseCheckInputStream(getClass().getResourceAsStream("compressed.aff"));
+ CloseCheckInputStream dictStream = new CloseCheckInputStream(getClass().getResourceAsStream("compressed.dic"));
+
+ new Dictionary(affixStream, dictStream);
+
+ assertFalse(affixStream.isClosed());
+ assertFalse(dictStream.isClosed());
+
+ affixStream.close();
+ dictStream.close();
+
+ assertTrue(affixStream.isClosed());
+ assertTrue(dictStream.isClosed());
+ }
+}
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemFilterTest.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilter.java
similarity index 69%
rename from lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemFilterTest.java
rename to lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilter.java
index dd273fa8dc5..af48427d522 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/HunspellStemFilterTest.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilter.java
@@ -1,4 +1,5 @@
package org.apache.lucene.analysis.hunspell;
+
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@@ -18,7 +19,6 @@ package org.apache.lucene.analysis.hunspell;
import java.io.IOException;
import java.io.InputStream;
-import java.text.ParseException;
import java.util.Arrays;
import org.apache.lucene.analysis.Analyzer;
@@ -26,54 +26,59 @@ import org.apache.lucene.analysis.BaseTokenStreamTestCase;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.core.KeywordTokenizer;
+import org.apache.lucene.analysis.hunspell.Dictionary;
+import org.apache.lucene.analysis.hunspell.HunspellStemFilter;
import org.apache.lucene.analysis.miscellaneous.SetKeywordMarkerFilter;
import org.apache.lucene.analysis.util.CharArraySet;
import org.apache.lucene.util.TestUtil;
import org.junit.AfterClass;
import org.junit.BeforeClass;
-public class HunspellStemFilterTest extends BaseTokenStreamTestCase {
+public class TestHunspellStemFilter extends BaseTokenStreamTestCase {
+ private static Dictionary dictionary;
- private static HunspellDictionary DICTIONARY;
@BeforeClass
- public static void beforeClass() throws IOException, ParseException {
- DICTIONARY = createDict(true);
+ public static void beforeClass() throws Exception {
+ try (InputStream affixStream = TestStemmer.class.getResourceAsStream("simple.aff");
+ InputStream dictStream = TestStemmer.class.getResourceAsStream("simple.dic")) {
+ dictionary = new Dictionary(affixStream, dictStream);
+ }
}
+
@AfterClass
public static void afterClass() {
- DICTIONARY = null;
- }
- public static HunspellDictionary createDict(boolean ignoreCase) throws IOException, ParseException {
- InputStream affixStream = HunspellStemmerTest.class.getResourceAsStream("test.aff");
- InputStream dictStream = HunspellStemmerTest.class.getResourceAsStream("test.dic");
-
- return new HunspellDictionary(affixStream, dictStream, TEST_VERSION_CURRENT, ignoreCase);
+ dictionary = null;
}
- /**
- * Simple test for KeywordAttribute
- */
+ /** Simple test for KeywordAttribute */
public void testKeywordAttribute() throws IOException {
MockTokenizer tokenizer = whitespaceMockTokenizer("lucene is awesome");
tokenizer.setEnableChecks(true);
- HunspellStemFilter filter = new HunspellStemFilter(tokenizer, DICTIONARY, TestUtil.nextInt(random(), 1, 3));
+ HunspellStemFilter filter = new HunspellStemFilter(tokenizer, dictionary, TestUtil.nextInt(random(), 1, 3));
assertTokenStreamContents(filter, new String[]{"lucene", "lucen", "is", "awesome"}, new int[] {1, 0, 1, 1});
- // assert with keywork marker
+ // assert with keyword marker
tokenizer = whitespaceMockTokenizer("lucene is awesome");
CharArraySet set = new CharArraySet(TEST_VERSION_CURRENT, Arrays.asList("Lucene"), true);
- filter = new HunspellStemFilter(new SetKeywordMarkerFilter(tokenizer, set), DICTIONARY, TestUtil.nextInt(random(), 1, 3));
+ filter = new HunspellStemFilter(new SetKeywordMarkerFilter(tokenizer, set), dictionary, TestUtil.nextInt(random(), 1, 3));
+ assertTokenStreamContents(filter, new String[]{"lucene", "is", "awesome"}, new int[] {1, 1, 1});
+ }
+
+ /** simple test for longestOnly option */
+ public void testLongestOnly() throws IOException {
+ MockTokenizer tokenizer = whitespaceMockTokenizer("lucene is awesome");
+ tokenizer.setEnableChecks(true);
+ HunspellStemFilter filter = new HunspellStemFilter(tokenizer, dictionary, true, TestUtil.nextInt(random(), 1, 3), true);
assertTokenStreamContents(filter, new String[]{"lucene", "is", "awesome"}, new int[] {1, 1, 1});
}
/** blast some random strings through the analyzer */
public void testRandomStrings() throws Exception {
Analyzer analyzer = new Analyzer() {
-
@Override
protected TokenStreamComponents createComponents(String fieldName) {
Tokenizer tokenizer = new MockTokenizer(MockTokenizer.WHITESPACE, false);
- return new TokenStreamComponents(tokenizer, new HunspellStemFilter(tokenizer, DICTIONARY, TestUtil.nextInt(random(), 1, 3)));
+ return new TokenStreamComponents(tokenizer, new HunspellStemFilter(tokenizer, dictionary, TestUtil.nextInt(random(), 1, 3)));
}
};
checkRandomData(random(), analyzer, 1000*RANDOM_MULTIPLIER);
@@ -84,7 +89,7 @@ public class HunspellStemFilterTest extends BaseTokenStreamTestCase {
@Override
protected TokenStreamComponents createComponents(String fieldName) {
Tokenizer tokenizer = new KeywordTokenizer();
- return new TokenStreamComponents(tokenizer, new HunspellStemFilter(tokenizer, DICTIONARY, TestUtil.nextInt(random(), 1, 3)));
+ return new TokenStreamComponents(tokenizer, new HunspellStemFilter(tokenizer, dictionary, TestUtil.nextInt(random(), 1, 3)));
}
};
checkOneTerm(a, "", "");
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilterFactory.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilterFactory.java
index e8e232ce60b..b671f6dbdbf 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilterFactory.java
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestHunspellStemFilterFactory.java
@@ -20,7 +20,6 @@ package org.apache.lucene.analysis.hunspell;
import java.io.Reader;
import java.io.StringReader;
-import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.util.BaseTokenStreamFactoryTestCase;
@@ -32,8 +31,8 @@ public class TestHunspellStemFilterFactory extends BaseTokenStreamFactoryTestCas
Reader reader = new StringReader("abc");
TokenStream stream = whitespaceMockTokenizer(reader);
stream = tokenFilterFactory("HunspellStem",
- "dictionary", "test.dic",
- "affix", "test.aff").create(stream);
+ "dictionary", "simple.dic",
+ "affix", "simple.aff").create(stream);
assertTokenStreamContents(stream, new String[] { "ab" });
}
@@ -41,7 +40,7 @@ public class TestHunspellStemFilterFactory extends BaseTokenStreamFactoryTestCas
public void testBogusArguments() throws Exception {
try {
tokenFilterFactory("HunspellStem",
- "dictionary", "test.dic",
+ "dictionary", "simple.dic",
"bogusArg", "bogusValue");
fail();
} catch (IllegalArgumentException expected) {
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestStemmer.java b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestStemmer.java
new file mode 100644
index 00000000000..dca9faa6b16
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/TestStemmer.java
@@ -0,0 +1,107 @@
+package org.apache.lucene.analysis.hunspell;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.lucene.analysis.hunspell.Dictionary;
+import org.apache.lucene.analysis.hunspell.Stemmer;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.InputStream;
+import java.util.Arrays;
+import java.util.List;
+
+public class TestStemmer extends LuceneTestCase {
+ private static Stemmer stemmer;
+
+ @BeforeClass
+ public static void beforeClass() throws Exception {
+ try (InputStream affixStream = TestStemmer.class.getResourceAsStream("simple.aff");
+ InputStream dictStream = TestStemmer.class.getResourceAsStream("simple.dic")) {
+ Dictionary dictionary = new Dictionary(affixStream, dictStream);
+ stemmer = new Stemmer(dictionary);
+ }
+ }
+
+ @AfterClass
+ public static void afterClass() {
+ stemmer = null;
+ }
+
+ public void testSimpleSuffix() {
+ assertStemsTo("lucene", "lucene", "lucen");
+ assertStemsTo("mahoute", "mahout");
+ }
+
+ public void testSimplePrefix() {
+ assertStemsTo("solr", "olr");
+ }
+
+ public void testRecursiveSuffix() {
+ assertStemsTo("abcd", "ab");
+ }
+
+ // all forms unmunched from dictionary
+ public void testAllStems() {
+ assertStemsTo("ab", "ab");
+ assertStemsTo("abc", "ab");
+ assertStemsTo("apach", "apach");
+ assertStemsTo("apache", "apach");
+ assertStemsTo("foo", "foo");
+ assertStemsTo("food", "foo");
+ assertStemsTo("foos", "foo");
+ assertStemsTo("lucen", "lucen");
+ assertStemsTo("lucene", "lucen", "lucene");
+ assertStemsTo("mahout", "mahout");
+ assertStemsTo("mahoute", "mahout");
+ assertStemsTo("moo", "moo");
+ assertStemsTo("mood", "moo");
+ assertStemsTo("olr", "olr");
+ assertStemsTo("solr", "olr");
+ }
+
+ // some bogus stuff that should not stem (empty lists)!
+ public void testBogusStems() {
+ assertStemsTo("abs");
+ assertStemsTo("abe");
+ assertStemsTo("sab");
+ assertStemsTo("sapach");
+ assertStemsTo("sapache");
+ assertStemsTo("apachee");
+ assertStemsTo("sfoo");
+ assertStemsTo("sfoos");
+ assertStemsTo("fooss");
+ assertStemsTo("lucenee");
+ assertStemsTo("solre");
+ }
+
+ private void assertStemsTo(String s, String... expected) {
+ Arrays.sort(expected);
+
+ List stems = stemmer.stem(s);
+ String actual[] = new String[stems.size()];
+ for (int i = 0; i < actual.length; i++) {
+ actual[i] = stems.get(i).toString();
+ }
+ Arrays.sort(actual);
+
+ assertArrayEquals(expected, actual);
+ }
+}
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testWrongAffixRule.aff b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/broken.aff
similarity index 100%
rename from lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testWrongAffixRule.aff
rename to lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/broken.aff
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testCompressed.aff b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/compressed.aff
similarity index 100%
rename from lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testCompressed.aff
rename to lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/compressed.aff
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testCompressed.dic b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/compressed.dic
similarity index 75%
rename from lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testCompressed.dic
rename to lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/compressed.dic
index bf237662017..dd3890fae31 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testCompressed.dic
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/compressed.dic
@@ -1,9 +1,9 @@
6
+ab/3
+apach/1
+foo/4
+foo/5
lucen/1
lucene
mahout/1
olr/2
-ab/3
-Apach/1
-foo/4
-Foo/5
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/mixedcase.dic b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/mixedcase.dic
new file mode 100644
index 00000000000..9fae253279e
--- /dev/null
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/mixedcase.dic
@@ -0,0 +1,10 @@
+9
+Ab/C
+apach/A
+Foo/D
+foo/E
+Lucen/A
+Lucene
+mahout/A
+Moo/E
+olr/B
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/test.aff b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/simple.aff
similarity index 100%
rename from lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/test.aff
rename to lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/simple.aff
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/test.dic b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/simple.dic
similarity index 68%
rename from lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/test.dic
rename to lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/simple.dic
index 12efd8fccb2..f7bbab3ba67 100644
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/test.dic
+++ b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/simple.dic
@@ -1,10 +1,10 @@
9
+ab/C
+apach/A
+foo/D
+foo/E
lucen/A
lucene
mahout/A
+moo/E
olr/B
-ab/C
-Apach/A
-Foo/E
-foo/D
-Moo/E
\ No newline at end of file
diff --git a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testOverride.dic b/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testOverride.dic
deleted file mode 100644
index c1111ef562b..00000000000
--- a/lucene/analysis/common/src/test/org/apache/lucene/analysis/hunspell/testOverride.dic
+++ /dev/null
@@ -1,3 +0,0 @@
-2
-lucen/ABC
-bar/A
\ No newline at end of file
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java b/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
similarity index 99%
rename from lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java
rename to lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
index e7a44fc37e0..eb0aa1a808e 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/BytesRefArray.java
+++ b/lucene/core/src/java/org/apache/lucene/util/BytesRefArray.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.search.suggest;
+package org.apache.lucene.util;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/Sort.java b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
similarity index 95%
rename from lucene/suggest/src/java/org/apache/lucene/search/suggest/Sort.java
rename to lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
index 8c6c20f1444..76781f8b8c7 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/Sort.java
+++ b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.search.suggest;
+package org.apache.lucene.util;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,11 +17,24 @@ package org.apache.lucene.search.suggest;
* limitations under the License.
*/
-import java.io.*;
-import java.util.*;
-
-import org.apache.lucene.util.*;
-import org.apache.lucene.util.PriorityQueue;
+import java.io.BufferedInputStream;
+import java.io.BufferedOutputStream;
+import java.io.Closeable;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
/**
* On-disk sorting of byte arrays. Each byte array (entry) is a composed of the following
@@ -35,7 +48,7 @@ import org.apache.lucene.util.PriorityQueue;
* @lucene.experimental
* @lucene.internal
*/
-public final class Sort {
+public final class OfflineSorter {
/** Convenience constant for megabytes */
public final static long MB = 1024 * 1024;
/** Convenience constant for gigabytes */
@@ -170,7 +183,7 @@ public final class Sort {
* @see #defaultTempDir()
* @see BufferSize#automatic()
*/
- public Sort() throws IOException {
+ public OfflineSorter() throws IOException {
this(DEFAULT_COMPARATOR, BufferSize.automatic(), defaultTempDir(), MAX_TEMPFILES);
}
@@ -180,14 +193,14 @@ public final class Sort {
* @see #defaultTempDir()
* @see BufferSize#automatic()
*/
- public Sort(Comparator comparator) throws IOException {
+ public OfflineSorter(Comparator comparator) throws IOException {
this(comparator, BufferSize.automatic(), defaultTempDir(), MAX_TEMPFILES);
}
/**
* All-details constructor.
*/
- public Sort(Comparator comparator, BufferSize ramBufferSize, File tempDirectory, int maxTempfiles) {
+ public OfflineSorter(Comparator comparator, BufferSize ramBufferSize, File tempDirectory, int maxTempfiles) {
if (ramBufferSize.bytes < ABSOLUTE_MIN_SORT_BUFFER_SIZE) {
throw new IllegalArgumentException(MIN_BUFFER_SIZE_MSG + ": " + ramBufferSize.bytes);
}
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestBytesRefArray.java b/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
similarity index 98%
rename from lucene/suggest/src/test/org/apache/lucene/search/suggest/TestBytesRefArray.java
rename to lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
index 935b71bc529..9fcd6a1b6df 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/TestBytesRefArray.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestBytesRefArray.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.search.suggest;
+package org.apache.lucene.util;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestSort.java b/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
similarity index 72%
rename from lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestSort.java
rename to lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
index 540fadedf11..b7f14d02ffb 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/TestSort.java
+++ b/lucene/core/src/test/org/apache/lucene/util/TestOfflineSorter.java
@@ -1,4 +1,4 @@
-package org.apache.lucene.search.suggest.fst;
+package org.apache.lucene.util;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@@ -17,70 +17,72 @@ package org.apache.lucene.search.suggest.fst;
* limitations under the License.
*/
-import java.io.*;
+import java.io.DataInputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
-import org.apache.lucene.search.suggest.Sort;
-import org.apache.lucene.search.suggest.Sort.BufferSize;
-import org.apache.lucene.search.suggest.Sort.ByteSequencesWriter;
-import org.apache.lucene.search.suggest.Sort.SortInfo;
-import org.apache.lucene.util.*;
-import org.junit.*;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.BufferSize;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
+import org.apache.lucene.util.OfflineSorter.SortInfo;
+import org.apache.lucene.util.TestUtil;
/**
* Tests for on-disk merge sorting.
*/
-public class TestSort extends LuceneTestCase {
+public class TestOfflineSorter extends LuceneTestCase {
private File tempDir;
- @Before
- public void prepareTempDir() throws IOException {
+ @Override
+ public void setUp() throws Exception {
+ super.setUp();
tempDir = TestUtil.getTempDir("mergesort");
TestUtil.rmDir(tempDir);
tempDir.mkdirs();
}
- @After
- public void cleanup() throws IOException {
+ @Override
+ public void tearDown() throws Exception {
if (tempDir != null)
TestUtil.rmDir(tempDir);
+ super.tearDown();
}
- @Test
public void testEmpty() throws Exception {
- checkSort(new Sort(), new byte [][] {});
+ checkSort(new OfflineSorter(), new byte [][] {});
}
- @Test
public void testSingleLine() throws Exception {
- checkSort(new Sort(), new byte [][] {
+ checkSort(new OfflineSorter(), new byte [][] {
"Single line only.".getBytes("UTF-8")
});
}
- @Test
public void testIntermediateMerges() throws Exception {
// Sort 20 mb worth of data with 1mb buffer, binary merging.
- SortInfo info = checkSort(new Sort(Sort.DEFAULT_COMPARATOR, BufferSize.megabytes(1), Sort.defaultTempDir(), 2),
- generateRandom((int)Sort.MB * 20));
+ SortInfo info = checkSort(new OfflineSorter(OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), OfflineSorter.defaultTempDir(), 2),
+ generateRandom((int)OfflineSorter.MB * 20));
assertTrue(info.mergeRounds > 10);
}
- @Test
public void testSmallRandom() throws Exception {
// Sort 20 mb worth of data with 1mb buffer.
- SortInfo sortInfo = checkSort(new Sort(Sort.DEFAULT_COMPARATOR, BufferSize.megabytes(1), Sort.defaultTempDir(), Sort.MAX_TEMPFILES),
- generateRandom((int)Sort.MB * 20));
+ SortInfo sortInfo = checkSort(new OfflineSorter(OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(1), OfflineSorter.defaultTempDir(), OfflineSorter.MAX_TEMPFILES),
+ generateRandom((int)OfflineSorter.MB * 20));
assertEquals(1, sortInfo.mergeRounds);
}
- @Test @Nightly
+ @Nightly
public void testLargerRandom() throws Exception {
// Sort 100MB worth of data with 15mb buffer.
- checkSort(new Sort(Sort.DEFAULT_COMPARATOR, BufferSize.megabytes(16), Sort.defaultTempDir(), Sort.MAX_TEMPFILES),
- generateRandom((int)Sort.MB * 100));
+ checkSort(new OfflineSorter(OfflineSorter.DEFAULT_COMPARATOR, BufferSize.megabytes(16), OfflineSorter.defaultTempDir(), OfflineSorter.MAX_TEMPFILES),
+ generateRandom((int)OfflineSorter.MB * 100));
}
private byte[][] generateRandom(int howMuchData) {
@@ -108,9 +110,9 @@ public class TestSort extends LuceneTestCase {
}
};
/**
- * Check sorting data on an instance of {@link Sort}.
+ * Check sorting data on an instance of {@link OfflineSorter}.
*/
- private SortInfo checkSort(Sort sort, byte[][] data) throws IOException {
+ private SortInfo checkSort(OfflineSorter sort, byte[][] data) throws IOException {
File unsorted = writeAll("unsorted", data);
Arrays.sort(data, unsignedByteOrderComparator);
@@ -147,7 +149,7 @@ public class TestSort extends LuceneTestCase {
private File writeAll(String name, byte[][] data) throws IOException {
File file = new File(tempDir, name);
- ByteSequencesWriter w = new Sort.ByteSequencesWriter(file);
+ ByteSequencesWriter w = new OfflineSorter.ByteSequencesWriter(file);
for (byte [] datum : data) {
w.write(datum);
}
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferedInputIterator.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferedInputIterator.java
index b9772fafebd..96c7cf85f60 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferedInputIterator.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/BufferedInputIterator.java
@@ -21,6 +21,7 @@ import java.io.IOException;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefArray;
import org.apache.lucene.util.Counter;
/**
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/InMemorySorter.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/InMemorySorter.java
index 0efc3a5fa7a..42e19a8f9b9 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/InMemorySorter.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/InMemorySorter.java
@@ -21,6 +21,7 @@ import java.util.Comparator;
import org.apache.lucene.search.suggest.fst.BytesRefSorter;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefArray;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.Counter;
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
index d804f38e1b1..d7011d435d9 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/SortedInputIterator.java
@@ -21,13 +21,14 @@ import java.io.File;
import java.io.IOException;
import java.util.Comparator;
-import org.apache.lucene.search.suggest.Sort.ByteSequencesReader;
-import org.apache.lucene.search.suggest.Sort.ByteSequencesWriter;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesReader;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
/**
* This wrapper buffers incoming elements and makes sure they are sorted based on given comparator.
@@ -141,13 +142,13 @@ public class SortedInputIterator implements InputIterator {
}
};
- private Sort.ByteSequencesReader sort() throws IOException {
+ private ByteSequencesReader sort() throws IOException {
String prefix = getClass().getSimpleName();
- File directory = Sort.defaultTempDir();
+ File directory = OfflineSorter.defaultTempDir();
tempInput = File.createTempFile(prefix, ".input", directory);
tempSorted = File.createTempFile(prefix, ".sorted", directory);
- final Sort.ByteSequencesWriter writer = new Sort.ByteSequencesWriter(tempInput);
+ final OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(tempInput);
boolean success = false;
try {
BytesRef spare;
@@ -158,8 +159,8 @@ public class SortedInputIterator implements InputIterator {
encode(writer, output, buffer, spare, source.payload(), source.weight());
}
writer.close();
- new Sort(tieBreakByCostComparator).sort(tempInput, tempSorted);
- ByteSequencesReader reader = new Sort.ByteSequencesReader(tempSorted);
+ new OfflineSorter(tieBreakByCostComparator).sort(tempInput, tempSorted);
+ ByteSequencesReader reader = new OfflineSorter.ByteSequencesReader(tempSorted);
success = true;
return reader;
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
index 6b2c1f6bbe1..5dad351a83f 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
@@ -31,7 +31,6 @@ import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.TokenStreamToAutomaton;
import org.apache.lucene.search.suggest.InputIterator;
import org.apache.lucene.search.suggest.Lookup;
-import org.apache.lucene.search.suggest.Sort;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.store.DataInput;
@@ -56,6 +55,7 @@ import org.apache.lucene.util.fst.PairOutputs;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util.MinResult;
import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.OfflineSorter;
/**
* Suggester that first analyzes the surface form, adds the
@@ -380,14 +380,14 @@ public class AnalyzingSuggester extends Lookup {
@Override
public void build(InputIterator iterator) throws IOException {
String prefix = getClass().getSimpleName();
- File directory = Sort.defaultTempDir();
+ File directory = OfflineSorter.defaultTempDir();
File tempInput = File.createTempFile(prefix, ".input", directory);
File tempSorted = File.createTempFile(prefix, ".sorted", directory);
hasPayloads = iterator.hasPayloads();
- Sort.ByteSequencesWriter writer = new Sort.ByteSequencesWriter(tempInput);
- Sort.ByteSequencesReader reader = null;
+ OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(tempInput);
+ OfflineSorter.ByteSequencesReader reader = null;
BytesRef scratch = new BytesRef();
TokenStreamToAutomaton ts2a = getTokenStreamToAutomaton();
@@ -463,12 +463,12 @@ public class AnalyzingSuggester extends Lookup {
writer.close();
// Sort all input/output pairs (required by FST.Builder):
- new Sort(new AnalyzingComparator(hasPayloads)).sort(tempInput, tempSorted);
+ new OfflineSorter(new AnalyzingComparator(hasPayloads)).sort(tempInput, tempSorted);
// Free disk space:
tempInput.delete();
- reader = new Sort.ByteSequencesReader(tempSorted);
+ reader = new OfflineSorter.ByteSequencesReader(tempSorted);
PairOutputs outputs = new PairOutputs(PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton());
Builder> builder = new Builder>(FST.INPUT_TYPE.BYTE1, outputs);
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
index db332474c9a..f425235e272 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FreeTextSuggester.java
@@ -20,6 +20,7 @@ package org.apache.lucene.search.suggest.analyzing;
// TODO
// - test w/ syns
// - add pruning of low-freq ngrams?
+
import java.io.File;
import java.io.IOException;
//import java.io.PrintWriter;
@@ -54,7 +55,6 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.suggest.InputIterator;
import org.apache.lucene.search.suggest.Lookup;
-import org.apache.lucene.search.suggest.Sort;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
@@ -74,6 +74,7 @@ import org.apache.lucene.util.fst.Outputs;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util.MinResult;
import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.OfflineSorter;
/**
* Builds an ngram model from the text sent to {@link
@@ -287,7 +288,7 @@ public class FreeTextSuggester extends Lookup {
}
String prefix = getClass().getSimpleName();
- File directory = Sort.defaultTempDir();
+ File directory = OfflineSorter.defaultTempDir();
// TODO: messy ... java7 has Files.createTempDirectory
// ... but 4.x is java6:
File tempIndexPath = null;
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
index 0a06b861e83..8ceb937e74d 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/ExternalRefSorter.java
@@ -17,14 +17,15 @@ package org.apache.lucene.search.suggest.fst;
* limitations under the License.
*/
-import java.io.*;
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
import java.util.Comparator;
-import org.apache.lucene.search.suggest.Sort;
-import org.apache.lucene.search.suggest.Sort.ByteSequencesReader;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.OfflineSorter;
/**
* Builds and iterates over sequences stored on disk.
@@ -32,19 +33,19 @@ import org.apache.lucene.util.IOUtils;
* @lucene.internal
*/
public class ExternalRefSorter implements BytesRefSorter, Closeable {
- private final Sort sort;
- private Sort.ByteSequencesWriter writer;
+ private final OfflineSorter sort;
+ private OfflineSorter.ByteSequencesWriter writer;
private File input;
private File sorted;
/**
* Will buffer all sequences to a temporary file and then sort (all on-disk).
*/
- public ExternalRefSorter(Sort sort) throws IOException {
+ public ExternalRefSorter(OfflineSorter sort) throws IOException {
this.sort = sort;
this.input = File.createTempFile("RefSorter-", ".raw",
- Sort.defaultTempDir());
- this.writer = new Sort.ByteSequencesWriter(input);
+ OfflineSorter.defaultTempDir());
+ this.writer = new OfflineSorter.ByteSequencesWriter(input);
}
@Override
@@ -59,14 +60,14 @@ public class ExternalRefSorter implements BytesRefSorter, Closeable {
closeWriter();
sorted = File.createTempFile("RefSorter-", ".sorted",
- Sort.defaultTempDir());
+ OfflineSorter.defaultTempDir());
sort.sort(input, sorted);
input.delete();
input = null;
}
- return new ByteSequenceIterator(new Sort.ByteSequencesReader(sorted));
+ return new ByteSequenceIterator(new OfflineSorter.ByteSequencesReader(sorted));
}
private void closeWriter() throws IOException {
@@ -93,10 +94,10 @@ public class ExternalRefSorter implements BytesRefSorter, Closeable {
* Iterate over byte refs in a file.
*/
class ByteSequenceIterator implements BytesRefIterator {
- private final ByteSequencesReader reader;
+ private final OfflineSorter.ByteSequencesReader reader;
private BytesRef scratch = new BytesRef();
- public ByteSequenceIterator(ByteSequencesReader reader) {
+ public ByteSequenceIterator(OfflineSorter.ByteSequencesReader reader) {
this.reader = reader;
}
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
index 912c1ee6bbc..0f410c69970 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/FSTCompletionLookup.java
@@ -19,26 +19,27 @@ package org.apache.lucene.search.suggest.fst;
import java.io.File;
import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.search.suggest.InputIterator;
import org.apache.lucene.search.suggest.Lookup;
-import org.apache.lucene.search.suggest.Sort.SortInfo;
-import org.apache.lucene.search.suggest.Sort;
import org.apache.lucene.search.suggest.fst.FSTCompletion.Completion;
import org.apache.lucene.search.suggest.tst.TSTLookup;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
-import org.apache.lucene.store.InputStreamDataInput;
-import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.util.*;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.NoOutputs;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CharsRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.OfflineSorter;
+import org.apache.lucene.util.OfflineSorter.SortInfo;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.UnicodeUtil;
/**
* An adapter from {@link Lookup} API to {@link FSTCompletion}.
@@ -150,12 +151,12 @@ public class FSTCompletionLookup extends Lookup {
throw new IllegalArgumentException("this suggester doesn't support payloads");
}
File tempInput = File.createTempFile(
- FSTCompletionLookup.class.getSimpleName(), ".input", Sort.defaultTempDir());
+ FSTCompletionLookup.class.getSimpleName(), ".input", OfflineSorter.defaultTempDir());
File tempSorted = File.createTempFile(
- FSTCompletionLookup.class.getSimpleName(), ".sorted", Sort.defaultTempDir());
+ FSTCompletionLookup.class.getSimpleName(), ".sorted", OfflineSorter.defaultTempDir());
- Sort.ByteSequencesWriter writer = new Sort.ByteSequencesWriter(tempInput);
- Sort.ByteSequencesReader reader = null;
+ OfflineSorter.ByteSequencesWriter writer = new OfflineSorter.ByteSequencesWriter(tempInput);
+ OfflineSorter.ByteSequencesReader reader = null;
ExternalRefSorter sorter = null;
// Push floats up front before sequences to sort them. For now, assume they are non-negative.
@@ -180,13 +181,13 @@ public class FSTCompletionLookup extends Lookup {
// We don't know the distribution of scores and we need to bucket them, so we'll sort
// and divide into equal buckets.
- SortInfo info = new Sort().sort(tempInput, tempSorted);
+ SortInfo info = new OfflineSorter().sort(tempInput, tempSorted);
tempInput.delete();
FSTCompletionBuilder builder = new FSTCompletionBuilder(
- buckets, sorter = new ExternalRefSorter(new Sort()), sharedTailLength);
+ buckets, sorter = new ExternalRefSorter(new OfflineSorter()), sharedTailLength);
final int inputLines = info.lines;
- reader = new Sort.ByteSequencesReader(tempSorted);
+ reader = new OfflineSorter.ByteSequencesReader(tempSorted);
long line = 0;
int previousBucket = 0;
int previousScore = 0;
diff --git a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
index ab668413831..d654f182e48 100644
--- a/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
+++ b/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
@@ -25,7 +25,6 @@ import java.util.List;
import org.apache.lucene.search.suggest.InputIterator;
import org.apache.lucene.search.suggest.Lookup;
-import org.apache.lucene.search.suggest.Sort.ByteSequencesWriter;
import org.apache.lucene.search.suggest.SortedInputIterator;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.ByteArrayDataOutput;
@@ -43,6 +42,7 @@ import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util.MinResult;
import org.apache.lucene.util.fst.Util;
+import org.apache.lucene.util.OfflineSorter.ByteSequencesWriter;
/**
* Suggester based on a weighted FST: it first traverses the prefix,
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java
index f8ccd35b55c..82775898475 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/BytesRefSortersTest.java
@@ -18,16 +18,16 @@ package org.apache.lucene.search.suggest.fst;
*/
import org.apache.lucene.search.suggest.InMemorySorter;
-import org.apache.lucene.search.suggest.Sort;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.OfflineSorter;
import org.junit.Test;
public class BytesRefSortersTest extends LuceneTestCase {
@Test
public void testExternalRefSorter() throws Exception {
- ExternalRefSorter s = new ExternalRefSorter(new Sort());
+ ExternalRefSorter s = new ExternalRefSorter(new OfflineSorter());
check(s);
s.close();
}
diff --git a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/LargeInputFST.java b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/LargeInputFST.java
index 48a1409d9f9..0cb6c668d02 100644
--- a/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/LargeInputFST.java
+++ b/lucene/suggest/src/test/org/apache/lucene/search/suggest/fst/LargeInputFST.java
@@ -17,10 +17,15 @@ package org.apache.lucene.search.suggest.fst;
* limitations under the License.
*/
-import java.io.*;
-import org.apache.lucene.search.suggest.Sort;
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.OfflineSorter;
/**
* Try to build a suggester from a large data set. The input is a simple text
@@ -33,7 +38,7 @@ public class LargeInputFST {
int buckets = 20;
int shareMaxTail = 10;
- ExternalRefSorter sorter = new ExternalRefSorter(new Sort());
+ ExternalRefSorter sorter = new ExternalRefSorter(new OfflineSorter());
FSTCompletionBuilder builder = new FSTCompletionBuilder(buckets, sorter, shareMaxTail);
BufferedReader reader = new BufferedReader(