Merged /lucene/dev/trunk:r1430124-1432061

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4547@1432065 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2013-01-11 14:39:45 +00:00
commit f7c90c50f3
139 changed files with 5586 additions and 4390 deletions

View File

@ -281,18 +281,9 @@
<copy todir="${fakeRelease}/lucene">
<fileset dir="lucene/dist"/>
</copy>
<copy todir="${fakeRelease}/lucene/changes">
<fileset dir="lucene/build/docs/changes"/>
</copy>
<get src="http://people.apache.org/keys/group/lucene.asc"
dest="${fakeRelease}/lucene/KEYS"/>
<copy todir="${fakeRelease}/solr">
<fileset dir="solr/package"/>
</copy>
<copy file="${fakeRelease}/lucene/KEYS" todir="${fakeRelease}/solr"/>
<copy todir="${fakeRelease}/solr/changes">
<fileset dir="solr/build/docs/changes"/>
</copy>
<makeurl file="${fakeRelease}" validate="false" property="fakeRelease.uri"/>
<exec executable="${python32.exe}" failonerror="true">
<arg value="-u"/>

View File

@ -106,7 +106,15 @@ Changes in backwards compatibility policy
* LUCENE-4659: Massive cleanup to CategoryPath API. Additionally, CategoryPath is
now immutable, so you don't need to clone() it. (Shai Erera)
* LUCENE-4670: StoredFieldsWriter and TermVectorsWriter have new finish* callbacks
which are called after a doc/field/term has been completely added.
(Adrien Grand, Robert Muir)
* LUCENE-4620: IntEncoder/Decoder were changed to do bulk encoding/decoding. As a
result, few other classes such as Aggregator and CategoryListIterator were
changed to handle bulk category ordinals. (Shai Erera)
New Features
* LUCENE-4226: New experimental StoredFieldsFormat that compresses chunks of
@ -324,6 +332,8 @@ Bug Fixes
* LUCENE-4662: Add missing elided articles and prepositions to FrenchAnalyzer's
DEFAULT_ARTICLES list passed to ElisionFilter. (David Leunen via Steve Rowe)
* LUCENE-4671: Fix CharsRef.subSequence method. (Tim Smith via Robert Muir)
Changes in Runtime Behavior

View File

@ -34,6 +34,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.List;
@ -66,6 +67,8 @@ 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.miscellaneous.HyphenatedWordsFilter;
import org.apache.lucene.analysis.miscellaneous.KeepWordFilter;
import org.apache.lucene.analysis.miscellaneous.LengthFilter;
import org.apache.lucene.analysis.miscellaneous.LimitTokenCountFilter;
import org.apache.lucene.analysis.miscellaneous.TrimFilter;
import org.apache.lucene.analysis.miscellaneous.WordDelimiterFilter;
@ -103,67 +106,145 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
static List<Constructor<? extends TokenFilter>> tokenfilters;
static List<Constructor<? extends CharFilter>> charfilters;
// TODO: fix those and remove
private static final Set<Class<?>> brokenComponents = Collections.newSetFromMap(new IdentityHashMap<Class<?>,Boolean>());
private static interface Predicate<T> {
boolean apply(T o);
}
private static final Predicate<Object[]> ALWAYS = new Predicate<Object[]>() {
public boolean apply(Object[] args) {
return true;
};
};
private static final Map<Constructor<?>,Predicate<Object[]>> brokenConstructors = new HashMap<Constructor<?>, Predicate<Object[]>>();
static {
// TODO: can we promote some of these to be only
// offsets offenders?
Collections.<Class<?>>addAll(brokenComponents,
// doesn't actual reset itself!
CachingTokenFilter.class,
// doesn't consume whole stream!
LimitTokenCountFilter.class,
// Not broken: we forcefully add this, so we shouldn't
// also randomly pick it:
ValidatingTokenFilter.class,
// NOTE: these by themselves won't cause any 'basic assertions' to fail.
// but see https://issues.apache.org/jira/browse/LUCENE-3920, if any
// tokenfilter that combines words (e.g. shingles) comes after them,
// this will create bogus offsets because their 'offsets go backwards',
// causing shingle or whatever to make a single token with a
// startOffset thats > its endOffset
// (see LUCENE-3738 for a list of other offenders here)
// broken!
NGramTokenizer.class,
// broken!
NGramTokenFilter.class,
// broken!
EdgeNGramTokenizer.class,
// broken!
EdgeNGramTokenFilter.class,
// broken!
WordDelimiterFilter.class,
// broken!
TrimFilter.class
);
try {
brokenConstructors.put(
LimitTokenCountFilter.class.getConstructor(TokenStream.class, int.class),
ALWAYS);
brokenConstructors.put(
LimitTokenCountFilter.class.getConstructor(TokenStream.class, int.class, boolean.class),
new Predicate<Object[]>() {
@Override
public boolean apply(Object[] args) {
assert args.length == 3;
return !((Boolean) args[2]); // args are broken if consumeAllTokens is false
}
});
for (Class<?> c : Arrays.<Class<?>>asList(
// TODO: can we promote some of these to be only
// offsets offenders?
// doesn't actual reset itself!
CachingTokenFilter.class,
// Not broken: we forcefully add this, so we shouldn't
// also randomly pick it:
ValidatingTokenFilter.class,
// NOTE: these by themselves won't cause any 'basic assertions' to fail.
// but see https://issues.apache.org/jira/browse/LUCENE-3920, if any
// tokenfilter that combines words (e.g. shingles) comes after them,
// this will create bogus offsets because their 'offsets go backwards',
// causing shingle or whatever to make a single token with a
// startOffset thats > its endOffset
// (see LUCENE-3738 for a list of other offenders here)
// broken!
NGramTokenizer.class,
// broken!
NGramTokenFilter.class,
// broken!
EdgeNGramTokenizer.class,
// broken!
EdgeNGramTokenFilter.class,
// broken!
WordDelimiterFilter.class)) {
for (Constructor<?> ctor : c.getConstructors()) {
brokenConstructors.put(ctor, ALWAYS);
}
}
} catch (Exception e) {
throw new Error(e);
}
}
// TODO: also fix these and remove (maybe):
// Classes that don't produce consistent graph offsets:
private static final Set<Class<?>> brokenOffsetsComponents = Collections.newSetFromMap(new IdentityHashMap<Class<?>,Boolean>());
// Classes/options that don't produce consistent graph offsets:
private static final Map<Constructor<?>,Predicate<Object[]>> brokenOffsetsConstructors = new HashMap<Constructor<?>, Predicate<Object[]>>();
static {
Collections.<Class<?>>addAll(brokenOffsetsComponents,
ReversePathHierarchyTokenizer.class,
PathHierarchyTokenizer.class,
HyphenationCompoundWordTokenFilter.class,
DictionaryCompoundWordTokenFilter.class,
// TODO: corrumpts graphs (offset consistency check):
PositionFilter.class,
// TODO: it seems to mess up offsets!?
WikipediaTokenizer.class,
// TODO: doesn't handle graph inputs
ThaiWordFilter.class,
// TODO: doesn't handle graph inputs
CJKBigramFilter.class,
// TODO: doesn't handle graph inputs (or even look at positionIncrement)
HyphenatedWordsFilter.class,
// LUCENE-4065: only if you pass 'false' to enablePositionIncrements!
TypeTokenFilter.class,
// TODO: doesn't handle graph inputs
CommonGramsQueryFilter.class
);
try {
brokenOffsetsConstructors.put(
TrimFilter.class.getConstructor(TokenStream.class, boolean.class),
new Predicate<Object[]>() {
@Override
public boolean apply(Object[] args) {
assert args.length == 2;
return (Boolean) args[1]; // args are broken if updateOffsets is true
}
});
brokenOffsetsConstructors.put(
TypeTokenFilter.class.getConstructor(boolean.class, TokenStream.class, Set.class, boolean.class),
new Predicate<Object[]>() {
@Override
public boolean apply(Object[] args) {
assert args.length == 4;
// LUCENE-4065: only if you pass 'false' to enablePositionIncrements!
return !(Boolean) args[0];
}
});
brokenOffsetsConstructors.put(
TypeTokenFilter.class.getConstructor(boolean.class, TokenStream.class, Set.class),
new Predicate<Object[]>() {
@Override
public boolean apply(Object[] args) {
assert args.length == 3;
// LUCENE-4065: only if you pass 'false' to enablePositionIncrements!
return !(Boolean) args[0];
}
});
brokenOffsetsConstructors.put(
LengthFilter.class.getConstructor(boolean.class, TokenStream.class, int.class, int.class),
new Predicate<Object[]>() {
@Override
public boolean apply(Object[] args) {
assert args.length == 4;
// LUCENE-4065: only if you pass 'false' to enablePositionIncrements!
return !(Boolean) args[0];
}
});
brokenOffsetsConstructors.put(
KeepWordFilter.class.getConstructor(boolean.class, TokenStream.class, CharArraySet.class),
new Predicate<Object[]>() {
@Override
public boolean apply(Object[] args) {
assert args.length == 3;
// LUCENE-4065: only if you pass 'false' to enablePositionIncrements!
return !(Boolean) args[0];
}
});
for (Class<?> c : Arrays.<Class<?>>asList(
ReversePathHierarchyTokenizer.class,
PathHierarchyTokenizer.class,
HyphenationCompoundWordTokenFilter.class,
DictionaryCompoundWordTokenFilter.class,
// TODO: corrumpts graphs (offset consistency check):
PositionFilter.class,
// TODO: it seems to mess up offsets!?
WikipediaTokenizer.class,
// TODO: doesn't handle graph inputs
ThaiWordFilter.class,
// TODO: doesn't handle graph inputs
CJKBigramFilter.class,
// TODO: doesn't handle graph inputs (or even look at positionIncrement)
HyphenatedWordsFilter.class,
// TODO: doesn't handle graph inputs
CommonGramsQueryFilter.class)) {
for (Constructor<?> ctor : c.getConstructors()) {
brokenOffsetsConstructors.put(ctor, ALWAYS);
}
}
} catch (Exception e) {
throw new Error(e);
}
}
@BeforeClass
public static void beforeClass() throws Exception {
List<Class<?>> analysisClasses = getClassesForPackage("org.apache.lucene.analysis");
@ -176,7 +257,6 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
// don't waste time with abstract classes or deprecated known-buggy ones
Modifier.isAbstract(modifiers) || !Modifier.isPublic(modifiers)
|| c.isSynthetic() || c.isAnonymousClass() || c.isMemberClass() || c.isInterface()
|| brokenComponents.contains(c)
|| c.isAnnotationPresent(Deprecated.class)
|| !(Tokenizer.class.isAssignableFrom(c) || TokenFilter.class.isAssignableFrom(c) || CharFilter.class.isAssignableFrom(c))
) {
@ -185,7 +265,7 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
for (final Constructor<?> ctor : c.getConstructors()) {
// don't test synthetic or deprecated ctors, they likely have known bugs:
if (ctor.isSynthetic() || ctor.isAnnotationPresent(Deprecated.class)) {
if (ctor.isSynthetic() || ctor.isAnnotationPresent(Deprecated.class) || brokenConstructors.get(ctor) == ALWAYS) {
continue;
}
if (Tokenizer.class.isAssignableFrom(c)) {
@ -679,7 +759,17 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
}
return null; // no success
}
private boolean broken(Constructor<?> ctor, Object[] args) {
final Predicate<Object[]> pred = brokenConstructors.get(ctor);
return pred != null && pred.apply(args);
}
private boolean brokenOffsets(Constructor<?> ctor, Object[] args) {
final Predicate<Object[]> pred = brokenOffsetsConstructors.get(ctor);
return pred != null && pred.apply(args);
}
// create a new random tokenizer from classpath
private TokenizerSpec newTokenizer(Random random, Reader reader) {
TokenizerSpec spec = new TokenizerSpec();
@ -688,11 +778,12 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
final StringBuilder descr = new StringBuilder();
final CheckThatYouDidntReadAnythingReaderWrapper wrapper = new CheckThatYouDidntReadAnythingReaderWrapper(reader);
final Object args[] = newTokenizerArgs(random, wrapper, ctor.getParameterTypes());
if (broken(ctor, args)) {
continue;
}
spec.tokenizer = createComponent(ctor, args, descr);
if (spec.tokenizer != null) {
if (brokenOffsetsComponents.contains(ctor.getDeclaringClass())) {
spec.offsetsAreCorrect = false;
}
spec.offsetsAreCorrect &= !brokenOffsets(ctor, args);
spec.toString = descr.toString();
} else {
assertFalse(ctor.getDeclaringClass().getName() + " has read something in ctor but failed with UOE/IAE", wrapper.readSomething);
@ -710,6 +801,9 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
while (true) {
final Constructor<? extends CharFilter> ctor = charfilters.get(random.nextInt(charfilters.size()));
final Object args[] = newCharFilterArgs(random, spec.reader, ctor.getParameterTypes());
if (broken(ctor, args)) {
continue;
}
reader = createComponent(ctor, args, descr);
if (reader != null) {
spec.reader = reader;
@ -746,11 +840,12 @@ public class TestRandomChains extends BaseTokenStreamTestCase {
}
final Object args[] = newFilterArgs(random, spec.stream, ctor.getParameterTypes());
if (broken(ctor, args)) {
continue;
}
final TokenFilter flt = createComponent(ctor, args, descr);
if (flt != null) {
if (brokenOffsetsComponents.contains(ctor.getDeclaringClass())) {
spec.offsetsAreCorrect = false;
}
spec.offsetsAreCorrect &= !brokenOffsets(ctor, args);
spec.stream = flt;
break;
}

View File

@ -132,7 +132,7 @@ public class TokenInfoDictionaryBuilder {
System.out.println(" encode...");
PositiveIntOutputs fstOutput = PositiveIntOutputs.getSingleton(true);
Builder<Long> fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, null, true);
Builder<Long> fstBuilder = new Builder<Long>(FST.INPUT_TYPE.BYTE2, 0, 0, true, true, Integer.MAX_VALUE, fstOutput, null, true, true);
IntsRef scratch = new IntsRef();
long ord = -1; // first ord will be 0
String lastValue = null;

View File

@ -458,7 +458,20 @@
<!-- ================================================================== -->
<target name="dist-src" depends="package-tgz-src"/>
<target name="dist-all" depends="dist, dist-src"/>
<target name="dist-all" depends="dist, dist-src, -dist-changes, -dist-keys"/>
<!-- copy changes/ to the release folder -->
<target name="-dist-changes">
<copy todir="${dist.dir}/changes">
<fileset dir="${build.dir}/docs/changes"/>
</copy>
</target>
<!-- copy KEYS to the release folder -->
<target name="-dist-keys">
<get src="http://people.apache.org/keys/group/lucene.asc"
dest="${dist.dir}/KEYS"/>
</target>
<target name="copy-to-stage">
<copy-to-stage-macro artifacts.dir="${dist.dir}"/>

View File

@ -24,23 +24,25 @@
<import file="../module-build.xml"/>
<path id="base.classpath">
<pathelement location="${common.dir}/build/core/classes/java"/>
<path id="classpath">
<path refid="base.classpath"/>
<pathelement path="${lucene-core.jar}"/>
<pathelement path="${queries.jar}"/>
<pathelement path="${project.classpath}"/>
<pathelement location="${build.dir}/classes/java" />
</path>
<path id="test.classpath">
<pathelement path="${analyzers-common.jar}"/>
<pathelement location="${common.dir}/build/test-framework/classes/java"/>
<pathelement location="${common.dir}/build/codecs/classes/java"/>
<path refid="classpath"/>
<path refid="junit-path"/>
<pathelement location="${build.dir}/classes/java"/>
<pathelement location="${test-framework.jar}"/>
<pathelement location="${codecs.jar}"/>
<path refid="test.base.classpath"/>
</path>
<target name="compile-core" depends="jar-queries,jar-analyzers-common,common.compile-core" />
<target name="jar-core" depends="common.jar-core" />
<target name="javadocs" depends="javadocs-queries,compile-core">
<invoke-module-javadoc>
<links>

View File

@ -113,7 +113,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
this.field = field;
this.doPackFST = doPackFST;
this.acceptableOverheadRatio = acceptableOverheadRatio;
builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doPackFST, acceptableOverheadRatio);
builder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doPackFST, acceptableOverheadRatio, true);
}
private class PostingsWriter extends PostingsConsumer {

View File

@ -419,7 +419,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
final ByteSequenceOutputs outputs = ByteSequenceOutputs.getSingleton();
final Builder<BytesRef> indexBuilder = new Builder<BytesRef>(FST.INPUT_TYPE.BYTE1,
0, 0, true, false, Integer.MAX_VALUE,
outputs, null, false);
outputs, null, false, true);
//if (DEBUG) {
// System.out.println(" compile index for prefix=" + prefix);
//}
@ -962,7 +962,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
0, 0, true,
true, Integer.MAX_VALUE,
noOutputs,
new FindBlocks(), false);
new FindBlocks(), false, true);
postingsWriter.setField(fieldInfo);
}

View File

@ -55,7 +55,10 @@ public abstract class StoredFieldsWriter implements Closeable {
* called even if the document has no stored fields, in
* this case <code>numStoredFields</code> will be zero. */
public abstract void startDocument(int numStoredFields) throws IOException;
/** Called when a document and all its fields have been added. */
public void finishDocument() throws IOException {}
/** Writes a single stored field. */
public abstract void writeField(FieldInfo info, StorableField field) throws IOException;
@ -116,6 +119,8 @@ public abstract class StoredFieldsWriter implements Closeable {
for (StorableField field : doc) {
writeField(fieldInfos.fieldInfo(field.name()), field);
}
finishDocument();
}
@Override

View File

@ -71,18 +71,27 @@ public abstract class TermVectorsWriter implements Closeable {
* has no vector fields, in this case <code>numVectorFields</code>
* will be zero. */
public abstract void startDocument(int numVectorFields) throws IOException;
/** Called after a doc and all its fields have been added. */
public void finishDocument() throws IOException {};
/** Called before writing the terms of the field.
* {@link #startTerm(BytesRef, int)} will be called <code>numTerms</code> times. */
public abstract void startField(FieldInfo info, int numTerms, boolean positions, boolean offsets, boolean payloads) throws IOException;
/** Called after a field and all its terms have been added. */
public void finishField() throws IOException {};
/** Adds a term and its term frequency <code>freq</code>.
* If this field has positions and/or offsets enabled, then
* {@link #addPosition(int, int, int, BytesRef)} will be called
* <code>freq</code> times respectively.
*/
public abstract void startTerm(BytesRef term, int freq) throws IOException;
/** Called after a term and all its positions have been added. */
public void finishTerm() throws IOException {}
/** Adds a term position and offsets */
public abstract void addPosition(int position, int startOffset, int endOffset, BytesRef payload) throws IOException;
@ -97,7 +106,7 @@ public abstract class TermVectorsWriter implements Closeable {
* check that this is the case to detect the JRE bug described
* in LUCENE-1282. */
public abstract void finish(FieldInfos fis, int numDocs) throws IOException;
/**
* Called by IndexWriter when writing new segments.
* <p>
@ -197,6 +206,7 @@ public abstract class TermVectorsWriter implements Closeable {
protected final void addAllDocVectors(Fields vectors, MergeState mergeState) throws IOException {
if (vectors == null) {
startDocument(0);
finishDocument();
return;
}
@ -275,10 +285,13 @@ public abstract class TermVectorsWriter implements Closeable {
addPosition(pos, startOffset, endOffset, payload);
}
}
finishTerm();
}
assert termCount == numTerms;
finishField();
}
assert fieldCount == numFields;
finishDocument();
}
/** Return the BytesRef Comparator used to sort terms

View File

@ -395,8 +395,10 @@ public final class CompressingStoredFieldsReader extends StoredFieldsReader {
* Copy compressed data.
*/
void copyCompressedData(DataOutput out) throws IOException {
final int chunkSize = chunkSize();
decompressor.copyCompressedData(fieldsStream, chunkSize, out);
final long chunkEnd = docBase + chunkDocs == numDocs
? fieldsStream.length()
: indexReader.getStartPointer(docBase + chunkDocs);
out.copyBytes(fieldsStream, chunkEnd - fieldsStream.getFilePointer());
}
}

View File

@ -136,19 +136,8 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
}
}
private void endWithPreviousDocument() throws IOException {
if (numBufferedDocs > 0) {
endOffsets[numBufferedDocs - 1] = bufferedDocs.length;
}
}
@Override
public void startDocument(int numStoredFields) throws IOException {
endWithPreviousDocument();
if (triggerFlush()) {
flush();
}
if (numBufferedDocs == this.numStoredFields.length) {
final int newLength = ArrayUtil.oversize(numBufferedDocs + 1, 4);
this.numStoredFields = Arrays.copyOf(this.numStoredFields, newLength);
@ -158,6 +147,14 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
++numBufferedDocs;
}
@Override
public void finishDocument() throws IOException {
endOffsets[numBufferedDocs - 1] = bufferedDocs.length;
if (triggerFlush()) {
flush();
}
}
private static void saveInts(int[] values, int length, DataOutput out) throws IOException {
assert length > 0;
if (length == 1) {
@ -295,9 +292,10 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
@Override
public void finish(FieldInfos fis, int numDocs) throws IOException {
endWithPreviousDocument();
if (numBufferedDocs > 0) {
flush();
} else {
assert bufferedDocs.length == 0;
}
if (docBase != numDocs) {
throw new RuntimeException("Wrote " + docBase + " docs, finish called with numDocs=" + numDocs);
@ -351,17 +349,13 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
}
if (compressionMode == matchingFieldsReader.getCompressionMode() // same compression mode
&& (numBufferedDocs == 0 || triggerFlush()) // starting a new chunk
&& numBufferedDocs == 0 // starting a new chunk
&& startOffsets[it.chunkDocs - 1] < chunkSize // chunk is small enough
&& startOffsets[it.chunkDocs - 1] + it.lengths[it.chunkDocs - 1] >= chunkSize // chunk is large enough
&& nextDeletedDoc(it.docBase, liveDocs, it.docBase + it.chunkDocs) == it.docBase + it.chunkDocs) { // no deletion in the chunk
assert docID == it.docBase;
// no need to decompress, just copy data
endWithPreviousDocument();
if (triggerFlush()) {
flush();
}
indexWriter.writeIndex(it.chunkDocs, fieldsStream.getFilePointer());
writeHeader(this.docBase, it.chunkDocs, it.numStoredFields, it.lengths);
it.copyCompressedData(fieldsStream);
@ -380,6 +374,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
final int diff = docID - it.docBase;
startDocument(it.numStoredFields[diff]);
bufferedDocs.writeBytes(it.bytes.bytes, it.bytes.offset + startOffsets[diff], it.lengths[diff]);
finishDocument();
++docCount;
mergeState.checkAbort.work(300);
}

View File

@ -140,14 +140,6 @@ public abstract class CompressionMode {
bytes.length = length;
}
@Override
public void copyCompressedData(DataInput in, int originalLength, DataOutput out) throws IOException {
final int copied = LZ4.copyCompressedData(in, originalLength, out);
if (copied != originalLength) {
throw new CorruptIndexException("Currupted compressed stream: expected " + originalLength + " bytes, but got at least" + copied);
}
}
@Override
public Decompressor clone() {
return this;
@ -224,13 +216,6 @@ public abstract class CompressionMode {
bytes.length = length;
}
@Override
public void copyCompressedData(DataInput in, int originalLength, DataOutput out) throws IOException {
final int compressedLength = in.readVInt();
out.writeVInt(compressedLength);
out.copyBytes(in, compressedLength);
}
@Override
public Decompressor clone() {
return new DeflateDecompressor();

View File

@ -24,7 +24,10 @@ import org.apache.lucene.store.DataOutput;
/**
* A data compressor.
*/
abstract class Compressor {
public abstract class Compressor {
/** Sole constructor, typically called from sub-classes. */
protected Compressor() {}
/**
* Compress bytes into <code>out</code>. It it the responsibility of the

View File

@ -20,13 +20,15 @@ package org.apache.lucene.codecs.compressing;
import java.io.IOException;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.BytesRef;
/**
* An decompressor.
* A decompressor.
*/
abstract class Decompressor implements Cloneable {
public abstract class Decompressor implements Cloneable {
/** Sole constructor, typically called from sub-classes. */
protected Decompressor() {}
/**
* Decompress bytes that were stored between offsets <code>offset</code> and
@ -44,10 +46,6 @@ abstract class Decompressor implements Cloneable {
*/
public abstract void decompress(DataInput in, int originalLength, int offset, int length, BytesRef bytes) throws IOException;
/** Copy a compressed stream whose original length is
* <code>originalLength</code> from <code>in</code> to <code>out</code>. */
public abstract void copyCompressedData(DataInput in, int originalLength, DataOutput out) throws IOException;
@Override
public abstract Decompressor clone();

View File

@ -506,51 +506,4 @@ class LZ4 {
encodeLastLiterals(src, anchor, srcEnd - anchor, out);
}
/** Copy bytes from <code>in</code> to <code>out</code> where
* <code>in</code> is a LZ4-encoded stream. This method copies enough bytes
* so that <code>out</code> can be used later on to restore the first
* <code>length</code> bytes of the stream. This method always reads at
* least one byte from <code>in</code> so make sure not to call this method
* if <code>in</code> reached the end of the stream, even if
* <code>length=0</code>. */
public static int copyCompressedData(DataInput in, int length, DataOutput out) throws IOException {
int n = 0;
do {
// literals
final byte token = in.readByte();
out.writeByte(token);
int literalLen = (token & 0xFF) >>> 4;
if (literalLen == 0x0F) {
byte len;
while ((len = in.readByte()) == (byte) 0xFF) {
literalLen += 0xFF;
out.writeByte(len);
}
literalLen += len & 0xFF;
out.writeByte(len);
}
out.copyBytes(in, literalLen);
n += literalLen;
if (n >= length) {
break;
}
// matchs
out.copyBytes(in, 2); // match dec
int matchLen = token & 0x0F;
if (matchLen == 0x0F) {
byte len;
while ((len = in.readByte()) == (byte) 0xFF) {
matchLen += 0xFF;
out.writeByte(len);
}
matchLen += len & 0xFF;
out.writeByte(len);
}
matchLen += MIN_MATCH;
n += matchLen;
} while (n < length);
return n;
}
}

View File

@ -124,17 +124,16 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
if (payloads)
bits |= Lucene40TermVectorsReader.STORE_PAYLOAD_WITH_TERMVECTOR;
tvf.writeByte(bits);
assert fieldCount <= numVectorFields;
if (fieldCount == numVectorFields) {
// last field of the document
// this is crazy because the file format is crazy!
for (int i = 1; i < fieldCount; i++) {
tvd.writeVLong(fps[i] - fps[i-1]);
}
}
}
@Override
public void finishDocument() throws IOException {
assert fieldCount == numVectorFields;
for (int i = 1; i < fieldCount; i++) {
tvd.writeVLong(fps[i] - fps[i-1]);
}
}
private final BytesRef lastTerm = new BytesRef(10);
// NOTE: we override addProx, so we don't need to buffer when indexing.
@ -222,20 +221,6 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
}
bufferedIndex++;
// dump buffer if we are done
if (bufferedIndex == bufferedFreq) {
if (payloads) {
tvf.writeBytes(payloadData.bytes, payloadData.offset, payloadData.length);
}
for (int i = 0; i < bufferedIndex; i++) {
if (offsets) {
tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
lastOffset = offsetEndBuffer[i];
}
}
}
} else if (positions) {
// write position delta
writePosition(position - lastPosition, payload);
@ -248,6 +233,25 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
}
}
@Override
public void finishTerm() throws IOException {
if (bufferedIndex > 0) {
// dump buffer
assert positions && (offsets || payloads);
assert bufferedIndex == bufferedFreq;
if (payloads) {
tvf.writeBytes(payloadData.bytes, payloadData.offset, payloadData.length);
}
for (int i = 0; i < bufferedIndex; i++) {
if (offsets) {
tvf.writeVInt(offsetStartBuffer[i] - lastOffset);
tvf.writeVInt(offsetEndBuffer[i] - offsetStartBuffer[i]);
lastOffset = offsetEndBuffer[i];
}
}
}
}
private void writePosition(int delta, BytesRef payload) throws IOException {
if (payloads) {
int payloadLength = payload == null ? 0 : payload.length;

View File

@ -108,6 +108,7 @@ final class StoredFieldsProcessor extends StoredFieldsConsumer {
while(lastDocID < docID) {
fieldsWriter.startDocument(0);
lastDocID++;
fieldsWriter.finishDocument();
}
}
@ -123,6 +124,7 @@ final class StoredFieldsProcessor extends StoredFieldsConsumer {
for (int i = 0; i < numStoredFields; i++) {
fieldsWriter.writeField(fieldInfos[i], storedFields[i]);
}
fieldsWriter.finishDocument();
lastDocID++;
}

View File

@ -78,6 +78,7 @@ final class TermVectorsConsumer extends TermsHashConsumer {
void fill(int docID) throws IOException {
while(lastDocID < docID) {
writer.startDocument(0);
writer.finishDocument();
lastDocID++;
}
}
@ -108,6 +109,7 @@ final class TermVectorsConsumer extends TermsHashConsumer {
for (int i = 0; i < numVectorFields; i++) {
perFields[i].finishDocument();
}
writer.finishDocument();
assert lastDocID == docState.docID: "lastDocID=" + lastDocID + " docState.docID=" + docState.docID;

View File

@ -182,7 +182,9 @@ final class TermVectorsConsumerPerField extends TermsHashConsumerPerField {
}
tv.addProx(freq, posReader, offReader);
}
tv.finishTerm();
}
tv.finishField();
termsHashPerField.reset();

View File

@ -218,7 +218,7 @@ public final class CharsRef implements Comparable<CharsRef>, CharSequence, Clone
if (start < 0 || end > length || start > end) {
throw new IndexOutOfBoundsException();
}
return new CharsRef(chars, offset + start, offset + end);
return new CharsRef(chars, offset + start, end - start);
}
/** @deprecated This comparator is only a transition mechanism */

View File

@ -84,11 +84,11 @@ public class Builder<T> {
/**
* Instantiates an FST/FSA builder without any pruning. A shortcut
* to {@link #Builder(FST.INPUT_TYPE, int, int, boolean,
* boolean, int, Outputs, FreezeTail, boolean)} with
* boolean, int, Outputs, FreezeTail, boolean, boolean)} with
* pruning options turned off.
*/
public Builder(FST.INPUT_TYPE inputType, Outputs<T> outputs) {
this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, false, PackedInts.COMPACT);
this(inputType, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, false, PackedInts.COMPACT, true);
}
/**
@ -97,9 +97,9 @@ public class Builder<T> {
*/
public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
FreezeTail<T> freezeTail, boolean willPackFST) {
FreezeTail<T> freezeTail, boolean willPackFST, boolean allowArrayArcs) {
this(inputType, minSuffixCount1, minSuffixCount2, doShareSuffix, doShareNonSingletonNodes,
shareMaxTailLength, outputs, freezeTail, willPackFST, PackedInts.DEFAULT);
shareMaxTailLength, outputs, freezeTail, willPackFST, PackedInts.DEFAULT, allowArrayArcs);
}
/**
@ -143,10 +143,14 @@ public class Builder<T> {
*
* @param acceptableOverheadRatio How to trade speed for space when building the FST. This option
* is only relevant when doPackFST is true. @see PackedInts#getMutable(int, int, float)
*
* @param allowArrayArcs Pass false to disable the array arc optimization
* while building the FST; this will make the resulting
* FST smaller but slower to traverse.
*/
public Builder(FST.INPUT_TYPE inputType, int minSuffixCount1, int minSuffixCount2, boolean doShareSuffix,
boolean doShareNonSingletonNodes, int shareMaxTailLength, Outputs<T> outputs,
FreezeTail<T> freezeTail, boolean doPackFST, float acceptableOverheadRatio) {
FreezeTail<T> freezeTail, boolean doPackFST, float acceptableOverheadRatio, boolean allowArrayArcs) {
this.minSuffixCount1 = minSuffixCount1;
this.minSuffixCount2 = minSuffixCount2;
this.freezeTail = freezeTail;
@ -154,7 +158,7 @@ public class Builder<T> {
this.shareMaxTailLength = shareMaxTailLength;
this.doPackFST = doPackFST;
this.acceptableOverheadRatio = acceptableOverheadRatio;
fst = new FST<T>(inputType, outputs, doPackFST, acceptableOverheadRatio);
fst = new FST<T>(inputType, outputs, doPackFST, acceptableOverheadRatio, allowArrayArcs);
if (doShareSuffix) {
dedupHash = new NodeHash<T>(fst);
} else {
@ -182,13 +186,6 @@ public class Builder<T> {
return dedupHash == null ? 0 : fst.nodeCount;
}
/** Pass false to disable the array arc optimization
* while building the FST; this will make the resulting
* FST smaller but slower to traverse. */
public void setAllowArrayArcs(boolean b) {
fst.setAllowArrayArcs(b);
}
private CompiledNode compileNode(UnCompiledNode<T> nodeIn, int tailLength) throws IOException {
final int node;
if (dedupHash != null && (doShareNonSingletonNodes || nodeIn.numArcs <= 1) && tailLength <= shareMaxTailLength) {

View File

@ -33,6 +33,7 @@ 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.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
@ -137,16 +138,18 @@ public final class FST<T> {
// if non-null, this FST accepts the empty string and
// produces this output
T emptyOutput;
private byte[] emptyOutputBytes;
// Not private to avoid synthetic access$NNN methods:
byte[] bytes;
int byteUpto = 0;
private int startNode = -1;
public final Outputs<T> outputs;
// Used for the BIT_TARGET_NEXT optimization (whereby
// instead of storing the address of the target node for
// a given arc, we mark a single bit noting that the next
// node in the byte[] is the target node):
private int lastFrozenNode;
private final T NO_OUTPUT;
@ -161,7 +164,7 @@ public final class FST<T> {
/** If arc has this label then that arc is final/accepted */
public static final int END_LABEL = -1;
private boolean allowArrayArcs = true;
private final boolean allowArrayArcs;
private Arc<T> cachedRootArcs[];
@ -262,9 +265,10 @@ public final class FST<T> {
// make a new empty FST, for building; Builder invokes
// this ctor
FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio) {
FST(INPUT_TYPE inputType, Outputs<T> outputs, boolean willPackFST, float acceptableOverheadRatio, boolean allowArrayArcs) {
this.inputType = inputType;
this.outputs = outputs;
this.allowArrayArcs = allowArrayArcs;
bytes = new byte[128];
NO_OUTPUT = outputs.getNoOutput();
if (willPackFST) {
@ -293,14 +297,15 @@ public final class FST<T> {
if (in.readByte() == 1) {
// accepts empty string
int numBytes = in.readVInt();
// messy
bytes = new byte[numBytes];
in.readBytes(bytes, 0, numBytes);
// De-serialize empty-string output:
BytesReader reader;
if (packed) {
reader = getBytesReader(0);
reader = new ForwardBytesReader(bytes, 0);
} else {
reader = getBytesReader(numBytes-1);
reader = new ReverseBytesReader(bytes, bytes.length-1);
}
emptyOutput = outputs.readFinalOutput(reader);
} else {
@ -335,6 +340,11 @@ public final class FST<T> {
NO_OUTPUT = outputs.getNoOutput();
cacheRootArcs();
// NOTE: bogus because this is only used during
// building; we need to break out mutable FST from
// immutable
allowArrayArcs = false;
}
public INPUT_TYPE getInputType() {
@ -412,26 +422,6 @@ public final class FST<T> {
} else {
emptyOutput = v;
}
// TODO: this is messy -- replace with sillyBytesWriter; maybe make
// bytes private
final int posSave = writer.getPosition();
outputs.writeFinalOutput(emptyOutput, writer);
emptyOutputBytes = new byte[writer.getPosition()-posSave];
if (!packed) {
// reverse
final int stopAt = (writer.getPosition() - posSave)/2;
int upto = 0;
while(upto < stopAt) {
final byte b = bytes[posSave + upto];
bytes[posSave+upto] = bytes[writer.getPosition()-upto-1];
bytes[writer.getPosition()-upto-1] = b;
upto++;
}
}
System.arraycopy(bytes, posSave, emptyOutputBytes, 0, writer.getPosition()-posSave);
writer.setPosition(posSave);
}
public void save(DataOutput out) throws IOException {
@ -453,7 +443,27 @@ public final class FST<T> {
// TODO: really we should encode this as an arc, arriving
// to the root node, instead of special casing here:
if (emptyOutput != null) {
// Accepts empty string
out.writeByte((byte) 1);
// Serialize empty-string output:
RAMOutputStream ros = new RAMOutputStream();
outputs.writeFinalOutput(emptyOutput, ros);
byte[] emptyOutputBytes = new byte[(int) ros.getFilePointer()];
ros.writeTo(emptyOutputBytes, 0);
if (!packed) {
// reverse
final int stopAt = emptyOutputBytes.length/2;
int upto = 0;
while(upto < stopAt) {
final byte b = emptyOutputBytes[upto];
emptyOutputBytes[upto] = emptyOutputBytes[emptyOutputBytes.length-upto-1];
emptyOutputBytes[emptyOutputBytes.length-upto-1] = b;
upto++;
}
}
out.writeVInt(emptyOutputBytes.length);
out.writeBytes(emptyOutputBytes, 0, emptyOutputBytes.length);
} else {
@ -1160,10 +1170,6 @@ public final class FST<T> {
return arcWithOutputCount;
}
public void setAllowArrayArcs(boolean v) {
allowArrayArcs = v;
}
/**
* Nodes will be expanded if their depth (distance from the root node) is
* &lt;= this value and their number of arcs is &gt;=
@ -1453,6 +1459,11 @@ public final class FST<T> {
this.outputs = outputs;
NO_OUTPUT = outputs.getNoOutput();
writer = new DefaultBytesWriter();
// NOTE: bogus because this is only used during
// building; we need to break out mutable FST from
// immutable
allowArrayArcs = false;
}
/** Expert: creates an FST by packing this one. This

View File

@ -0,0 +1,227 @@
package org.apache.lucene.util.packed;
/*
* 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 static org.apache.lucene.util.packed.BlockPackedWriter.BPV_SHIFT;
import static org.apache.lucene.util.packed.BlockPackedWriter.MIN_VALUE_EQUALS_0;
import static org.apache.lucene.util.packed.BlockPackedWriter.checkBlockSize;
import java.io.EOFException;
import java.io.IOException;
import java.util.Arrays;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.LongsRef;
/**
* Reader for sequences of longs written with {@link BlockPackedWriter}.
* @see BlockPackedWriter
* @lucene.internal
*/
public final class BlockPackedReader {
static long zigZagDecode(long n) {
return ((n >>> 1) ^ -(n & 1));
}
// same as DataInput.readVLong but supports negative values
static long readVLong(DataInput in) throws IOException {
byte b = in.readByte();
if (b >= 0) return b;
long i = b & 0x7FL;
b = in.readByte();
i |= (b & 0x7FL) << 7;
if (b >= 0) return i;
b = in.readByte();
i |= (b & 0x7FL) << 14;
if (b >= 0) return i;
b = in.readByte();
i |= (b & 0x7FL) << 21;
if (b >= 0) return i;
b = in.readByte();
i |= (b & 0x7FL) << 28;
if (b >= 0) return i;
b = in.readByte();
i |= (b & 0x7FL) << 35;
if (b >= 0) return i;
b = in.readByte();
i |= (b & 0x7FL) << 42;
if (b >= 0) return i;
b = in.readByte();
i |= (b & 0x7FL) << 49;
if (b >= 0) return i;
b = in.readByte();
i |= (b & 0xFFL) << 56;
return i;
}
final DataInput in;
final int packedIntsVersion;
final long valueCount;
final int blockSize;
final LongsRef values;
byte[] blocks;
int off;
long ord;
/** Sole constructor.
* @param blockSize the number of values of a block, must be equal to the
* block size of the {@link BlockPackedWriter} which has
* been used to write the stream
*/
public BlockPackedReader(DataInput in, int packedIntsVersion, int blockSize, long valueCount) {
checkBlockSize(blockSize);
this.in = in;
this.packedIntsVersion = packedIntsVersion;
this.blockSize = blockSize;
this.values = new LongsRef(blockSize);
assert valueCount >= 0;
this.valueCount = valueCount;
off = blockSize;
ord = 0;
}
/** Skip exactly <code>count</code> values. */
public void skip(long count) throws IOException {
assert count >= 0;
if (ord + count > valueCount || ord + count < 0) {
throw new EOFException();
}
// 1. skip buffered values
final int skipBuffer = (int) Math.min(count, blockSize - off);
off += skipBuffer;
ord += skipBuffer;
count -= skipBuffer;
if (count == 0L) {
return;
}
// 2. skip as many blocks as necessary
assert off == blockSize;
while (count >= blockSize) {
final int token = in.readByte() & 0xFF;
final int bitsPerValue = token >>> BPV_SHIFT;
if (bitsPerValue > 64) {
throw new IOException("Corrupted");
}
if ((token & MIN_VALUE_EQUALS_0) == 0) {
readVLong(in);
}
final long blockBytes = PackedInts.Format.PACKED.byteCount(packedIntsVersion, blockSize, bitsPerValue);
skipBytes(blockBytes);
ord += blockSize;
count -= blockSize;
}
if (count == 0L) {
return;
}
// 3. skip last values
assert count < blockSize;
refill();
ord += count;
off += count;
}
private void skipBytes(long count) throws IOException {
if (in instanceof IndexInput) {
final IndexInput iin = (IndexInput) in;
iin.seek(iin.getFilePointer() + count);
} else {
if (blocks == null) {
blocks = new byte[blockSize];
}
long skipped = 0;
while (skipped < count) {
final int toSkip = (int) Math.min(blocks.length, count - skipped);
in.readBytes(blocks, 0, toSkip);
skipped += toSkip;
}
}
}
/** Read the next value. */
public long next() throws IOException {
next(1);
assert values.length == 1;
return values.longs[values.offset];
}
/** Read between <tt>1</tt> and <code>count</code> values. */
public LongsRef next(int count) throws IOException {
assert count > 0;
if (ord == valueCount) {
throw new EOFException();
}
if (off == blockSize) {
refill();
}
count = Math.min(count, blockSize - off);
count = (int) Math.min(count, valueCount - ord);
values.offset = off;
values.length = count;
off += count;
ord += count;
return values;
}
private void refill() throws IOException {
final int token = in.readByte() & 0xFF;
final boolean minEquals0 = (token & MIN_VALUE_EQUALS_0) != 0;
final int bitsPerValue = token >>> BPV_SHIFT;
if (bitsPerValue > 64) {
throw new IOException("Corrupted");
}
final long minValue = minEquals0 ? 0L : zigZagDecode(1L + readVLong(in));
assert minEquals0 || minValue != 0;
if (bitsPerValue == 0) {
Arrays.fill(values.longs, minValue);
} else {
final PackedInts.Decoder decoder = PackedInts.getDecoder(PackedInts.Format.PACKED, packedIntsVersion, bitsPerValue);
final int iterations = blockSize / decoder.valueCount();
final int blocksSize = iterations * 8 * decoder.blockCount();
if (blocks == null || blocks.length < blocksSize) {
blocks = new byte[blocksSize];
}
final int valueCount = (int) Math.min(this.valueCount - ord, blockSize);
final int blocksCount = (int) PackedInts.Format.PACKED.byteCount(packedIntsVersion, valueCount, bitsPerValue);
in.readBytes(blocks, 0, blocksCount);
decoder.decode(blocks, 0, values.longs, 0, iterations);
if (minValue != 0) {
for (int i = 0; i < valueCount; ++i) {
values.longs[i] += minValue;
}
}
}
off = 0;
}
/** Return the offset of the next value to read. */
public long ord() {
return ord;
}
}

View File

@ -0,0 +1,164 @@
package org.apache.lucene.util.packed;
/*
* 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.util.Arrays;
import org.apache.lucene.store.DataOutput;
/**
* A writer for large sequences of longs.
* <p>
* The sequence is divided into fixed-size blocks and for each block, the
* difference between each value and the minimum value of the block is encoded
* using as few bits as possible. Memory usage of this class is proportional to
* the block size. Each block has an overhead between 1 and 10 bytes to store
* the minimum value and the number of bits per value of the block.
* @see BlockPackedReader
* @lucene.internal
*/
public final class BlockPackedWriter {
static final int MAX_BLOCK_SIZE = 1 << (30 - 3);
static final int MIN_VALUE_EQUALS_0 = 1 << 0;
static final int BPV_SHIFT = 1;
static void checkBlockSize(int blockSize) {
if (blockSize <= 0 || blockSize > MAX_BLOCK_SIZE) {
throw new IllegalArgumentException("blockSize must be > 0 and < " + MAX_BLOCK_SIZE + ", got " + blockSize);
}
if (blockSize % 64 != 0) {
throw new IllegalArgumentException("blockSize must be a multiple of 64, got " + blockSize);
}
}
static long zigZagEncode(long n) {
return (n >> 63) ^ (n << 1);
}
// same as DataOutput.writeVLong but accepts negative values
static void writeVLong(DataOutput out, long i) throws IOException {
int k = 0;
while ((i & ~0x7FL) != 0L && k++ < 8) {
out.writeByte((byte)((i & 0x7FL) | 0x80L));
i >>>= 7;
}
out.writeByte((byte) i);
}
final DataOutput out;
final long[] values;
byte[] blocks;
int off;
long ord;
boolean finished;
/**
* Sole constructor.
* @param blockSize the number of values of a single block, must be a multiple of <tt>64</tt>
*/
public BlockPackedWriter(DataOutput out, int blockSize) {
checkBlockSize(blockSize);
this.out = out;
values = new long[blockSize];
off = 0;
ord = 0L;
finished = false;
}
private void checkNotFinished() {
if (finished) {
throw new IllegalStateException("Already finished");
}
}
/** Append a new long. */
public void add(long l) throws IOException {
checkNotFinished();
if (off == values.length) {
flush();
}
values[off++] = l;
++ord;
}
/** Flush all buffered data to disk. This instance is not usable anymore
* after this method has been called. */
public void finish() throws IOException {
checkNotFinished();
if (off > 0) {
flush();
}
finished = true;
}
private void flush() throws IOException {
assert off > 0;
long min = Long.MAX_VALUE, max = Long.MIN_VALUE;
for (int i = 0; i < off; ++i) {
min = Math.min(values[i], min);
max = Math.max(values[i], max);
}
final long delta = max - min;
final int bitsRequired = delta < 0 ? 64 : delta == 0L ? 0 : PackedInts.bitsRequired(delta);
if (bitsRequired == 64) {
// no need to delta-encode
min = 0L;
} else if (min > 0L) {
// make min as small as possible so that writeVLong requires fewer bytes
min = Math.max(0L, max - PackedInts.maxValue(bitsRequired));
}
final int token = (bitsRequired << BPV_SHIFT) | (min == 0 ? MIN_VALUE_EQUALS_0 : 0);
out.writeByte((byte) token);
if (min != 0) {
writeVLong(out, zigZagEncode(min) - 1);
}
if (bitsRequired > 0) {
if (min != 0) {
for (int i = 0; i < off; ++i) {
values[i] -= min;
}
}
final PackedInts.Encoder encoder = PackedInts.getEncoder(PackedInts.Format.PACKED, PackedInts.VERSION_CURRENT, bitsRequired);
final int iterations = values.length / encoder.valueCount();
final int blockSize = encoder.blockCount() * 8 * iterations;
if (blocks == null || blocks.length < blockSize) {
blocks = new byte[blockSize];
}
if (off < values.length) {
Arrays.fill(values, off, values.length, 0L);
}
encoder.encode(values, 0, blocks, 0, iterations);
final int blockCount = (int) PackedInts.Format.PACKED.byteCount(PackedInts.VERSION_CURRENT, off, bitsRequired);
out.writeBytes(blocks, blockCount);
}
off = 0;
}
/** Return the number of values which have been added. */
public long ord() {
return ord;
}
}

View File

@ -80,16 +80,6 @@ public abstract class AbstractTestCompressionMode extends LuceneTestCase {
return Arrays.copyOfRange(bytes.bytes, bytes.offset, bytes.offset + bytes.length);
}
static byte[] copyCompressedData(Decompressor decompressor, byte[] compressed, int originalLength) throws IOException {
GrowableByteArrayDataOutput out = new GrowableByteArrayDataOutput(compressed.length);
decompressor.copyCompressedData(new ByteArrayDataInput(compressed), originalLength, out);
return Arrays.copyOf(out.bytes, out.length);
}
byte[] copyCompressedData(byte[] compressed, int originalLength) throws IOException {
return copyCompressedData(mode.newDecompressor(), compressed, originalLength);
}
public void testDecompress() throws IOException {
final int iterations = atLeast(10);
for (int i = 0; i < iterations; ++i) {
@ -117,17 +107,10 @@ public abstract class AbstractTestCompressionMode extends LuceneTestCase {
}
}
public void testCopyCompressedData() throws IOException {
final byte[] decompressed = randomArray();
final byte[] compressed = compress(decompressed);
assertArrayEquals(compressed, copyCompressedData(compressed, decompressed.length));
}
public byte[] test(byte[] decompressed) throws IOException {
final byte[] compressed = compress(decompressed);
final byte[] restored = decompress(compressed, decompressed.length);
assertEquals(decompressed.length, restored.length);
assertArrayEquals(compressed, copyCompressedData(compressed, decompressed.length));
return compressed;
}

View File

@ -116,11 +116,28 @@ public class TestCharsRef extends LuceneTestCase {
}
// LUCENE-3590: fix off-by-one in subsequence, and fully obey interface
// LUCENE-4671: fix subSequence
public void testCharSequenceSubSequence() {
CharSequence c = new CharsRef("abc");
CharSequence sequences[] = {
new CharsRef("abc"),
new CharsRef("0abc".toCharArray(), 1, 3),
new CharsRef("abc0".toCharArray(), 0, 3),
new CharsRef("0abc0".toCharArray(), 1, 3)
};
for (CharSequence c : sequences) {
doTestSequence(c);
}
}
private void doTestSequence(CharSequence c) {
// slice
assertEquals("a", c.subSequence(0, 1).toString());
// mid subsequence
assertEquals("b", c.subSequence(1, 2).toString());
// end subsequence
assertEquals("bc", c.subSequence(1, 3).toString());
// empty subsequence
assertEquals("", c.subSequence(0, 0).toString());

View File

@ -310,7 +310,7 @@ public class TestFSTs extends LuceneTestCase {
final boolean doRewrite = random().nextBoolean();
Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite);
Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, doRewrite, true);
boolean storeOrd = random().nextBoolean();
if (VERBOSE) {
@ -453,8 +453,7 @@ public class TestFSTs extends LuceneTestCase {
this.outputs = outputs;
this.doPack = doPack;
builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack);
builder.setAllowArrayArcs(!noArcArrays);
builder = new Builder<T>(inputMode == 0 ? FST.INPUT_TYPE.BYTE1 : FST.INPUT_TYPE.BYTE4, 0, prune, prune == 0, true, Integer.MAX_VALUE, outputs, null, doPack, !noArcArrays);
}
protected abstract T getOutput(IntsRef input, int ord) throws IOException;
@ -1063,7 +1062,7 @@ public class TestFSTs extends LuceneTestCase {
public void testFinalOutputOnEndState() throws Exception {
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean());
final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE4, 2, 0, true, true, Integer.MAX_VALUE, outputs, null, random().nextBoolean(), true);
builder.add(Util.toUTF32("stat", new IntsRef()), 17L);
builder.add(Util.toUTF32("station", new IntsRef()), 10L);
final FST<Long> fst = builder.finish();
@ -1078,7 +1077,7 @@ public class TestFSTs extends LuceneTestCase {
public void testInternalFinalState() throws Exception {
final PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
final boolean willRewrite = random().nextBoolean();
final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite);
final Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, 0, 0, true, true, Integer.MAX_VALUE, outputs, null, willRewrite, true);
builder.add(Util.toIntsRef(new BytesRef("stat"), new IntsRef()), outputs.getNoOutput());
builder.add(Util.toIntsRef(new BytesRef("station"), new IntsRef()), outputs.getNoOutput());
final FST<Long> fst = builder.finish();
@ -1101,7 +1100,7 @@ public class TestFSTs extends LuceneTestCase {
final Long nothing = outputs.getNoOutput();
final Builder<Long> b = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT);
final FST<Long> fst = new FST<Long>(FST.INPUT_TYPE.BYTE1, outputs, false, PackedInts.COMPACT, true);
final Builder.UnCompiledNode<Long> rootNode = new Builder.UnCompiledNode<Long>(b, 0);

View File

@ -27,6 +27,8 @@ import java.util.Locale;
import java.util.Random;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.store.ByteArrayDataInput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@ -875,4 +877,102 @@ public class TestPackedInts extends LuceneTestCase {
in.close();
dir.close();
}
public void testBlockPackedReaderWriter() throws IOException {
final int iters = atLeast(2);
for (int iter = 0; iter < iters; ++iter) {
final int blockSize = 64 * _TestUtil.nextInt(random(), 1, 1 << 12);
final int valueCount = random().nextInt(1 << 18);
final long[] values = new long[valueCount];
long minValue = 0;
int bpv = 0;
for (int i = 0; i < valueCount; ++i) {
if (i % blockSize == 0) {
minValue = rarely() ? random().nextInt(256) : rarely() ? -5 : random().nextLong();
bpv = random().nextInt(65);
}
if (bpv == 0) {
values[i] = minValue;
} else if (bpv == 64) {
values[i] = random().nextLong();
} else {
values[i] = minValue + _TestUtil.nextLong(random(), 0, (1L << bpv) - 1);
}
}
final Directory dir = newDirectory();
final IndexOutput out = dir.createOutput("out.bin", IOContext.DEFAULT);
final BlockPackedWriter writer = new BlockPackedWriter(out, blockSize);
for (int i = 0; i < valueCount; ++i) {
assertEquals(i, writer.ord());
writer.add(values[i]);
}
assertEquals(valueCount, writer.ord());
writer.finish();
assertEquals(valueCount, writer.ord());
final long fp = out.getFilePointer();
out.close();
DataInput in = dir.openInput("out.bin", IOContext.DEFAULT);
if (random().nextBoolean()) {
byte[] buf = new byte[(int) fp];
in.readBytes(buf, 0, (int) fp);
((IndexInput) in).close();
in = new ByteArrayDataInput(buf);
}
final BlockPackedReader reader = new BlockPackedReader(in, PackedInts.VERSION_CURRENT, blockSize, valueCount);
for (int i = 0; i < valueCount; ) {
if (random().nextBoolean()) {
assertEquals("" + i, values[i], reader.next());
++i;
} else {
final LongsRef nextValues = reader.next(_TestUtil.nextInt(random(), 1, 1024));
for (int j = 0; j < nextValues.length; ++j) {
assertEquals("" + (i + j), values[i + j], nextValues.longs[nextValues.offset + j]);
}
i += nextValues.length;
}
assertEquals(i, reader.ord());
}
assertEquals(fp, in instanceof ByteArrayDataInput ? ((ByteArrayDataInput) in).getPosition() : ((IndexInput) in).getFilePointer());
try {
reader.next();
assertTrue(false);
} catch (IOException e) {
// OK
}
if (in instanceof ByteArrayDataInput) {
((ByteArrayDataInput) in).setPosition(0);
} else {
((IndexInput) in).seek(0L);
}
final BlockPackedReader reader2 = new BlockPackedReader(in, PackedInts.VERSION_CURRENT, blockSize, valueCount);
int i = 0;
while (true) {
final int skip = _TestUtil.nextInt(random(), 0, valueCount - i);
reader2.skip(skip);
i += skip;
assertEquals(i, reader2.ord());
if (i == valueCount) {
break;
} else {
assertEquals(values[i], reader2.next());
++i;
}
}
assertEquals(fp, in instanceof ByteArrayDataInput ? ((ByteArrayDataInput) in).getPosition() : ((IndexInput) in).getFilePointer());
try {
reader2.skip(1);
assertTrue(false);
} catch (IOException e) {
// OK
}
if (in instanceof IndexInput) {
((IndexInput) in).close();
}
dir.close();
}
}
}

View File

@ -81,5 +81,12 @@
</links>
</invoke-module-javadoc>
</target>
<target name="run-encoding-benchmark" depends="compile-test">
<java classname="org.apache.lucene.util.encoding.EncodingSpeed" fork="true" failonerror="true">
<classpath refid="test.classpath" />
<classpath path="${build.dir}/classes/test" />
</java>
</target>
</project>

View File

@ -39,7 +39,7 @@ import org.apache.lucene.store.Directory;
*
* @lucene.experimental
*/
public class AssociationIndexer {
public class CategoryAssociationsIndexer {
/**
* Create an index, and adds to it sample documents and categories.
@ -75,13 +75,11 @@ public class AssociationIndexer {
++nFacetsAdded;
}
// and also those with associations
CategoryPath[] associationsPaths = AssociationUtils.categories[docNum];
CategoryAssociation[] associationsValues = AssociationUtils.associations[docNum];
CategoryPath[] associationsPaths = CategoryAssociationsUtils.categories[docNum];
CategoryAssociation[] associationsValues = CategoryAssociationsUtils.associations[docNum];
for (int i = 0; i < associationsPaths.length; i++) {
associations.setAssociation(associationsPaths[i], associationsValues[i]);
ExampleUtils.log("\t $$$$ Association: ("
+ associationsPaths[i] + "," + associationsValues[i]
+ ")");
ExampleUtils.log("\t $$$$ Association: (" + associationsPaths[i] + "," + associationsValues[i] + ")");
++nFacetsAdded;
}

View File

@ -31,15 +31,15 @@ import org.apache.lucene.facet.search.results.FacetResult;
*
* @lucene.experimental
*/
public class AssociationMain {
public class CategoryAssociationsMain {
/**
* Driver for the simple sample.
* @throws Exception on error (no detailed exception handling here for sample simplicity
*/
public static void main(String[] args) throws Exception {
new AssociationMain().runSumIntAssociationSample();
new AssociationMain().runSumFloatAssociationSample();
new CategoryAssociationsMain().runSumIntAssociationSample();
new CategoryAssociationsMain().runSumFloatAssociationSample();
ExampleUtils.log("DONE");
}
@ -51,10 +51,10 @@ public class AssociationMain {
// index the sample documents
ExampleUtils.log("index the sample documents...");
AssociationIndexer.index(indexDir, taxoDir);
CategoryAssociationsIndexer.index(indexDir, taxoDir);
ExampleUtils.log("search the sample documents...");
List<FacetResult> facetRes = AssociationSearcher.searchSumIntAssociation(indexDir, taxoDir);
List<FacetResult> facetRes = CategoryAssociationsSearcher.searchSumIntAssociation(indexDir, taxoDir);
ExampleResult res = new ExampleResult();
res.setFacetResults(facetRes);
@ -69,10 +69,10 @@ public class AssociationMain {
// index the sample documents
ExampleUtils.log("index the sample documents...");
AssociationIndexer.index(indexDir, taxoDir);
CategoryAssociationsIndexer.index(indexDir, taxoDir);
ExampleUtils.log("search the sample documents...");
List<FacetResult> facetRes = AssociationSearcher.searchSumFloatAssociation(indexDir, taxoDir);
List<FacetResult> facetRes = CategoryAssociationsSearcher.searchSumFloatAssociation(indexDir, taxoDir);
ExampleResult res = new ExampleResult();
res.setFacetResults(facetRes);

View File

@ -37,18 +37,15 @@ import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
*
* @lucene.experimental
*/
public class AssociationSearcher {
public class CategoryAssociationsSearcher {
/** Search an index with a sum of int-association. */
public static List<FacetResult> searchSumIntAssociation(Directory indexDir,
Directory taxoDir) throws Exception {
public static List<FacetResult> searchSumIntAssociation(Directory indexDir, Directory taxoDir) throws Exception {
// prepare index reader
IndexReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
AssociationIntSumFacetRequest facetRequest = new AssociationIntSumFacetRequest(
new CategoryPath("tags"), 10);
AssociationIntSumFacetRequest facetRequest = new AssociationIntSumFacetRequest(new CategoryPath("tags"), 10);
List<FacetResult> res = SimpleSearcher.searchWithRequest(indexReader, taxo, null, facetRequest);
// close readers
@ -59,14 +56,12 @@ public class AssociationSearcher {
}
/** Search an index with a sum of float-association. */
public static List<FacetResult> searchSumFloatAssociation(Directory indexDir,
Directory taxoDir) throws Exception {
public static List<FacetResult> searchSumFloatAssociation(Directory indexDir, Directory taxoDir) throws Exception {
// prepare index reader
IndexReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
AssociationFloatSumFacetRequest facetRequest = new AssociationFloatSumFacetRequest(
new CategoryPath("genre"), 10);
AssociationFloatSumFacetRequest facetRequest = new AssociationFloatSumFacetRequest(new CategoryPath("genre"), 10);
List<FacetResult> res = SimpleSearcher.searchWithRequest(indexReader, taxo, null, facetRequest);

View File

@ -25,7 +25,7 @@ import org.apache.lucene.facet.taxonomy.CategoryPath;
/**
* @lucene.experimental
*/
public class AssociationUtils {
public class CategoryAssociationsUtils {
/**
* Categories: categories[D][N] == category-path with association no. N for

View File

@ -1,11 +1,7 @@
package org.apache.lucene.facet.example.simple;
import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
import org.apache.lucene.facet.example.ExampleUtils;
import org.apache.lucene.facet.taxonomy.CategoryPath;

View File

@ -1,92 +0,0 @@
package org.apache.lucene.facet.associations;
import java.io.IOException;
import java.util.HashMap;
import org.apache.lucene.facet.index.CategoryListBuilder;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.util.BytesRef;
/*
* 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.
*/
/**
* A {@link CategoryListBuilder} which encodes category-association value pairs
* in addition to regular counting list category ordinals. Every
* category-association pair is written under the respective association's
* {@link CategoryAssociation#getCategoryListID()}.
*/
public class AssociationsCategoryListBuilder extends CategoryListBuilder {
private final CategoryAssociationsContainer associations;
private final HashMap<String,BytesRef> perAssociationBytes = new HashMap<String,BytesRef>();
private final ByteArrayDataOutput output = new ByteArrayDataOutput();
public AssociationsCategoryListBuilder(CategoryAssociationsContainer associations,
CategoryListParams categoryListParams, FacetIndexingParams indexingParams, TaxonomyWriter taxoWriter) {
super(categoryListParams, indexingParams, taxoWriter);
this.associations = associations;
}
@Override
public void handle(int ordinal, CategoryPath cp) throws IOException {
super.handle(ordinal, cp);
// build per-association key BytesRef
CategoryAssociation association = associations.getAssociation(cp);
if (association == null) {
// it is ok to set a null association for a category - it's treated as a
// regular category in that case.
return;
}
BytesRef bytes = perAssociationBytes.get(association.getCategoryListID());
if (bytes == null) {
bytes = new BytesRef();
perAssociationBytes.put(association.getCategoryListID(), bytes);
}
int maxBytesNeeded = 4 /* int */ + association.maxBytesNeeded();
if (bytes.bytes.length - bytes.length < maxBytesNeeded) {
bytes.grow(bytes.bytes.length + maxBytesNeeded);
}
// reset the output to write from bytes.length (current position) until the end
output.reset(bytes.bytes, bytes.length, bytes.bytes.length - bytes.length);
output.writeInt(ordinal);
// encode the association bytes
association.serialize(output);
// update BytesRef
bytes.length = output.getPosition();
}
@Override
public HashMap<String,BytesRef> finish() {
// build the ordinals list
HashMap<String,BytesRef> result = super.finish();
// add per association bytes
result.putAll(perAssociationBytes);
return result;
}
}

View File

@ -7,7 +7,7 @@ import java.util.Map;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.facet.index.CategoryListBuilder;
import org.apache.lucene.facet.index.CountingListBuilder;
import org.apache.lucene.facet.index.DrillDownStream;
import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.index.params.CategoryListParams;
@ -15,6 +15,8 @@ import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -94,15 +96,16 @@ public class AssociationsFacetFields extends FacetFields {
return categoryLists;
}
/**
* Returns a {@link CategoryListBuilder} for encoding the given categories and
* associations.
*/
@Override
protected CategoryListBuilder getCategoryListBuilder(CategoryListParams categoryListParams,
Iterable<CategoryPath> categories) {
return new AssociationsCategoryListBuilder((CategoryAssociationsContainer) categories, categoryListParams,
indexingParams, taxonomyWriter);
protected Map<String,BytesRef> getCategoryListData(CategoryListParams categoryListParams, IntsRef ordinals,
Iterable<CategoryPath> categories) throws IOException {
AssociationsListBuilder associations = new AssociationsListBuilder((CategoryAssociationsContainer) categories);
CountingListBuilder counting = new CountingListBuilder(categoryListParams, indexingParams, taxonomyWriter);
// CountingListBuilder modifies the ordinals array, by e.g. adding parent ordinals, sorting etc.
// Therefore first build the associations list and only afterwards the counting list.
final Map<String,BytesRef> res = associations.build(ordinals, categories);
res.putAll(counting.build(ordinals, categories));
return res;
}
@Override

View File

@ -0,0 +1,89 @@
package org.apache.lucene.facet.associations;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.facet.index.CategoryListBuilder;
import org.apache.lucene.facet.index.CountingListBuilder;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.store.ByteArrayDataOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* 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.
*/
/**
* A {@link AssociationsListBuilder} which encodes category-association value pairs.
* Every category-association pair is written under the respective association's
* {@link CategoryAssociation#getCategoryListID()}.
* <p>
* <b>NOTE:</b> associations list do not encode the counting list data. You
* should use {@link CountingListBuilder} to build that information and then
* merge the results of both {@link #build(IntsRef, Iterable)}.
*/
public class AssociationsListBuilder implements CategoryListBuilder {
private final CategoryAssociationsContainer associations;
private final ByteArrayDataOutput output = new ByteArrayDataOutput();
public AssociationsListBuilder(CategoryAssociationsContainer associations) {
this.associations = associations;
}
@Override
public Map<String,BytesRef> build(IntsRef ordinals, Iterable<CategoryPath> categories) throws IOException {
final HashMap<String,BytesRef> res = new HashMap<String,BytesRef>();
int idx = 0;
for (CategoryPath cp : categories) {
// build per-association key BytesRef
CategoryAssociation association = associations.getAssociation(cp);
if (association == null) {
// it is ok to set a null association for a category - it's treated as a
// regular category in that case.
++idx;
continue;
}
BytesRef bytes = res.get(association.getCategoryListID());
if (bytes == null) {
bytes = new BytesRef(32);
res.put(association.getCategoryListID(), bytes);
}
int maxBytesNeeded = 4 /* int */ + association.maxBytesNeeded() + bytes.length;
if (bytes.bytes.length < maxBytesNeeded) {
bytes.grow(maxBytesNeeded);
}
// reset the output to write from bytes.length (current position) until the end
output.reset(bytes.bytes, bytes.length, bytes.bytes.length - bytes.length);
output.writeInt(ordinals.ints[idx++]);
// encode the association bytes
association.serialize(output);
// update BytesRef
bytes.length = output.getPosition();
}
return res;
}
}

View File

@ -53,20 +53,22 @@ public abstract class AssociationsPayloadIterator<T extends CategoryAssociation>
}
/**
* Skip to the requested document. Returns true iff the document has categort
* association values and they were read successfully.
* Skip to the requested document. Returns true iff the document has category
* association values and they were read successfully. Associations are
* handled through {@link #handleAssociation(int, CategoryAssociation)} by
* extending classes.
*/
public boolean setNextDoc(int docId) throws IOException {
protected final boolean setNextDoc(int docID) throws IOException {
if (!hasAssociations) { // there are no associations at all
return false;
}
if (!pi.setdoc(docId)) { // no associations for the requested document
BytesRef bytes = pi.getPayload(docID);
if (bytes == null) { // no associations for the requested document
return false;
}
BytesRef associations = pi.getPayload();
ByteArrayDataInput in = new ByteArrayDataInput(associations.bytes, associations.offset, associations.length);
ByteArrayDataInput in = new ByteArrayDataInput(bytes.bytes, bytes.offset, bytes.length);
while (!in.eof()) {
int ordinal = in.readInt();
association.deserialize(in);

View File

@ -55,5 +55,10 @@ public class CategoryAssociationsContainer implements Iterable<CategoryPath> {
public void clear() {
categoryAssociations.clear();
}
@Override
public String toString() {
return categoryAssociations.toString();
}
}

View File

@ -71,5 +71,10 @@ public class CategoryFloatAssociation implements CategoryAssociation {
public float getValue() {
return value;
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + value + ")";
}
}

View File

@ -72,4 +72,9 @@ public class CategoryIntAssociation implements CategoryAssociation {
return value;
}
@Override
public String toString() {
return getClass().getSimpleName() + "(" + value + ")";
}
}

View File

@ -40,23 +40,17 @@ public class FloatAssociationsPayloadIterator extends AssociationsPayloadIterato
protected void handleAssociation(int ordinal, CategoryFloatAssociation association) {
ordinalAssociations.put(ordinal, association.getValue());
}
@Override
public boolean setNextDoc(int docId) throws IOException {
ordinalAssociations.clear();
return super.setNextDoc(docId);
}
/**
* Get the float association value for the given ordinal, or
* {@link Float#NaN} in case the ordinal has no association value.
* Returns the float association values of the categories that are associated
* with the given document, or {@code null} if the document has no
* associations.
* <p>
* <b>NOTE:</b> you are not expected to modify the returned map.
*/
public float getAssociation(int ordinal) {
if (ordinalAssociations.containsKey(ordinal)) {
return ordinalAssociations.get(ordinal);
}
return Float.NaN;
public IntToFloatMap getAssociations(int docID) throws IOException {
ordinalAssociations.clear();
return setNextDoc(docID) ? ordinalAssociations : null;
}
}

View File

@ -31,12 +31,6 @@ public class IntAssociationsPayloadIterator extends AssociationsPayloadIterator<
private final IntToIntMap ordinalAssociations = new IntToIntMap();
/**
* The long-special-value returned for ordinals which have no associated int
* value. It is not in the int range of values making it a valid mark.
*/
public final static long NO_ASSOCIATION = Integer.MAX_VALUE + 1;
public IntAssociationsPayloadIterator(IndexReader reader, String field, CategoryIntAssociation association)
throws IOException {
super(reader, field, association);
@ -47,22 +41,16 @@ public class IntAssociationsPayloadIterator extends AssociationsPayloadIterator<
ordinalAssociations.put(ordinal, association.getValue());
}
@Override
public boolean setNextDoc(int docId) throws IOException {
ordinalAssociations.clear();
return super.setNextDoc(docId);
}
/**
* Get the integer association value for the given ordinal, or
* {@link #NO_ASSOCIATION} in case the ordinal has no association value.
* Returns the integer association values of the categories that are
* associated with the given document, or {@code null} if the document has no
* associations.
* <p>
* <b>NOTE:</b> you are not expected to modify the returned map.
*/
public long getAssociation(int ordinal) {
if (ordinalAssociations.containsKey(ordinal)) {
return ordinalAssociations.get(ordinal);
}
return NO_ASSOCIATION;
public IntToIntMap getAssociations(int docID) throws IOException {
ordinalAssociations.clear();
return setNextDoc(docID) ? ordinalAssociations : null;
}
}

View File

@ -1,19 +1,11 @@
package org.apache.lucene.facet.index;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map.Entry;
import java.util.Map;
import org.apache.lucene.facet.index.categorypolicy.OrdinalPolicy;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.facet.util.PartitionsUtils;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.UnsafeByteArrayOutputStream;
import org.apache.lucene.util.encoding.IntEncoder;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -33,149 +25,14 @@ import org.apache.lucene.util.encoding.IntEncoder;
*/
/**
* Builds a category list by encoding the category ordinals into one or more
* {@link BytesRef}. Each {@link BytesRef} corresponds to a set of ordinals that
* belong to the same partition. When partitions are not enabled (i.e.
* {@link FacetIndexingParams#getPartitionSize()} returns
* {@link Integer#MAX_VALUE}), only one {@link BytesRef} is returned by this
* class.
* Builds a category list data by encoding the appropriate information for every
* category and ordinal given to {@link #build(IntsRef, Iterable)}.
*
* @lucene.experimental
*/
public class CategoryListBuilder {
/** Specializes encoding ordinals when partitions are enabled/disabled. */
private static abstract class OrdinalsEncoder {
OrdinalsEncoder() {}
public abstract void encode(int ordinal);
public abstract HashMap<String,BytesRef> finish();
}
public interface CategoryListBuilder {
private static final class NoPartitionsOrdinalsEncoder extends OrdinalsEncoder {
private final IntEncoder encoder;
private final UnsafeByteArrayOutputStream ubaos;
private final String name;
NoPartitionsOrdinalsEncoder(CategoryListParams categoryListParams) {
name = categoryListParams.getTerm().text();
encoder = categoryListParams.createEncoder();
ubaos = new UnsafeByteArrayOutputStream();
encoder.reInit(ubaos);
}
@Override
public void encode(int ordinal) {
try {
encoder.encode(ordinal);
} catch (IOException e) {
// shouldn't happen as we're writing to byte[]
throw new RuntimeException("unexpected exception", e);
}
}
@Override
public HashMap<String,BytesRef> finish() {
try {
encoder.close();
} catch (IOException e) {
// shouldn't happen as we're writing to byte[]
throw new RuntimeException("unexpected exception", e);
}
HashMap<String,BytesRef> result = new HashMap<String,BytesRef>();
result.put(name, new BytesRef(ubaos.toByteArray(), ubaos.getStartPos(), ubaos.length()));
return result;
}
}
private static final class PerPartitionOrdinalsEncoder extends OrdinalsEncoder {
private final FacetIndexingParams indexingParams;
private final CategoryListParams categoryListParams;
private final int partitionSize;
private final HashMap<String,IntEncoder> partitionEncoder = new HashMap<String,IntEncoder>();
private final HashMap<String,UnsafeByteArrayOutputStream> partitionBytes = new HashMap<String,UnsafeByteArrayOutputStream>();
PerPartitionOrdinalsEncoder(FacetIndexingParams indexingParams, CategoryListParams categoryListParams) {
this.indexingParams = indexingParams;
this.categoryListParams = categoryListParams;
this.partitionSize = indexingParams.getPartitionSize();
}
@Override
public void encode(int ordinal) {
final String name = PartitionsUtils.partitionNameByOrdinal(indexingParams, categoryListParams, ordinal);
IntEncoder encoder = partitionEncoder.get(name);
if (encoder == null) {
encoder = categoryListParams.createEncoder();
final UnsafeByteArrayOutputStream ubaos = new UnsafeByteArrayOutputStream();
encoder.reInit(ubaos);
partitionEncoder.put(name, encoder);
partitionBytes.put(name, ubaos);
}
try {
encoder.encode(ordinal % partitionSize);
} catch (IOException e) {
// shouldn't happen as we're writing to byte[]
throw new RuntimeException("unexpected exception", e);
}
}
@Override
public HashMap<String,BytesRef> finish() {
// finish encoding
IOUtils.closeWhileHandlingException(partitionEncoder.values());
HashMap<String,BytesRef> bytes = new HashMap<String,BytesRef>();
for (Entry<String,UnsafeByteArrayOutputStream> e : partitionBytes.entrySet()) {
UnsafeByteArrayOutputStream ubaos = e.getValue();
bytes.put(e.getKey(), new BytesRef(ubaos.toByteArray(), ubaos.getStartPos(), ubaos.length()));
}
return bytes;
}
}
private final TaxonomyWriter taxoWriter;
private final OrdinalsEncoder ordinalsEncoder;
private final OrdinalPolicy ordinalPolicy;
public CategoryListBuilder(CategoryListParams categoryListParams, FacetIndexingParams indexingParams,
TaxonomyWriter taxoWriter) {
this.taxoWriter = taxoWriter;
this.ordinalPolicy = indexingParams.getOrdinalPolicy();
if (indexingParams.getPartitionSize() == Integer.MAX_VALUE) {
ordinalsEncoder = new NoPartitionsOrdinalsEncoder(categoryListParams);
} else {
ordinalsEncoder = new PerPartitionOrdinalsEncoder(indexingParams, categoryListParams);
}
}
/**
* Encodes the given ordinal as well as any of its parent ordinals (per
* {@link OrdinalPolicy}).
*/
public void handle(int ordinal, CategoryPath cp) throws IOException {
ordinalsEncoder.encode(ordinal);
// add all parent ordinals, per OrdinalPolicy
int parent = taxoWriter.getParent(ordinal);
while (parent > 0) {
if (ordinalPolicy.shouldAdd(parent)) {
ordinalsEncoder.encode(parent);
}
parent = taxoWriter.getParent(parent);
}
}
/**
* Returns the encoded ordinals data. Every returned {@link BytesRef}
* corresponds to a single partition (as defined by
* {@link FacetIndexingParams#getPartitionSize()}) and the key denotes the
* partition ID. When no partitions are defined, the returned map includes
* only one value.
*/
public HashMap<String,BytesRef> finish() {
return ordinalsEncoder.finish();
}
/** Returns the encoded ordinals data. */
public Map<String,BytesRef> build(IntsRef ordinals, Iterable<CategoryPath> categories) throws IOException;
}

View File

@ -0,0 +1,160 @@
package org.apache.lucene.facet.index;
import java.io.IOException;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Map.Entry;
import org.apache.lucene.facet.index.categorypolicy.OrdinalPolicy;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.facet.util.PartitionsUtils;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.encoding.IntEncoder;
/*
* 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.
*/
/**
* A {@link CategoryListBuilder} which builds a counting list data by encoding
* the category ordinals into one or more {@link BytesRef}. Each
* {@link BytesRef} corresponds to a set of ordinals that belong to the same
* partition. When partitions are not enabled (i.e.
* {@link FacetIndexingParams#getPartitionSize()} returns
* {@link Integer#MAX_VALUE}), only one {@link BytesRef} is returned by this
* class.
* <p>
* Counting lists are used usually for computing the weight of categories by
* summing their number of occurrences (hence counting) in a result set.
*/
public class CountingListBuilder implements CategoryListBuilder {
/** Specializes encoding ordinals when partitions are enabled/disabled. */
private static abstract class OrdinalsEncoder {
OrdinalsEncoder() {}
public abstract Map<String,BytesRef> encode(IntsRef ordinals);
}
private static final class NoPartitionsOrdinalsEncoder extends OrdinalsEncoder {
private final IntEncoder encoder;
private final String name;
NoPartitionsOrdinalsEncoder(CategoryListParams categoryListParams) {
name = categoryListParams.getTerm().text();
encoder = categoryListParams.createEncoder();
}
@Override
public Map<String,BytesRef> encode(IntsRef ordinals) {
final BytesRef bytes = new BytesRef(128); // should be enough for most common applications
encoder.encode(ordinals, bytes);
return Collections.singletonMap(name, bytes);
}
}
private static final class PerPartitionOrdinalsEncoder extends OrdinalsEncoder {
private final FacetIndexingParams indexingParams;
private final CategoryListParams categoryListParams;
private final int partitionSize;
private final HashMap<String,IntEncoder> partitionEncoder = new HashMap<String,IntEncoder>();
PerPartitionOrdinalsEncoder(FacetIndexingParams indexingParams, CategoryListParams categoryListParams) {
this.indexingParams = indexingParams;
this.categoryListParams = categoryListParams;
this.partitionSize = indexingParams.getPartitionSize();
}
@Override
public HashMap<String,BytesRef> encode(IntsRef ordinals) {
// build the partitionOrdinals map
final HashMap<String,IntsRef> partitionOrdinals = new HashMap<String,IntsRef>();
for (int i = 0; i < ordinals.length; i++) {
int ordinal = ordinals.ints[i];
final String name = PartitionsUtils.partitionNameByOrdinal(indexingParams, categoryListParams, ordinal);
IntsRef partitionOrds = partitionOrdinals.get(name);
if (partitionOrds == null) {
partitionOrds = new IntsRef(32);
partitionOrdinals.put(name, partitionOrds);
partitionEncoder.put(name, categoryListParams.createEncoder());
}
partitionOrds.ints[partitionOrds.length++] = ordinal % partitionSize;
}
HashMap<String,BytesRef> partitionBytes = new HashMap<String,BytesRef>();
for (Entry<String,IntsRef> e : partitionOrdinals.entrySet()) {
String name = e.getKey();
final IntEncoder encoder = partitionEncoder.get(name);
final BytesRef bytes = new BytesRef(128); // should be enough for most common applications
encoder.encode(e.getValue(), bytes);
partitionBytes.put(name, bytes);
}
return partitionBytes;
}
}
private final OrdinalsEncoder ordinalsEncoder;
private final TaxonomyWriter taxoWriter;
private final OrdinalPolicy ordinalPolicy;
public CountingListBuilder(CategoryListParams categoryListParams, FacetIndexingParams indexingParams,
TaxonomyWriter taxoWriter) {
this.taxoWriter = taxoWriter;
this.ordinalPolicy = indexingParams.getOrdinalPolicy();
if (indexingParams.getPartitionSize() == Integer.MAX_VALUE) {
ordinalsEncoder = new NoPartitionsOrdinalsEncoder(categoryListParams);
} else {
ordinalsEncoder = new PerPartitionOrdinalsEncoder(indexingParams, categoryListParams);
}
}
/**
* Every returned {@link BytesRef} corresponds to a single partition (as
* defined by {@link FacetIndexingParams#getPartitionSize()}) and the key
* denotes the partition ID. When no partitions are defined, the returned map
* contains only one value.
* <p>
* <b>NOTE:</b> the {@code ordinals} array is modified by adding parent
* ordinals to it. Also, some encoders may sort the array and remove duplicate
* ordinals. Therefore you may want to invoke this method after you finished
* processing the array for other purposes.
*/
@Override
public Map<String,BytesRef> build(IntsRef ordinals, Iterable<CategoryPath> categories) throws IOException {
int upto = ordinals.length; // since we add ordinals to IntsRef, iterate upto original length
for (int i = 0; i < upto; i++) {
int ordinal = ordinals.ints[i];
int parent = taxoWriter.getParent(ordinal);
while (parent > 0) {
if (ordinalPolicy.shouldAdd(parent)) {
ordinals.ints[ordinals.length++] = parent;
}
parent = taxoWriter.getParent(parent);
}
}
return ordinalsEncoder.encode(ordinals);
}
}

View File

@ -2,6 +2,7 @@ package org.apache.lucene.facet.index;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
@ -21,6 +22,7 @@ import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -69,7 +71,7 @@ public class FacetFields {
return true;
}
void setCategoriesData(HashMap<String,BytesRef> categoriesData) {
void setCategoriesData(Map<String,BytesRef> categoriesData) {
this.categoriesData = categoriesData.entrySet().iterator();
}
@ -132,6 +134,9 @@ public class FacetFields {
*/
protected Map<CategoryListParams,Iterable<CategoryPath>> createCategoryListMapping(
Iterable<CategoryPath> categories) {
if (indexingParams.getAllCategoryListParams().size() == 1) {
return Collections.singletonMap(indexingParams.getCategoryListParams(null), categories);
}
HashMap<CategoryListParams,Iterable<CategoryPath>> categoryLists =
new HashMap<CategoryListParams,Iterable<CategoryPath>>();
for (CategoryPath cp : categories) {
@ -147,10 +152,15 @@ public class FacetFields {
return categoryLists;
}
/** Returns a {@link CategoryListBuilder} for encoding the given categories. */
protected CategoryListBuilder getCategoryListBuilder(CategoryListParams categoryListParams,
Iterable<CategoryPath> categories /* needed for AssociationsFacetFields */) {
return new CategoryListBuilder(categoryListParams, indexingParams, taxonomyWriter);
/**
* Returns the category list data, as a mapping from key to {@link BytesRef}
* which includes the encoded data. Every ordinal in {@code ordinals}
* corrspond to a {@link CategoryPath} returned from {@code categories}.
*/
protected Map<String,BytesRef> getCategoryListData(CategoryListParams categoryListParams,
IntsRef ordinals, Iterable<CategoryPath> categories /* needed for AssociationsFacetFields */)
throws IOException {
return new CountingListBuilder(categoryListParams, indexingParams, taxonomyWriter).build(ordinals, categories);
}
/**
@ -185,17 +195,25 @@ public class FacetFields {
// for each CLP we add a different field for drill-down terms as well as for
// counting list data.
IntsRef ordinals = new IntsRef(32); // should be enough for most common applications
for (Entry<CategoryListParams, Iterable<CategoryPath>> e : categoryLists.entrySet()) {
final CategoryListParams clp = e.getKey();
final String field = clp.getTerm().field();
// add the counting list data
CategoryListBuilder categoriesPayloadBuilder = getCategoryListBuilder(clp, e.getValue());
// build category list data
ordinals.length = 0; // reset
int maxNumOrds = 0;
for (CategoryPath cp : e.getValue()) {
int ordinal = taxonomyWriter.addCategory(cp);
categoriesPayloadBuilder.handle(ordinal , cp);
maxNumOrds += cp.length; // ordinal and potentially all parents
if (ordinals.ints.length < maxNumOrds) {
ordinals.grow(maxNumOrds);
}
ordinals.ints[ordinals.length++] = ordinal;
}
HashMap<String,BytesRef> categoriesData = categoriesPayloadBuilder.finish();
Map<String,BytesRef> categoriesData = getCategoryListData(clp, ordinals, e.getValue());
// add the counting list data
CountingListStream ts = new CountingListStream();
ts.setCategoriesData(categoriesData);
doc.add(new Field(field, ts, COUNTING_LIST_PAYLOAD_TYPE));

View File

@ -17,10 +17,7 @@ package org.apache.lucene.facet.index;
* limitations under the License.
*/
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.InputStream;
import java.util.HashMap;
import java.util.Map;
@ -36,6 +33,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.encoding.IntDecoder;
import org.apache.lucene.util.encoding.IntEncoder;
@ -187,7 +185,7 @@ public class OrdinalMappingAtomicReader extends FilterAtomicReader {
private class OrdinalMappingDocsAndPositionsEnum extends FilterDocsAndPositionsEnum {
private final IntEncoder encoder;
private final IntDecoder decoder;
private final ByteArrayOutputStream os = new ByteArrayOutputStream();
private final IntsRef ordinals = new IntsRef(32);
private final BytesRef payloadOut = new BytesRef();
public OrdinalMappingDocsAndPositionsEnum(DocsAndPositionsEnum in, CategoryListParams params) {
@ -202,21 +200,14 @@ public class OrdinalMappingAtomicReader extends FilterAtomicReader {
if (payload == null) {
return payload;
} else {
InputStream is = new ByteArrayInputStream(payload.bytes, payload.offset, payload.length);
decoder.reInit(is);
os.reset();
encoder.reInit(os);
long ordinal;
while ((ordinal = decoder.decode()) != IntDecoder.EOS) {
int newOrdinal = ordinalMap[(int)ordinal];
encoder.encode(newOrdinal);
decoder.decode(payload, ordinals);
// map the ordinals
for (int i = 0; i < ordinals.length; i++) {
ordinals.ints[i] = ordinalMap[ordinals.ints[i]];
}
encoder.close();
// TODO (Facet): avoid copy?
byte out[] = os.toByteArray();
payloadOut.bytes = out;
payloadOut.offset = 0;
payloadOut.length = out.length;
encoder.encode(ordinals, payloadOut);
return payloadOut;
}
}

View File

@ -7,7 +7,7 @@ import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.facet.search.CategoryListIterator;
import org.apache.lucene.facet.search.PayloadIntDecodingIterator;
import org.apache.lucene.facet.search.PayloadCategoryListIteraor;
import org.apache.lucene.facet.search.TotalFacetCounts;
import org.apache.lucene.facet.util.PartitionsUtils;
import org.apache.lucene.util.encoding.DGapIntEncoder;
@ -142,7 +142,7 @@ public class CategoryListParams implements Serializable {
int partition) throws IOException {
String categoryListTermStr = PartitionsUtils.partitionName(this, partition);
Term payloadTerm = new Term(term.field(), categoryListTermStr);
return new PayloadIntDecodingIterator(reader, payloadTerm,
return new PayloadCategoryListIteraor(reader, payloadTerm,
createEncoder().createMatchingDecoder());
}

View File

@ -83,18 +83,9 @@ public class FacetIndexingParams {
}
/**
* The name of the category list to put this category in, or {@code null} if
* this category should not be aggregatable.
* <p>
* By default, all categories are written to the same category list, but
* applications which know in advance that in some situations only parts of
* the category hierarchy needs to be counted can divide the categories into
* two or more different category lists.
* <p>
* If {@code null} is returned for a category, it means that this category
* should not appear in any category list, and thus weights for it cannot be
* aggregated. This category can still be used for drill-down, even though the
* its weight is unknown.
* Returns the {@link CategoryListParams} for this {@link CategoryPath}. The
* default implementation returns the same {@link CategoryListParams} for all
* categories (even if {@code category} is {@code null}).
*
* @see PerDimensionIndexingParams
*/

View File

@ -78,7 +78,9 @@ public class PerDimensionIndexingParams extends FacetIndexingParams {
/**
* Returns the {@link CategoryListParams} for the corresponding dimension
* which is returned by {@code category.getComponent(0)}.
* which is returned by {@code category.getComponent(0)}. If {@code category}
* is {@code null}, or was not specified in the map given to the constructor,
* returns the default {@link CategoryListParams}.
*/
@Override
public CategoryListParams getCategoryListParams(CategoryPath category) {

View File

@ -2,6 +2,8 @@ package org.apache.lucene.facet.search;
import java.io.IOException;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -20,20 +22,10 @@ import java.io.IOException;
*/
/**
* An interface for iterating over a "category list", i.e., the list of
* categories per document.
* An interface for obtaining the category ordinals of documents.
* <p>
* <b>NOTE:</b>
* <ul>
* <li>This class operates as a key to a Map. Appropriate implementation of
* <code>hashCode()</code> and <code>equals()</code> must be provided.
* <li>{@link #init()} must be called before you consume any categories, or call
* {@link #skipTo(int)}.
* <li>{@link #skipTo(int)} must be called before any calls to
* {@link #nextCategory()}.
* <li>{@link #nextCategory()} returns values &lt; {@link Integer#MAX_VALUE}, so
* you can use it as a stop condition.
* </ul>
* <b>NOTE:</b> this class operates as a key to a map, and therefore you should
* implement {@code equals()} and {@code hashCode()} for proper behavior.
*
* @lucene.experimental
*/
@ -41,29 +33,20 @@ public interface CategoryListIterator {
/**
* Initializes the iterator. This method must be called before any calls to
* {@link #skipTo(int)}, and its return value indicates whether there are
* any relevant documents for this iterator. If it returns false, any call
* to {@link #skipTo(int)} will return false as well.<br>
* <b>NOTE:</b> calling this method twice may result in skipping over
* documents for some implementations. Also, calling it again after all
* documents were consumed may yield unexpected behavior.
* {@link #getOrdinals(int, IntsRef)}, and its return value indicates whether there are
* any relevant documents for this iterator.
*/
public boolean init() throws IOException;
/**
* Skips forward to document docId. Returns true iff this document exists
* and has any categories. This method must be called before calling
* {@link #nextCategory()} for a particular document.<br>
* <b>NOTE:</b> Users should call this method with increasing docIds, and
* implementations can assume that this is the case.
* Stores the category ordinals of the given document ID in the given
* {@link IntsRef}, starting at position 0 upto {@link IntsRef#length}. Grows
* the {@link IntsRef} if it is not large enough.
*
* <p>
* <b>NOTE:</b> if the requested document does not category ordinals
* associated with it, {@link IntsRef#length} is set to zero.
*/
public boolean skipTo(int docId) throws IOException;
/**
* Returns the next category for the current document that is set through
* {@link #skipTo(int)}, or a number higher than {@link Integer#MAX_VALUE}.
* No assumptions can be made on the order of the categories.
*/
public long nextCategory() throws IOException;
public void getOrdinals(int docID, IntsRef ints) throws IOException;
}

View File

@ -5,7 +5,7 @@ import java.io.IOException;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.UnsafeByteArrayInputStream;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.encoding.IntDecoder;
/*
@ -26,44 +26,21 @@ import org.apache.lucene.util.encoding.IntDecoder;
*/
/**
* A payload deserializer comes with its own working space (buffer). One need to
* define the {@link IndexReader} and {@link Term} in which the payload resides.
* The iterator then consumes the payload information of each document and
* decodes it into categories. A typical use case of this class is:
*
* <pre class="prettyprint">
* IndexReader reader = [open your reader];
* Term t = new Term(&quot;field&quot;, &quot;where-payload-exists&quot;);
* CategoryListIterator cli = new PayloadIntDecodingIterator(reader, t);
* if (!cli.init()) {
* // it means there are no payloads / documents associated with that term.
* // Usually a sanity check. However, init() must be called.
* }
* DocIdSetIterator disi = [you usually iterate on something else, such as a Scorer];
* int doc;
* while ((doc = disi.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
* cli.setdoc(doc);
* long category;
* while ((category = cli.nextCategory()) &lt; Integer.MAX_VALUE) {
* }
* }
* </pre>
* A {@link CategoryListIterator} which reads the category ordinals from a
* payload.
*
* @lucene.experimental
*/
public class PayloadIntDecodingIterator implements CategoryListIterator {
public class PayloadCategoryListIteraor implements CategoryListIterator {
private final UnsafeByteArrayInputStream ubais;
private final IntDecoder decoder;
private final IndexReader indexReader;
private final Term term;
private final PayloadIterator pi;
private final int hashCode;
public PayloadIntDecodingIterator(IndexReader indexReader, Term term, IntDecoder decoder) throws IOException {
public PayloadCategoryListIteraor(IndexReader indexReader, Term term, IntDecoder decoder) throws IOException {
pi = new PayloadIterator(indexReader, term);
ubais = new UnsafeByteArrayInputStream();
this.decoder = decoder;
hashCode = indexReader.hashCode() ^ term.hashCode();
this.term = term;
@ -72,10 +49,10 @@ public class PayloadIntDecodingIterator implements CategoryListIterator {
@Override
public boolean equals(Object other) {
if (!(other instanceof PayloadIntDecodingIterator)) {
if (!(other instanceof PayloadCategoryListIteraor)) {
return false;
}
PayloadIntDecodingIterator that = (PayloadIntDecodingIterator) other;
PayloadCategoryListIteraor that = (PayloadCategoryListIteraor) other;
if (hashCode != that.hashCode) {
return false;
}
@ -95,21 +72,12 @@ public class PayloadIntDecodingIterator implements CategoryListIterator {
}
@Override
public long nextCategory() throws IOException {
return decoder.decode();
}
@Override
public boolean skipTo(int docId) throws IOException {
if (!pi.setdoc(docId)) {
return false;
public void getOrdinals(int docID, IntsRef ints) throws IOException {
ints.length = 0;
BytesRef payload = pi.getPayload(docID);
if (payload != null) {
decoder.decode(payload, ints);
}
// Initializing the decoding mechanism with the new payload data
BytesRef data = pi.getPayload();
ubais.reInit(data.bytes, data.offset, data.length + data.offset);
decoder.reInit(ubais);
return true;
}
}

View File

@ -34,9 +34,9 @@ import org.apache.lucene.util.BytesRef;
* A utility class for iterating through a posting list of a given term and
* retrieving the payload of the first position in every document. For
* efficiency, this class does not check if documents passed to
* {@link #setdoc(int)} are deleted, since it is usually used to iterate on
* {@link #getPayload(int)} are deleted, since it is usually used to iterate on
* payloads of documents that matched a query. If you need to skip over deleted
* documents, you should do so before calling {@link #setdoc(int)}.
* documents, you should do so before calling {@link #getPayload(int)}.
*
* @lucene.experimental
*/
@ -84,8 +84,8 @@ public class PayloadIterator {
/**
* Initialize the iterator. Should be done before the first call to
* {@link #setdoc(int)}. Returns {@code false} if no category list is found,
* or the category list has no documents.
* {@link #getPayload(int)}. Returns {@code false} if no category list is
* found, or the category list has no documents.
*/
public boolean init() throws IOException {
nextSegment();
@ -93,30 +93,29 @@ public class PayloadIterator {
}
/**
* Skip forward to document docId. Return true if this document exists and
* has any payload.
* <P>
* Users should call this method with increasing docIds, and implementations
* can assume that this is the case.
* Returns the {@link BytesRef payload} of the given document, or {@code null}
* if the document does not exist, there are no more documents in the posting
* list, or the document exists but has not payload. You should call
* {@link #init()} before the first call to this method.
*/
public boolean setdoc(int docId) throws IOException {
public BytesRef getPayload(int docID) throws IOException {
if (!hasMore) {
return false;
return null;
}
// re-basing docId->localDocID is done fewer times than currentDoc->globalDoc
int localDocID = docId - curDocBase;
int localDocID = docID - curDocBase;
if (curDocID > localDocID) {
// document does not exist
return false;
return null;
}
if (curDocID < localDocID) {
// look for the document either in that segment, or others
while (hasMore && (curDocID = currentDPE.advance(localDocID)) == DocIdSetIterator.NO_MORE_DOCS) {
nextSegment(); // also updates curDocID
localDocID = docId - curDocBase;
localDocID = docID - curDocBase;
// nextSegment advances to nextDoc, so check if we still need to advance
if (curDocID >= localDocID) {
break;
@ -127,7 +126,7 @@ public class PayloadIterator {
// 1. we iterated over all segments (hasMore=false)
// 2. current segment advanced to a doc, either requested or higher
if (!hasMore || curDocID != localDocID) {
return false;
return null;
}
}
@ -135,12 +134,7 @@ public class PayloadIterator {
assert currentDPE.freq() == 1 : "expecting freq=1 (got " + currentDPE.freq() + ") term=" + term + " doc=" + (curDocID + curDocBase);
int pos = currentDPE.nextPosition();
assert pos != -1 : "no positions for term=" + term + " doc=" + (curDocID + curDocBase);
data = currentDPE.getPayload();
return data != null;
return currentDPE.getPayload();
}
public BytesRef getPayload() {
return data;
}
}

View File

@ -10,6 +10,7 @@ import java.util.logging.Level;
import java.util.logging.Logger;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.facet.search.aggregator.Aggregator;
import org.apache.lucene.facet.search.params.FacetSearchParams;
@ -231,9 +232,9 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
facetArrays.free(); // to get a cleared array for this partition
}
HashMap<CategoryListIterator, Aggregator> categoryLists = getCategoryListMap(
facetArrays, partition);
HashMap<CategoryListIterator, Aggregator> categoryLists = getCategoryListMap(facetArrays, partition);
IntsRef ordinals = new IntsRef(32); // a reasonable start capacity for most common apps
for (Entry<CategoryListIterator, Aggregator> entry : categoryLists.entrySet()) {
CategoryListIterator categoryList = entry.getKey();
if (!categoryList.init()) {
@ -244,14 +245,11 @@ public class StandardFacetsAccumulator extends FacetsAccumulator {
ScoredDocIDsIterator iterator = docids.iterator();
while (iterator.next()) {
int docID = iterator.getDocID();
if (!categoryList.skipTo(docID)) {
categoryList.getOrdinals(docID, ordinals);
if (ordinals.length == 0) {
continue;
}
categorator.setNextDoc(docID, iterator.getScore());
long ordinal;
while ((ordinal = categoryList.nextCategory()) <= Integer.MAX_VALUE) {
categorator.aggregate((int) ordinal);
}
categorator.aggregate(docID, iterator.getScore(), ordinals);
}
}
}

View File

@ -2,6 +2,8 @@ package org.apache.lucene.facet.search.aggregator;
import java.io.IOException;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -36,16 +38,9 @@ import java.io.IOException;
public interface Aggregator {
/**
* Specify the document (and its score in the search) that the following
* {@link #aggregate(int)} calls will pertain to.
* Aggregate the ordinals of the given document ID (and its score). The given
* ordinals offset is always zero.
*/
void setNextDoc(int docid, float score) throws IOException;
/**
* Collect (and do whatever an implementation deems appropriate) the
* category given by its ordinal. This category belongs to a document
* given earlier by {@link #setNextDoc(int, float)}.
*/
void aggregate(int ordinal);
public void aggregate(int docID, float score, IntsRef ordinals) throws IOException;
}

View File

@ -1,5 +1,9 @@
package org.apache.lucene.facet.search.aggregator;
import java.io.IOException;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -29,9 +33,12 @@ public class ComplementCountingAggregator extends CountingAggregator {
}
@Override
public void aggregate(int ordinal) {
assert counterArray[ordinal]!=0:"complement aggregation: count is about to become negative for ordinal "+ordinal;
--counterArray[ordinal];
public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
for (int i = 0; i < ordinals.length; i++) {
int ord = ordinals.ints[i];
assert counterArray[ord] != 0 : "complement aggregation: count is about to become negative for ordinal " + ord;
--counterArray[ord];
}
}
}

View File

@ -1,5 +1,9 @@
package org.apache.lucene.facet.search.aggregator;
import java.io.IOException;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -27,21 +31,17 @@ package org.apache.lucene.facet.search.aggregator;
public class CountingAggregator implements Aggregator {
protected int[] counterArray;
@Override
public void aggregate(int ordinal) {
++counterArray[ordinal];
}
@Override
public void setNextDoc(int docid, float score) {
// There's nothing for us to do here since we only increment the count by 1
// in this aggregator.
}
public CountingAggregator(int[] counterArray) {
this.counterArray = counterArray;
}
@Override
public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
for (int i = 0; i < ordinals.length; i++) {
counterArray[ordinals.ints[i]]++;
}
}
@Override
public boolean equals(Object obj) {
@ -54,8 +54,7 @@ public class CountingAggregator implements Aggregator {
@Override
public int hashCode() {
int hashCode = counterArray == null ? 0 : counterArray.hashCode();
return hashCode;
return counterArray == null ? 0 : counterArray.hashCode();
}
}

View File

@ -1,5 +1,9 @@
package org.apache.lucene.facet.search.aggregator;
import java.io.IOException;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -26,7 +30,6 @@ package org.apache.lucene.facet.search.aggregator;
public class ScoringAggregator implements Aggregator {
private final float[] scoreArray;
private float score;
private final int hashCode;
public ScoringAggregator(float[] counterArray) {
@ -35,10 +38,12 @@ public class ScoringAggregator implements Aggregator {
}
@Override
public void aggregate(int ordinal) {
scoreArray[ordinal] += score;
public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
for (int i = 0; i < ordinals.length; i++) {
scoreArray[ordinals.ints[i]] += score;
}
}
@Override
public boolean equals(Object obj) {
if (obj == null || obj.getClass() != this.getClass()) {
@ -53,8 +58,4 @@ public class ScoringAggregator implements Aggregator {
return hashCode;
}
@Override
public void setNextDoc(int docid, float score) {
this.score = score;
}
}

View File

@ -7,6 +7,8 @@ import org.apache.lucene.facet.associations.FloatAssociationsPayloadIterator;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.search.aggregator.Aggregator;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.collections.IntToFloatMap;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -48,13 +50,18 @@ public class AssociationFloatSumAggregator implements Aggregator {
}
@Override
public void aggregate(int ordinal) {
float association = associations.getAssociation(ordinal);
if (!Float.isNaN(association)) {
sumArray[ordinal] += association;
public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
IntToFloatMap values = associations.getAssociations(docID);
if (values != null) {
for (int i = 0; i < ordinals.length; i++) {
int ord = ordinals.ints[i];
if (values.containsKey(ord)) {
sumArray[ord] += values.get(ord);
}
}
}
}
@Override
public boolean equals(Object obj) {
if (obj == null || obj.getClass() != this.getClass()) {
@ -69,9 +76,4 @@ public class AssociationFloatSumAggregator implements Aggregator {
return field.hashCode();
}
@Override
public void setNextDoc(int docid, float score) throws IOException {
associations.setNextDoc(docid);
}
}

View File

@ -7,6 +7,8 @@ import org.apache.lucene.facet.associations.IntAssociationsPayloadIterator;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.search.aggregator.Aggregator;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.collections.IntToIntMap;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -48,13 +50,18 @@ public class AssociationIntSumAggregator implements Aggregator {
}
@Override
public void aggregate(int ordinal) {
long association = associations.getAssociation(ordinal);
if (association != IntAssociationsPayloadIterator.NO_ASSOCIATION) {
sumArray[ordinal] += association;
public void aggregate(int docID, float score, IntsRef ordinals) throws IOException {
IntToIntMap values = associations.getAssociations(docID);
if (values != null) {
for (int i = 0; i < ordinals.length; i++) {
int ord = ordinals.ints[i];
if (values.containsKey(ord)) {
sumArray[ord] += values.get(ord);
}
}
}
}
@Override
public boolean equals(Object obj) {
if (obj == null || obj.getClass() != this.getClass()) {
@ -69,9 +76,4 @@ public class AssociationIntSumAggregator implements Aggregator {
return field.hashCode();
}
@Override
public void setNextDoc(int docid, float score) throws IOException {
associations.setNextDoc(docid);
}
}

View File

@ -2,13 +2,12 @@ package org.apache.lucene.facet.search.cache;
import java.io.IOException;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.search.CategoryListIterator;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.util.collections.IntArray;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -56,33 +55,26 @@ public class CategoryListData {
protected CategoryListData() {
}
/**
* Compute category list data for caching for faster iteration.
*/
/** Compute category list data for caching for faster iteration. */
CategoryListData(IndexReader reader, TaxonomyReader taxo,
FacetIndexingParams iparams, CategoryListParams clp) throws IOException {
final int maxDoc = reader.maxDoc();
int[][][]dpf = new int[maxDoc][][];
int numPartitions = (int)Math.ceil(taxo.getSize()/(double)iparams.getPartitionSize());
IntArray docCategories = new IntArray();
for (int part=0; part<numPartitions; part++) {
IntsRef ordinals = new IntsRef(32);
for (int part = 0; part < numPartitions; part++) {
CategoryListIterator cli = clp.createCategoryListIterator(reader, part);
if (cli.init()) {
for (int doc=0; doc<maxDoc; doc++) {
if (cli.skipTo(doc)) {
docCategories.clear(false);
if (dpf[doc]==null) {
for (int doc = 0; doc < maxDoc; doc++) {
cli.getOrdinals(doc, ordinals);
if (ordinals.length > 0) {
if (dpf[doc] == null) {
dpf[doc] = new int[numPartitions][];
}
long category;
while ((category = cli.nextCategory()) <= Integer.MAX_VALUE) {
docCategories.addToArray((int)category);
}
final int size = docCategories.size();
dpf[doc][part] = new int[size];
for (int i=0; i<size; i++) {
dpf[doc][part][i] = docCategories.get(i);
dpf[doc][part] = new int[ordinals.length];
for (int i = 0; i < ordinals.length; i++) {
dpf[doc][part][i] = ordinals.ints[i];
}
}
}
@ -98,14 +90,11 @@ public class CategoryListData {
return new RAMCategoryListIterator(partition, docPartitionCategories);
}
/**
* Internal: category list iterator over uncompressed category info in RAM
*/
/** Internal: category list iterator over uncompressed category info in RAM */
private static class RAMCategoryListIterator implements CategoryListIterator {
private final int part;
private final int[][][] dpc;
private int currDoc = -1;
private int nextCategoryIndex = -1;
RAMCategoryListIterator(int part, int[][][] docPartitionCategories) {
this.part = part;
@ -114,25 +103,22 @@ public class CategoryListData {
@Override
public boolean init() throws IOException {
return dpc!=null && dpc.length>part;
return dpc != null && dpc.length > part;
}
@Override
public long nextCategory() throws IOException {
if (nextCategoryIndex >= dpc[currDoc][part].length) {
return 1L+Integer.MAX_VALUE;
public void getOrdinals(int docID, IntsRef ints) throws IOException {
ints.length = 0;
if (dpc.length > docID && dpc[docID] != null && dpc[docID][part] != null) {
if (ints.ints.length < dpc[docID][part].length) {
ints.grow(dpc[docID][part].length);
}
ints.length = 0;
for (int i = 0; i < dpc[docID][part].length; i++) {
ints.ints[ints.length++] = dpc[docID][part][i];
}
}
return dpc[currDoc][part][nextCategoryIndex++];
}
@Override
public boolean skipTo(int docId) throws IOException {
final boolean res = dpc.length>docId && dpc[docId]!=null && dpc[docId][part]!=null;
if (res) {
currDoc = docId;
nextCategoryIndex = 0;
}
return res;
}
}
}

View File

@ -48,8 +48,7 @@ public class CountFacetRequest extends FacetRequest {
@Override
public Aggregator createAggregator(boolean useComplements,
FacetArrays arrays, IndexReader reader,
TaxonomyReader taxonomy) {
FacetArrays arrays, IndexReader reader, TaxonomyReader taxonomy) {
// we rely on that, if needed, result is cleared by arrays!
int[] a = arrays.getIntArray();
if (useComplements) {

View File

@ -5,6 +5,7 @@ import java.util.ArrayList;
import java.util.List;
import org.apache.lucene.facet.search.CategoryListIterator;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -33,16 +34,13 @@ public class MultiCategoryListIterator implements CategoryListIterator {
private final CategoryListIterator[] iterators;
private final List<CategoryListIterator> validIterators;
private final List<CategoryListIterator> perDocValidIterators;
/** Receives the iterators to iterate on */
public MultiCategoryListIterator(CategoryListIterator... iterators) {
this.iterators = iterators;
this.validIterators = new ArrayList<CategoryListIterator>();
this.perDocValidIterators = new ArrayList<CategoryListIterator>();
}
/** Fails if all given iterators fail to init */
@Override
public boolean init() throws IOException {
for (CategoryListIterator cli : iterators) {
@ -52,35 +50,17 @@ public class MultiCategoryListIterator implements CategoryListIterator {
}
return !validIterators.isEmpty();
}
/**
* Return a value larger than {@link Integer#MAX_VALUE} only if all
* iterators are exhausted
*/
@Override
public long nextCategory() throws IOException {
while (!perDocValidIterators.isEmpty()) {
long value = perDocValidIterators.get(0).nextCategory();
if (value <= Integer.MAX_VALUE) {
return value;
}
perDocValidIterators.remove(0);
}
return 0x100000000L;
}
/**
* Fails only if skipTo on all the provided iterators returned {@code false}
*/
@Override
public boolean skipTo(int docId) throws IOException {
perDocValidIterators.clear();
public void getOrdinals(int docID, IntsRef ints) throws IOException {
IntsRef tmp = new IntsRef(ints.length);
for (CategoryListIterator cli : validIterators) {
if (cli.skipTo(docId)) {
perDocValidIterators.add(cli);
cli.getOrdinals(docID, tmp);
if (ints.ints.length < ints.length + tmp.length) {
ints.grow(ints.length + tmp.length);
}
ints.length += tmp.length;
}
return !perDocValidIterators.isEmpty();
}
}

View File

@ -1,229 +0,0 @@
package org.apache.lucene.util;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
/*
* 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.
*/
/**
* Variable-length encoding of 32-bit integers, into 8-bit bytes. A number is encoded as follows:
* <ul>
* <li>If it is less than 127 and non-negative (i.e., if the number uses only 7 bits), it is encoded as
* as single byte: 0bbbbbbb.
* <li>If its highest nonzero bit is greater than bit 6 (0x40), it is represented as a series of
* bytes, each byte's
* 7 LSB containing bits from the original value, with the MSB set for all but the last
* byte. The first encoded byte contains the highest nonzero bits from the
* original; the second byte contains the next 7 MSB; and so on, with the last byte
* containing the 7 LSB of the original.
* </ul>
* Examples:
* <ol>
* <li>n = 117 = 1110101: This has fewer than 8 significant bits, and so is encoded as
* 01110101 = 0x75.
* <li>n = 100000 = (binary) 11000011010100000. This has 17 significant bits, and so needs
* three Vint8 bytes. Left-zero-pad it to a multiple of 7 bits, then split it into chunks of 7
* and add an MSB, 0 for the last byte, 1 for the others: 1|0000110 1|0001101 0|0100000
* = 0x86 0x8D 0x20.
* </ol>
* This encoder/decoder will correctly handle any 32-bit integer, but for negative numbers,
* and positive numbers with more than 28 significant bits, encoding requires 5 bytes; this
* is not an efficient encoding scheme for large
* positive numbers or any negative number.
* <p>
* <b>Compatibility:</b><br>
* This class has been used in products that have shipped to customers, and is needed to
* decode legacy data. Do not modify this class in ways that will break compatibility.
*
* @lucene.experimental
*/
public class Vint8 {
/**
* Because Java lacks call-by-reference, this class boxes the decoding position, which
* is initially set by the caller, and returned after decoding, incremented by the number
* of bytes processed.
*/
public static class Position {
/**
* Creates a position value set to zero.
*/
public Position() {
// The initial position is zero by default.
}
/**
* Creates a position set to {@code initialPosition}.
* @param initialPosition The starting decoding position in the source buffer.
*/
public Position(int initialPosition) {
this.pos = initialPosition;
}
/**
* The value passed by reference.
*/
public int pos;
}
/**
* Returns the number of bytes needed to encode {@code number}.
* @param number The number whose encoded length is needed.
* @return The number of bytes needed to encode {@code number}.
*/
public static int bytesNeeded(int number) {
if ((number & ~0x7F) == 0) {
return 1;
} else if ((number & ~0x3FFF) == 0) {
return 2;
} else if ((number & ~0x1FFFFF) == 0) {
return 3;
} else if ((number & ~0xFFFFFFF) == 0) {
return 4;
} else {
return 5;
}
}
/**
* The maximum number of bytes needed to encode a number using {@code Vint8}.
*/
public static final int MAXIMUM_BYTES_NEEDED = 5;
/**
* Encodes {@code number} to {@code out}.
* @param number The value to be written in encoded form, to {@code out}.
* @param out The output stream receiving the encoded bytes.
* @exception IOException If there is a problem writing to {@code out}.
*/
public static void encode(int number, OutputStream out) throws IOException {
if ((number & ~0x7F) == 0) {
out.write(number);
} else if ((number & ~0x3FFF) == 0) {
out.write(0x80 | (number >> 7));
out.write(0x7F & number);
} else if ((number & ~0x1FFFFF) == 0) {
out.write(0x80 | (number >> 14));
out.write(0x80 | (number >> 7));
out.write(0x7F & number);
} else if ((number & ~0xFFFFFFF) == 0) {
out.write(0x80 | (number >> 21));
out.write(0x80 | (number >> 14));
out.write(0x80 | (number >> 7));
out.write(0x7F & number);
} else {
out.write(0x80 | (number >> 28));
out.write(0x80 | (number >> 21));
out.write(0x80 | (number >> 14));
out.write(0x80 | (number >> 7));
out.write(0x7F & number);
}
}
/**
* Encodes {@code number} into {@code dest}, starting at offset {@code start} from
* the beginning of the array. This method assumes {@code dest} is large enough to
* hold the required number of bytes.
* @param number The number to be encoded.
* @param dest The destination array.
* @param start The starting offset in the array.
* @return The number of bytes used in the array.
*/
public static int encode(int number, byte[] dest, int start) {
if ((number & ~0x7F) == 0) {
dest[start] = (byte) number;
return 1;
} else if ((number & ~0x3FFF) == 0) {
dest[start] = (byte) (0x80 | ((number & 0x3F80) >> 7));
dest[start + 1] = (byte) (number & 0x7F);
return 2;
} else if ((number & ~0x1FFFFF) == 0) {
dest[start] = (byte) (0x80 | ((number & 0x1FC000) >> 14));
dest[start + 1] = (byte) (0x80 | ((number & 0x3F80) >> 7));
dest[start + 2] = (byte) (number & 0x7F);
return 3;
} else if ((number & ~0xFFFFFFF) == 0) {
dest[start] = (byte) (0x80 | ((number & 0xFE00000) >> 21));
dest[start + 1] = (byte) (0x80 | ((number & 0x1FC000) >> 14));
dest[start + 2] = (byte) (0x80 | ((number & 0x3F80) >> 7));
dest[start + 3] = (byte) (number & 0x7F);
return 4;
} else {
dest[start] = (byte) (0x80 | ((number & 0xF0000000) >> 28));
dest[start + 1] = (byte) (0x80 | ((number & 0xFE00000) >> 21));
dest[start + 2] = (byte) (0x80 | ((number & 0x1FC000) >> 14));
dest[start + 3] = (byte) (0x80 | ((number & 0x3F80) >> 7));
dest[start + 4] = (byte) (number & 0x7F);
return 5;
}
}
/**
* Decodes a 32-bit integer from {@code bytes}, beginning at offset {@code pos.pos}.
* The decoded value is returned, and {@code pos.pos} is incremented by the number of
* bytes processed.
* @param bytes The byte array containing an encoded value.
* @param pos On entry, the starting position in the array; on return, one greater
* than the position of the last byte decoded in the call.
* @return The decoded value.
*/
public static int decode(byte[] bytes, Position pos) {
int value = 0;
while (true) {
byte first = bytes[pos.pos];
++pos.pos;
value |= first & 0x7F;
if ((first & 0x80) == 0) {
return value;
}
value <<= 7;
}
}
/**
* Decodes a 32-bit integer from bytes read from {@code in}. Bytes are read,
* one at a time, from {@code in}, and it is assumed they represent a 32-bit
* integer encoded using this class's encoding scheme. The decoded value is
* returned.
* @param in The input stream containing the encoded bytes.
* @return The decoded value.
* @exception EOFException If the stream ends before a value has been decoded.
*/
public static int decode(InputStream in) throws IOException {
int value = 0;
while (true) {
int first = in.read();
if (first < 0) {
throw new EOFException();
}
value |= first & 0x7F;
if ((first & 0x80) == 0) {
return value;
}
value <<= 7;
}
}
/**
* The default ctor is made private because all methods of this class are static.
*/
private Vint8() {
// Just making it impossible to instantiate.
}
}

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -27,38 +27,31 @@ import java.io.OutputStream;
* read more on the two implementations {@link FourFlagsIntEncoder} and
* {@link EightFlagsIntEncoder}.
* <p>
* Extensions of this class need to implement {@link #encode(int)} in order to
* build the proper indicator (flags). When enough values were accumulated
* (typically the batch size), extensions can call {@link #encodeChunk()} to
* flush the indicator and the rest of the values.
* Extensions of this class need to implement {@link #encode(IntsRef, BytesRef)}
* in order to build the proper indicator (flags). When enough values were
* accumulated (typically the batch size), extensions can call
* {@link #encodeChunk(BytesRef)} to flush the indicator and the rest of the
* values.
* <p>
* <b>NOTE:</b> flags encoders do not accept values &le; 0 (zero) in their
* {@link #encode(int)}. For performance reasons they do not check that
* condition, however if such value is passed the result stream may be corrupt
* or an exception will be thrown. Also, these encoders perform the best when
* there are many consecutive small values (depends on the encoder
* {@link #encode(IntsRef, BytesRef)}. For performance reasons they do not check
* that condition, however if such value is passed the result stream may be
* corrupt or an exception will be thrown. Also, these encoders perform the best
* when there are many consecutive small values (depends on the encoder
* implementation). If that is not the case, the encoder will occupy 1 more byte
* for every <i>batch</i> number of integers, over whatever
* {@link VInt8IntEncoder} would have occupied. Therefore make sure to check
* whether your data fits into the conditions of the specific encoder.
* <p>
* For the reasons mentioned above, these encoders are usually chained with
* {@link UniqueValuesIntEncoder} and {@link DGapIntEncoder} in the following
* manner: <code><pre class="prettyprint">
* IntEncoder fourFlags =
* new SortingEncoderFilter(new UniqueValuesIntEncoder(new DGapIntEncoder(new FlagsIntEncoderImpl())));
* </pre></code>
* {@link UniqueValuesIntEncoder} and {@link DGapIntEncoder}.
*
* @lucene.experimental
*/
public abstract class ChunksIntEncoder extends IntEncoder {
/** Holds the values which must be encoded, outside the indicator. */
protected final int[] encodeQueue;
protected int encodeQueueSize = 0;
/** Encoder used to encode values outside the indicator. */
protected final IntEncoder encoder = new VInt8IntEncoder();
protected final IntsRef encodeQueue;
/** Represents bits flag byte. */
protected int indicator = 0;
@ -67,39 +60,33 @@ public abstract class ChunksIntEncoder extends IntEncoder {
protected byte ordinal = 0;
protected ChunksIntEncoder(int chunkSize) {
encodeQueue = new int[chunkSize];
encodeQueue = new IntsRef(chunkSize);
}
/**
* Encodes the values of the current chunk. First it writes the indicator, and
* then it encodes the values outside the indicator.
*/
protected void encodeChunk() throws IOException {
out.write(indicator);
for (int i = 0; i < encodeQueueSize; ++i) {
encoder.encode(encodeQueue[i]);
protected void encodeChunk(BytesRef buf) {
// ensure there's enough room in the buffer
int maxBytesRequired = buf.length + 1 + encodeQueue.length * 4; /* indicator + at most 4 bytes per positive VInt */
if (buf.bytes.length < maxBytesRequired) {
buf.grow(maxBytesRequired);
}
encodeQueueSize = 0;
ordinal = 0;
indicator = 0;
buf.bytes[buf.length++] = ((byte) indicator);
for (int i = 0; i < encodeQueue.length; i++) {
VInt8.encode(encodeQueue.ints[i], buf);
}
reset();
}
@Override
public void close() throws IOException {
if (ordinal != 0) {
encodeChunk();
}
encoder.close();
super.close();
}
@Override
public void reInit(OutputStream out) {
encoder.reInit(out);
super.reInit(out);
protected void reset() {
ordinal = 0;
indicator = 0;
encodeQueueSize = 0;
encodeQueue.length = 0;
}
}

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.InputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,10 +21,8 @@ import java.io.InputStream;
*/
/**
* An {@link IntDecoder} which wraps another {@link IntDecoder} and reverts the
* d-gap that was encoded by {@link DGapIntEncoder}. The wrapped decoder
* performs the actual decoding, while this class simply adds the decoded value
* to the previous value.
* An {@link IntDecoder} which wraps another decoder and reverts the d-gap that
* was encoded by {@link DGapIntEncoder}.
*
* @lucene.experimental
*/
@ -32,26 +30,23 @@ public class DGapIntDecoder extends IntDecoder {
private final IntDecoder decoder;
private int prev = 0;
public DGapIntDecoder(IntDecoder decoder) {
this.decoder = decoder;
}
@Override
public long decode() throws IOException {
long decode = decoder.decode();
if (decode == EOS) {
return EOS;
}
return prev += decode;
protected void reset() {
decoder.reset();
}
@Override
public void reInit(InputStream in) {
decoder.reInit(in);
prev = 0;
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
decoder.doDecode(buf, values, upto);
int prev = 0;
for (int i = 0; i < values.length; i++) {
values.ints[i] += prev;
prev = values.ints[i];
}
}
@Override

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -27,7 +27,7 @@ import java.io.OutputStream;
* space) if the values are 'close' to each other.
* <p>
* <b>NOTE:</b> this encoder assumes the values are given to
* {@link #encode(int)} in an ascending sorted manner, which ensures only
* {@link #encode(IntsRef, BytesRef)} in an ascending sorted manner, which ensures only
* positive values are encoded and thus yields better performance. If you are
* not sure whether the values are sorted or not, it is possible to chain this
* encoder with {@link SortingIntEncoder} to ensure the values will be
@ -37,17 +37,20 @@ import java.io.OutputStream;
*/
public class DGapIntEncoder extends IntEncoderFilter {
private int prev = 0;
/** Initializes with the given encoder. */
public DGapIntEncoder(IntEncoder encoder) {
super(encoder);
}
@Override
public void encode(int value) throws IOException {
encoder.encode(value - prev);
prev = value;
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
int prev = 0;
for (int i = values.offset; i < upto; i++) {
int tmp = values.ints[i];
values.ints[i] -= prev;
prev = tmp;
}
encoder.doEncode(values, buf, upto);
}
@Override
@ -55,12 +58,6 @@ public class DGapIntEncoder extends IntEncoderFilter {
return new DGapIntDecoder(encoder.createMatchingDecoder());
}
@Override
public void reInit(OutputStream out) {
super.reInit(out);
prev = 0;
}
@Override
public String toString() {
return "DGap (" + encoder.toString() + ")";

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.InputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,20 +21,17 @@ import java.io.InputStream;
*/
/**
* Decodes data which was encoded by {@link EightFlagsIntEncoder}. Scans
* the <code>indicator</code>, one flag (1-bits) at a time, and decodes extra
* data using {@link VInt8IntDecoder}.
* Decodes values encoded with {@link EightFlagsIntEncoder}.
*
* @see EightFlagsIntEncoder
* @lucene.experimental
*/
public class EightFlagsIntDecoder extends IntDecoder {
/**
/*
* Holds all combinations of <i>indicator</i> for fast decoding (saves time
* on real-time bit manipulation)
*/
private static final byte[][] decodeTable = new byte[256][8];
private static final byte[][] DECODE_TABLE = new byte[256][8];
/** Generating all combinations of <i>indicator</i> into separate flags. */
static {
@ -42,45 +39,36 @@ public class EightFlagsIntDecoder extends IntDecoder {
--i;
for (int j = 8; j != 0;) {
--j;
decodeTable[i][j] = (byte) ((i >>> j) & 0x1);
DECODE_TABLE[i][j] = (byte) ((i >>> j) & 0x1);
}
}
}
private final IntDecoder decoder = new VInt8IntDecoder();
/** The indicator for decoding a chunk of 8 integers. */
private int indicator;
/** Used as an ordinal of 0 - 7, as the decoder decodes chunks of 8 integers. */
private int ordinal = 0;
@Override
public long decode() throws IOException {
// If we've decoded 8 integers, read the next indicator.
if ((ordinal & 0x7) == 0) {
indicator = in.read();
if (indicator < 0) {
return EOS;
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
while (buf.offset < upto) {
// read indicator
int indicator = buf.bytes[buf.offset++] & 0xFF;
int ordinal = 0;
int capacityNeeded = values.length + 8;
if (values.ints.length < capacityNeeded) {
values.grow(capacityNeeded);
}
// process indicator, until we read 8 values, or end-of-buffer
while (ordinal != 8) {
if (DECODE_TABLE[indicator][ordinal++] == 0) {
if (buf.offset == upto) { // end of buffer
return;
}
// decode the value from the stream.
values.ints[values.length++] = VInt8.decode(buf) + 2;
} else {
values.ints[values.length++] = 1;
}
}
ordinal = 0;
}
if (decodeTable[indicator][ordinal++] == 0) {
// decode the value from the stream.
long decode = decoder.decode();
return decode == EOS ? EOS : decode + 2;
}
return 1;
}
@Override
public void reInit(InputStream in) {
super.reInit(in);
decoder.reInit(in);
ordinal = 0;
indicator = 0;
}
@Override

View File

@ -1,6 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -20,14 +21,15 @@ import java.io.IOException;
*/
/**
* A {@link ChunksIntEncoder} which encodes data in chunks of 8. Every group starts with a single
* byte (called indicator) which represents 8 - 1 bit flags, where the value:
* A {@link ChunksIntEncoder} which encodes data in chunks of 8. Every group
* starts with a single byte (called indicator) which represents 8 - 1 bit
* flags, where the value:
* <ul>
* <li>1 means the encoded value is '1'
* <li>0 means the value is encoded using {@link VInt8IntEncoder}, and the
* encoded bytes follow the indicator.<br>
* Since value 0 is illegal, and 1 is encoded in the indicator, the actual
* value that is encoded is <code>value-2</code>, which saves some more bits.
* Since value 0 is illegal, and 1 is encoded in the indicator, the actual value
* that is encoded is <code>value-2</code>, which saves some more bits.
* </ul>
* Encoding example:
* <ul>
@ -46,28 +48,36 @@ import java.io.IOException;
*/
public class EightFlagsIntEncoder extends ChunksIntEncoder {
/**
/*
* Holds all combinations of <i>indicator</i> flags for fast encoding (saves
* time on bit manipulation at encode time)
*/
private static byte[] encodeTable = new byte[] { 0x1, 0x2, 0x4, 0x8, 0x10, 0x20, 0x40, (byte) 0x80 };
private static final byte[] ENCODE_TABLE = new byte[] { 0x1, 0x2, 0x4, 0x8, 0x10, 0x20, 0x40, (byte) 0x80 };
public EightFlagsIntEncoder() {
super(8);
}
@Override
public void encode(int data) throws IOException {
if (data == 1) {
indicator |= encodeTable[ordinal];
} else {
encodeQueue[encodeQueueSize++] = data - 2;
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
for (int i = values.offset; i < upto; i++) {
int value = values.ints[i];
if (value == 1) {
indicator |= ENCODE_TABLE[ordinal];
} else {
encodeQueue.ints[encodeQueue.length++] = value - 2;
}
++ordinal;
// encode the chunk and the indicator
if (ordinal == 8) {
encodeChunk(buf);
}
}
++ordinal;
// If 8 values were encoded thus far, 'flush' them including the indicator.
if ((ordinal & 0x7) == 0) {
encodeChunk();
// encode remaining values
if (ordinal != 0) {
encodeChunk(buf);
}
}
@ -78,7 +88,7 @@ public class EightFlagsIntEncoder extends ChunksIntEncoder {
@Override
public String toString() {
return "EightFlags (" + encoder.toString() + ")";
return "EightFlags (VInt)";
}
}

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.InputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,11 +21,8 @@ import java.io.InputStream;
*/
/**
* Decodes data which was encoded by {@link FourFlagsIntEncoder}. Scans
* the <code>indicator</code>, one flag (1-bits) at a time, and decodes extra
* data using {@link VInt8IntDecoder}.
* Decodes values encoded with {@link FourFlagsIntEncoder}.
*
* @see FourFlagsIntEncoder
* @lucene.experimental
*/
public class FourFlagsIntDecoder extends IntDecoder {
@ -34,7 +31,7 @@ public class FourFlagsIntDecoder extends IntDecoder {
* Holds all combinations of <i>indicator</i> for fast decoding (saves time
* on real-time bit manipulation)
*/
private final static byte[][] decodeTable = new byte[256][4];
private final static byte[][] DECODE_TABLE = new byte[256][4];
/** Generating all combinations of <i>indicator</i> into separate flags. */
static {
@ -42,46 +39,36 @@ public class FourFlagsIntDecoder extends IntDecoder {
--i;
for (int j = 4; j != 0;) {
--j;
decodeTable[i][j] = (byte) ((i >>> (j << 1)) & 0x3);
DECODE_TABLE[i][j] = (byte) ((i >>> (j << 1)) & 0x3);
}
}
}
private final IntDecoder decoder = new VInt8IntDecoder();
/** The indicator for decoding a chunk of 4 integers. */
private int indicator;
/** Used as an ordinal of 0 - 3, as the decoder decodes chunks of 4 integers. */
private int ordinal = 0;
@Override
public long decode() throws IOException {
// If we've decoded 8 integers, read the next indicator.
if ((ordinal & 0x3) == 0) {
indicator = in.read();
if (indicator < 0) {
return EOS;
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
while (buf.offset < upto) {
// read indicator
int indicator = buf.bytes[buf.offset++] & 0xFF;
int ordinal = 0;
int capacityNeeded = values.length + 4;
if (values.ints.length < capacityNeeded) {
values.grow(capacityNeeded);
}
while (ordinal != 4) {
byte decodeVal = DECODE_TABLE[indicator][ordinal++];
if (decodeVal == 0) {
if (buf.offset == upto) { // end of buffer
return;
}
// decode the value from the stream.
values.ints[values.length++] = VInt8.decode(buf) + 4;
} else {
values.ints[values.length++] = decodeVal;
}
}
ordinal = 0;
}
byte decodeVal = decodeTable[indicator][ordinal++];
if (decodeVal == 0) {
// decode the value from the stream.
long decode = decoder.decode();
return decode == EOS ? EOS : decode + 4;
}
return decodeVal;
}
@Override
public void reInit(InputStream in) {
super.reInit(in);
decoder.reInit(in);
ordinal = 0;
indicator = 0;
}
@Override

View File

@ -1,6 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -48,11 +49,11 @@ import java.io.IOException;
*/
public class FourFlagsIntEncoder extends ChunksIntEncoder {
/**
/*
* Holds all combinations of <i>indicator</i> flags for fast encoding (saves
* time on bit manipulation @ encode time)
*/
private static byte[][] encodeTable = new byte[][] {
private static final byte[][] ENCODE_TABLE = new byte[][] {
new byte[] { 0x00, 0x00, 0x00, 0x00 },
new byte[] { 0x01, 0x04, 0x10, 0x40 },
new byte[] { 0x02, 0x08, 0x20, (byte) 0x80 },
@ -63,26 +64,26 @@ public class FourFlagsIntEncoder extends ChunksIntEncoder {
super(4);
}
/**
* Small values (<=3) are stored in the <code>indicator</code> while larger
* values are saved for later encoding in the {@link #encodeQueue}. Since
* Vint8 will only encode values larger or equal to 4, the values saves for
* encoded are transformed to (value - 4).<br>
* When a chunk is ready (got 4 values), the {@link #encodeChunk()}
* takes control.
*/
@Override
public void encode(int data) throws IOException {
if (data <= 3) {
indicator |= encodeTable[data][ordinal];
} else {
encodeQueue[encodeQueueSize++] = data - 4;
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
for (int i = values.offset; i < upto; i++) {
int value = values.ints[i];
if (value <= 3) {
indicator |= ENCODE_TABLE[value][ordinal];
} else {
encodeQueue.ints[encodeQueue.length++] = value - 4;
}
++ordinal;
// encode the chunk and the indicator
if (ordinal == 4) {
encodeChunk(buf);
}
}
++ordinal;
// If 4 values were encoded thus far, 'flush' them including the indicator.
if ((ordinal & 0x3) == 0) {
encodeChunk();
// encode remaining values
if (ordinal != 0) {
encodeChunk(buf);
}
}
@ -93,7 +94,7 @@ public class FourFlagsIntEncoder extends ChunksIntEncoder {
@Override
public String toString() {
return "FourFlags (" + encoder.toString() + ")";
return "FourFlags (VInt)";
}
}

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.InputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,33 +21,50 @@ import java.io.InputStream;
*/
/**
* Decodes integers from a set {@link InputStream}. For re-usability, the
* decoder's input stream can be set by ({@link #reInit(InputStream)}).
* By design, Decoders are NOT thread-safe.
* Decodes integers from a set {@link BytesRef}.
*
* @lucene.experimental
*/
public abstract class IntDecoder {
/** A special long value which is used to indicate end-of-stream has reached. */
public static final long EOS = 0x100000000L;
/** Input stream from which the encoded bytes are read */
protected InputStream in;
/** Sets the input stream from which the encoded data is read. */
public void reInit(InputStream in) {
this.in = in;
/**
* Performs the actual decoding. Values should be read from
* {@link BytesRef#offset} up to {@code upto}. Also, {@code values} offset and
* length are set to 0 and the encoder is expected to update
* {@link IntsRef#length}, but not {@link IntsRef#offset}.
*
* <p>
* <b>NOTE:</b> it is ok to use the buffer's offset as the current position in
* the buffer (and modify it), it will be reset by
* {@link #decode(BytesRef, IntsRef)}.
*/
protected abstract void doDecode(BytesRef buf, IntsRef values, int upto);
/**
* Called before {@link #doDecode(BytesRef, IntsRef, int)} so that decoders
* can reset their state.
*/
protected void reset() {
// do nothing by default
}
/**
* Decodes data received from the input stream, and returns one decoded
* integer. If end of stream is reached, {@link #EOS} is returned.
*
* @return one decoded integer as long or {@link #EOS} if end-of-stream
* reached.
* @throws IOException if an I/O error occurs
* Decodes the values from the buffer into the given {@link IntsRef}. Note
* that {@code values.offset} and {@code values.length} are set to 0.
*/
public abstract long decode() throws IOException;
public final void decode(BytesRef buf, IntsRef values) {
values.offset = values.length = 0; // must do that because we cannot grow() them otherwise
// some decoders may use the buffer's offset as a position index, so save
// current offset.
int bufOffset = buf.offset;
reset();
doDecode(buf, values, buf.offset + buf.length);
assert values.offset == 0 : "offset should not have been modified by the decoder.";
// fix offset
buf.offset = bufOffset;
}
}

View File

@ -1,8 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.Closeable;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -22,94 +21,47 @@ import java.io.OutputStream;
*/
/**
* Encodes integers to a set {@link OutputStream}. Extending classes need to
* override {@link #encode(int)} to encode the value using their encoding
* algorithm. The default implementation of {@link #close()} closes the set
* {@link OutputStream}.
* <p>
* The default {@link #IntEncoder() constructor} is provided for convenience
* only. One must call {@link #reInit(OutputStream)} before calling
* {@link #encode(int)} or {@link #close()}.
* <p>
* For convenience, each encoder implements {@link #createMatchingDecoder()} for
* easy access to the matching decoder.
* <p>
* <b>NOTE:</b> some implementations may buffer the encoded values in memory
* (such as {@link IntEncoderFilter} implementations) and encoding will happen
* only upon calling {@link #close()}. Therefore it is important to always call
* {@link #close()} on the encoder at hand.
* <p>
* <b>NOTE:</b> encoders are usually not thread safe, unless specifically
* documented otherwise by an implementation.
* Encodes integers to a set {@link BytesRef}. For convenience, each encoder
* implements {@link #createMatchingDecoder()} for easy access to the matching
* decoder.
*
* @lucene.experimental
*/
public abstract class IntEncoder implements Closeable {
public abstract class IntEncoder {
protected OutputStream out = null;
public IntEncoder() {}
/**
* Default constructor, provided here for robustness: if in the future a
* constructor with parameters will be added, this might break custom
* implementations of this class which call this implicit constructor. So we
* make it explicit to avoid any such issue in the future.
* Performs the actual encoding. Values should be read from
* {@link IntsRef#offset} up to {@code upto}. Also, it is guaranteed that
* {@code buf's} offset and length are set to 0 and the encoder is expected to
* update {@link BytesRef#length}, but not {@link BytesRef#offset}.
*/
public IntEncoder() {
protected abstract void doEncode(IntsRef values, BytesRef buf, int upto);
/**
* Called before {@link #doEncode(IntsRef, BytesRef, int)} so that encoders
* can reset their state.
*/
protected void reset() {
// do nothing by default
}
/**
* Instructs the encoder to finish the encoding process. This method closes
* the output stream which was specified by {@link #reInit(OutputStream)
* reInit}. An implementation may do here additional cleanup required to
* complete the encoding, such as flushing internal buffers, etc.<br>
* Once this method was called, no further calls to {@link #encode(int)
* encode} should be made before first calling {@link #reInit(OutputStream)
* reInit}.
* <p>
* <b>NOTE:</b> overriding classes should make sure they either call
* <code>super.close()</code> or close the output stream themselves.
* Encodes the values to the given buffer. Note that the buffer's offset and
* length are set to 0.
*/
@Override
public void close() throws IOException {
if (out != null) {
out.close();
}
public final void encode(IntsRef values, BytesRef buf) {
buf.offset = buf.length = 0;
reset();
doEncode(values, buf, values.offset + values.length);
assert buf.offset == 0 : "offset should not have been modified by the encoder.";
}
/**
* Encodes an integer to the output stream given in
* {@link #reInit(OutputStream) reInit}
*/
public abstract void encode(int value) throws IOException;
/**
* Returns an {@link IntDecoder} which matches this encoder. Every encoder
* must return an {@link IntDecoder} and <code>null</code> is not a valid
* value. If an encoder is just a filter, it should at least return its
* wrapped encoder's matching decoder.
* <p>
* <b>NOTE:</b> this method should create a new instance of the matching
* decoder and leave the instance sharing to the caller. Returning the same
* instance over and over is risky because encoders and decoders are not
* thread safe.
* Returns an {@link IntDecoder} which can decode the values that were encoded
* with this encoder.
*/
public abstract IntDecoder createMatchingDecoder();
/**
* Reinitializes the encoder with the give {@link OutputStream}. For
* re-usability it can be changed without the need to reconstruct a new
* object.
* <p>
* <b>NOTE:</b> after calling {@link #close()}, one <u><i>must</i></u> call
* this method even if the output stream itself hasn't changed. An example
* case is that the output stream wraps a byte[], and the output stream itself
* is reset, but its instance hasn't changed. Some implementations of
* {@link IntEncoder} may write some metadata about themselves to the output
* stream, and therefore it is imperative that one calls this method before
* encoding any data.
*/
public void reInit(OutputStream out) {
this.out = out;
}
}

View File

@ -1,7 +1,5 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.OutputStream;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,17 +19,7 @@ import java.io.OutputStream;
*/
/**
* An abstract implementation of {@link IntEncoder} which is served as a filter
* on the values to encode. An encoder filter wraps another {@link IntEncoder}
* which does the actual encoding. This allows for chaining filters and
* encoders, such as: <code><pre class="prettyprint">
* new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEnoder()));
* {@link UniqueValuesIntEncoder} followed by {@link DGapIntEncoder}
</pre></code>
* <p>
* The default implementation implements {@link #close()} by closing the wrapped
* encoder and {@link #reInit(OutputStream)} by re-initializing the wrapped
* encoder.
* An abstract implementation of {@link IntEncoder} which wraps another encoder.
*
* @lucene.experimental
*/
@ -44,15 +32,8 @@ public abstract class IntEncoderFilter extends IntEncoder {
}
@Override
public void close() throws IOException {
// There is no need to call super.close(), since we don't pass the output
// stream to super.
encoder.close();
}
@Override
public void reInit(OutputStream out) {
encoder.reInit(out);
public void reset() {
encoder.reset();
}
}

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.InputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,54 +21,65 @@ import java.io.InputStream;
*/
/**
* Decodes data which was encoded by {@link NOnesIntEncoder}. Uses a
* {@link FourFlagsIntDecoder} to perform the actual encoding and translates the
* values back as described in {@link NOnesIntEncoder}.
* Decodes values encoded encoded with {@link NOnesIntEncoder}.
*
* @see NOnesIntEncoder
* @lucene.experimental
*/
public class NOnesIntDecoder extends FourFlagsIntDecoder {
/** Number of consecutive '1's to generate upon decoding a '2'. */
private int n;
private int onesCounter;
// Number of consecutive '1's to generate upon decoding a '2'
private final int n;
private final IntsRef internalBuffer;
/**
* Constructs a decoder with a given N (Number of consecutive '1's which are
* translated into a single target value '2'.
*/
public NOnesIntDecoder(int n) {
this.n = n;
// initial size (room for 100 integers)
internalBuffer = new IntsRef(100);
}
@Override
public long decode() throws IOException {
// If we read '2', we should return n '1's.
if (onesCounter > 0) {
--onesCounter;
return 1;
}
long decode = super.decode();
if (decode == 1) {
return 1;
}
if (decode == 2) {
onesCounter = n - 1;
return 1;
}
if (decode == 3) {
return 2;
}
return decode == EOS ? EOS : decode - 1;
protected void reset() {
internalBuffer.length = 0;
super.reset();
}
@Override
public void reInit(InputStream in) {
super.reInit(in);
onesCounter = 0;
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
super.doDecode(buf, internalBuffer, upto);
if (values.ints.length < internalBuffer.length) {
// need space for internalBuffer.length to internalBuffer.length*N,
// grow mildly at first
values.grow(internalBuffer.length * n/2);
}
for (int i = 0; i < internalBuffer.length; i++) {
int decode = internalBuffer.ints[i];
if (decode == 1) {
if (values.length == values.ints.length) {
values.grow(values.length + 10); // grow by few items, however not too many
}
// 1 is 1
values.ints[values.length++] = 1;
} else if (decode == 2) {
if (values.length + n >= values.ints.length) {
values.grow(values.length + n); // grow by few items, however not too many
}
// '2' means N 1's
for (int j = 0; j < n; j++) {
values.ints[values.length++] = 1;
}
} else {
if (values.length == values.ints.length) {
values.grow(values.length + 10); // grow by few items, however not too many
}
// any other value is val-1
values.ints[values.length++] = decode - 1;
}
}
}
@Override

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -50,11 +50,10 @@ import java.io.OutputStream;
*/
public class NOnesIntEncoder extends FourFlagsIntEncoder {
private final IntsRef internalBuffer;
/** Number of consecutive '1's to be translated into single target value '2'. */
private int n;
/** Counts the number of consecutive ones seen. */
private int onesCounter = 0;
private final int n;
/**
* Constructs an encoder with a given value of N (N: Number of consecutive
@ -62,38 +61,48 @@ public class NOnesIntEncoder extends FourFlagsIntEncoder {
*/
public NOnesIntEncoder(int n) {
this.n = n;
internalBuffer = new IntsRef(n);
}
@Override
public void close() throws IOException {
// We might have ones in our buffer, encode them as neccesary.
while (onesCounter-- > 0) {
super.encode(1);
protected void reset() {
internalBuffer.length = 0;
super.reset();
}
@Override
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
// make sure the internal buffer is large enough
if (values.length > internalBuffer.ints.length) {
internalBuffer.grow(values.length);
}
super.close();
}
@Override
public void encode(int value) throws IOException {
if (value == 1) {
// Increment the number of consecutive ones seen so far
if (++onesCounter == n) {
super.encode(2);
onesCounter = 0;
int onesCounter = 0;
for (int i = values.offset; i < upto; i++) {
int value = values.ints[i];
if (value == 1) {
// every N 1's should be encoded as '2'
if (++onesCounter == n) {
internalBuffer.ints[internalBuffer.length++] = 2;
onesCounter = 0;
}
} else {
// there might have been 1's that we need to encode
while (onesCounter > 0) {
--onesCounter;
internalBuffer.ints[internalBuffer.length++] = 1;
}
// encode value as value+1
internalBuffer.ints[internalBuffer.length++] = value + 1;
}
return;
}
// If it's not one - there might have been ones we had to encode prior to
// this value
// there might have been 1's that we need to encode
while (onesCounter > 0) {
--onesCounter;
super.encode(1);
internalBuffer.ints[internalBuffer.length++] = 1;
}
// encode value + 1 --> the translation.
super.encode(value + 1);
super.doEncode(internalBuffer, buf, internalBuffer.length);
}
@Override
@ -101,12 +110,6 @@ public class NOnesIntEncoder extends FourFlagsIntEncoder {
return new NOnesIntDecoder(n);
}
@Override
public void reInit(OutputStream out) {
super.reInit(out);
onesCounter = 0;
}
@Override
public String toString() {
return "NOnes (" + n + ") (" + super.toString() + ")";

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.StreamCorruptedException;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -21,41 +21,24 @@ import java.io.StreamCorruptedException;
*/
/**
* A simple stream decoder which can decode values encoded with
* {@link SimpleIntEncoder}.
* Decodes values encoded with {@link SimpleIntEncoder}.
*
* @lucene.experimental
*/
public class SimpleIntDecoder extends IntDecoder {
/**
* reusable buffer - allocated only once as this is not a thread-safe object
*/
private byte[] buffer = new byte[4];
@Override
public long decode() throws IOException {
// we need exactly 4 bytes to decode an int in this decoder impl, otherwise, throw an exception
int offset = 0;
while (offset < 4) {
int nRead = in.read(buffer, offset, 4 - offset);
if (nRead == -1) {
if (offset > 0) {
throw new StreamCorruptedException(
"Need 4 bytes for decoding an int, got only " + offset);
}
return EOS;
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
while (buf.offset < upto) {
if (values.length == values.ints.length) {
values.grow(values.length + 10); // grow by few items, however not too many
}
offset += nRead;
values.ints[values.length++] =
((buf.bytes[buf.offset++] & 0xFF) << 24) |
((buf.bytes[buf.offset++] & 0xFF) << 16) |
((buf.bytes[buf.offset++] & 0xFF) << 8) |
(buf.bytes[buf.offset++] & 0xFF);
}
int v = buffer[3] & 0xff;
v |= (buffer[2] << 8) & 0xff00;
v |= (buffer[1] << 16) & 0xff0000;
v |= (buffer[0] << 24) & 0xff000000;
return v;
}
@Override

View File

@ -1,6 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -26,22 +27,21 @@ import java.io.IOException;
*/
public class SimpleIntEncoder extends IntEncoder {
/**
* This method makes sure the value wasn't previously encoded by checking
* against the Set. If the value wasn't encoded, it's added to the Set, and
* encoded with {#link Vint8#encode}
*
* @param value
* an integer to be encoded
* @throws IOException
* possibly thrown by the OutputStream
*/
@Override
public void encode(int value) throws IOException {
out.write(value >>> 24);
out.write((value >> 16) & 0xFF);
out.write((value >> 8) & 0xFF);
out.write(value & 0xFF);
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
// ensure there's enough room in the buffer
int bytesNeeded = values.length * 4;
if (buf.bytes.length < bytesNeeded) {
buf.grow(bytesNeeded);
}
for (int i = values.offset; i < upto; i++) {
int value = values.ints[i];
buf.bytes[buf.length++] = (byte) (value >>> 24);
buf.bytes[buf.length++] = (byte) ((value >> 16) & 0xFF);
buf.bytes[buf.length++] = (byte) ((value >> 8) & 0xFF);
buf.bytes[buf.length++] = (byte) (value & 0xFF);
}
}
@Override

View File

@ -1,9 +1,10 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.OutputStream;
import java.util.Arrays;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -23,47 +24,21 @@ import java.util.Arrays;
/**
* An {@link IntEncoderFilter} which sorts the values to encode in ascending
* order before encoding them. Encoding therefore happens upon calling
* {@link #close()}. Since this encoder is usually chained with another encoder
* that relies on sorted values, it does not offer a default constructor.
* order before encoding them.
*
* @lucene.experimental
*/
public class SortingIntEncoder extends IntEncoderFilter {
private float grow = 2.0f;
private int index = 0;
private int[] set = new int[1024];
/** Initializes with the given encoder. */
public SortingIntEncoder(IntEncoder encoder) {
super(encoder);
}
@Override
public void close() throws IOException {
if (index == 0) {
return;
}
Arrays.sort(set, 0, index);
for (int i = 0; i < index; i++) {
encoder.encode(set[i]);
}
encoder.close();
index = 0;
super.close();
}
@Override
public void encode(int value) throws IOException {
if (index == set.length) {
int[] newSet = new int[(int) (set.length * grow)];
System.arraycopy(set, 0, newSet, 0, set.length);
set = newSet;
}
set[index++] = value;
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
Arrays.sort(values.ints, values.offset, upto);
encoder.doEncode(values, buf, upto);
}
@Override
@ -71,12 +46,6 @@ public class SortingIntEncoder extends IntEncoderFilter {
return encoder.createMatchingDecoder();
}
@Override
public void reInit(OutputStream out) {
super.reInit(out);
index = 0;
}
@Override
public String toString() {
return "Sorting (" + encoder.toString() + ")";

View File

@ -1,7 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import java.io.OutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -22,7 +22,7 @@ import java.io.OutputStream;
/**
* An {@link IntEncoderFilter} which ensures only unique values are encoded. The
* implementation assumes the values given to {@link #encode(int)} are sorted.
* implementation assumes the values given to {@link #encode(IntsRef, BytesRef)} are sorted.
* If this is not the case, you can chain this encoder with
* {@link SortingIntEncoder}.
*
@ -30,26 +30,23 @@ import java.io.OutputStream;
*/
public final class UniqueValuesIntEncoder extends IntEncoderFilter {
/**
* Denotes an illegal value which we can use to init 'prev' to. Since all
* encoded values are integers, this value is init to MAX_INT+1 and is of type
* long. Therefore we are guaranteed not to get this value in encode.
*/
private static final long ILLEGAL_VALUE = Integer.MAX_VALUE + 1;
private long prev = ILLEGAL_VALUE;
/** Constructs a new instance with the given encoder. */
public UniqueValuesIntEncoder(IntEncoder encoder) {
super(encoder);
}
@Override
public void encode(int value) throws IOException {
if (prev != value) {
encoder.encode(value);
prev = value;
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
int prev = values.ints[values.offset];
int idx = values.offset + 1;
for (int i = idx; i < upto; i++) {
if (values.ints[i] != prev) {
values.ints[idx++] = values.ints[i];
prev = values.ints[i];
}
}
values.length = idx - values.offset;
encoder.doEncode(values, buf, idx);
}
@Override
@ -57,12 +54,6 @@ public final class UniqueValuesIntEncoder extends IntEncoderFilter {
return encoder.createMatchingDecoder();
}
@Override
public void reInit(OutputStream out) {
super.reInit(out);
prev = ILLEGAL_VALUE;
}
@Override
public String toString() {
return "Unique (" + encoder.toString() + ")";

View File

@ -0,0 +1,138 @@
package org.apache.lucene.util.encoding;
import org.apache.lucene.util.BytesRef;
/*
* 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.
*/
/**
* Variable-length encoding of 32-bit integers, into 8-bit bytes. A number is
* encoded as follows:
* <ul>
* <li>If it is less than 127 and non-negative (i.e., if the number uses only 7
* bits), it is encoded as as single byte: 0bbbbbbb.
* <li>If its highest nonzero bit is greater than bit 6 (0x40), it is
* represented as a series of bytes, each byte's 7 LSB containing bits from the
* original value, with the MSB set for all but the last byte. The first encoded
* byte contains the highest nonzero bits from the original; the second byte
* contains the next 7 MSB; and so on, with the last byte containing the 7 LSB
* of the original.
* </ul>
* Examples:
* <ol>
* <li>n = 117 = 1110101: This has fewer than 8 significant bits, and so is
* encoded as 01110101 = 0x75.
* <li>n = 100000 = (binary) 11000011010100000. This has 17 significant bits,
* and so needs three Vint8 bytes. Left-zero-pad it to a multiple of 7 bits,
* then split it into chunks of 7 and add an MSB, 0 for the last byte, 1 for the
* others: 1|0000110 1|0001101 0|0100000 = 0x86 0x8D 0x20.
* </ol>
* {@link #encode(int, BytesRef)} and {@link #decode(BytesRef)} will correctly
* handle any 32-bit integer, but for negative numbers, and positive numbers
* with more than 28 significant bits, encoding requires 5 bytes; this is not an
* efficient encoding scheme for large positive numbers or any negative number.
*
* @lucene.experimental
*/
public class VInt8 {
/** The maximum number of bytes needed to encode an integer. */
public static final int MAXIMUM_BYTES_NEEDED = 5;
/**
* Decodes an int from the given bytes, starting at {@link BytesRef#offset}.
* Returns the decoded bytes and updates {@link BytesRef#offset}.
*/
public static int decode(BytesRef bytes) {
/*
This is the original code of this method, but a Hotspot bug
corrupted the for-loop of DataInput.readVInt() (see LUCENE-2975)
so the loop was unwounded here too, to be on the safe side
int value = 0;
while (true) {
byte first = bytes.bytes[bytes.offset++];
value |= first & 0x7F;
if ((first & 0x80) == 0) {
return value;
}
value <<= 7;
}
*/
// byte 1
byte b = bytes.bytes[bytes.offset++];
if (b >= 0) return b;
// byte 2
int value = b & 0x7F;
b = bytes.bytes[bytes.offset++];
value = (value << 7) | b & 0x7F;
if (b >= 0) return value;
// byte 3
b = bytes.bytes[bytes.offset++];
value = (value << 7) | b & 0x7F;
if (b >= 0) return value;
// byte 4
b = bytes.bytes[bytes.offset++];
value = (value << 7) | b & 0x7F;
if (b >= 0) return value;
// byte 5
b = bytes.bytes[bytes.offset++];
return (value << 7) | b & 0x7F;
}
/**
* Encodes the given number into bytes, starting at {@link BytesRef#length}.
* Assumes that the array is large enough.
*/
public static void encode(int value, BytesRef bytes) {
if ((value & ~0x7F) == 0) {
bytes.bytes[bytes.length] = (byte) value;
bytes.length++;
} else if ((value & ~0x3FFF) == 0) {
bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0x3F80) >> 7));
bytes.bytes[bytes.length + 1] = (byte) (value & 0x7F);
bytes.length += 2;
} else if ((value & ~0x1FFFFF) == 0) {
bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0x3F80) >> 7));
bytes.bytes[bytes.length + 2] = (byte) (value & 0x7F);
bytes.length += 3;
} else if ((value & ~0xFFFFFFF) == 0) {
bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
bytes.bytes[bytes.length + 2] = (byte) (0x80 | ((value & 0x3F80) >> 7));
bytes.bytes[bytes.length + 3] = (byte) (value & 0x7F);
bytes.length += 4;
} else {
bytes.bytes[bytes.length] = (byte) (0x80 | ((value & 0xF0000000) >> 28));
bytes.bytes[bytes.length + 1] = (byte) (0x80 | ((value & 0xFE00000) >> 21));
bytes.bytes[bytes.length + 2] = (byte) (0x80 | ((value & 0x1FC000) >> 14));
bytes.bytes[bytes.length + 3] = (byte) (0x80 | ((value & 0x3F80) >> 7));
bytes.bytes[bytes.length + 4] = (byte) (value & 0x7F);
bytes.length += 5;
}
}
private VInt8() {
// Just making it impossible to instantiate.
}
}

View File

@ -1,6 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -20,33 +21,19 @@ import java.io.IOException;
*/
/**
* An {@link IntDecoder} which can decode values encoded by
* {@link VInt8IntEncoder}.
* Decodes values encoded by {@link VInt8IntEncoder}.
*
* @lucene.experimental
*/
public class VInt8IntDecoder extends IntDecoder {
private boolean legalEOS = true;
@Override
public long decode() throws IOException {
int value = 0;
while (true) {
int first = in.read();
if (first < 0) {
if (!legalEOS) {
throw new IOException("Unexpected End-Of-Stream");
}
return EOS;
protected void doDecode(BytesRef buf, IntsRef values, int upto) {
while (buf.offset < upto) {
if (values.length == values.ints.length) {
values.grow(values.length + 10); // grow by few items, however not too many
}
value |= first & 0x7F;
if ((first & 0x80) == 0) {
legalEOS = true;
return value;
}
legalEOS = false;
value <<= 7;
values.ints[values.length++] = VInt8.decode(buf);
}
}

View File

@ -1,6 +1,7 @@
package org.apache.lucene.util.encoding;
import java.io.IOException;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -49,27 +50,14 @@ import java.io.IOException;
public class VInt8IntEncoder extends IntEncoder {
@Override
public void encode(int value) throws IOException {
if ((value & ~0x7F) == 0) {
out.write(value);
} else if ((value & ~0x3FFF) == 0) {
out.write(0x80 | (value >> 7));
out.write(0x7F & value);
} else if ((value & ~0x1FFFFF) == 0) {
out.write(0x80 | (value >> 14));
out.write(0x80 | (value >> 7));
out.write(0x7F & value);
} else if ((value & ~0xFFFFFFF) == 0) {
out.write(0x80 | (value >> 21));
out.write(0x80 | (value >> 14));
out.write(0x80 | (value >> 7));
out.write(0x7F & value);
} else {
out.write(0x80 | (value >> 28));
out.write(0x80 | (value >> 21));
out.write(0x80 | (value >> 14));
out.write(0x80 | (value >> 7));
out.write(0x7F & value);
protected void doEncode(IntsRef values, BytesRef buf, int upto) {
int maxBytesNeeded = 5 * values.length; // at most 5 bytes per VInt
if (buf.bytes.length < maxBytesNeeded) {
buf.grow(maxBytesNeeded);
}
for (int i = values.offset; i < upto; i++) {
VInt8.encode(values.ints[i], buf);
}
}

View File

@ -25,49 +25,8 @@ mechanisms to create new ones. The super class for all encoders is
encoders there is a matching {@link
org.apache.lucene.util.encoding.IntDecoder} implementation (not all
encoders need a decoder).
<p>An encoder encodes the integers that are passed to {@link
org.apache.lucene.util.encoding.IntEncoder#encode(int) encode} into a
set output stream (see {@link
org.apache.lucene.util.encoding.IntEncoder#reInit(OutputStream)
reInit}). One should always call {@link
org.apache.lucene.util.encoding.IntEncoder#close() close} when all
integers have been encoded, to ensure proper finish by the encoder. Some
encoders buffer values in-memory and encode in batches in order to
optimize the encoding, and not closing them may result in loss of
information or corrupt stream.
<p>A proper and typical usage of an encoder looks like this:
<blockquote><pre class="prettyprint"><code>
int[] data = &lt;the values to encode&gt;
IntEncoder encoder = new VInt8IntEncoder();
OutputStream out = new ByteArrayOutputStream();
encoder.reInit(out);
for (int val : data) {
encoder.encode(val);
}
encoder.close();
// Print the bytes in binary
byte[] bytes = out.toByteArray();
for (byte b : bytes) {
System.out.println(Integer.toBinaryString(b));
}
</code></pre></blockquote>
Each encoder also implements {@link
org.apache.lucene.util.encoding.IntEncoder#createMatchingDecoder()
createMatchingDecoder} which returns the matching decoder for this encoder.
As mentioned above, not all encoders have a matching decoder (like some
encoder filters which are explained next), however every encoder should
return a decoder following a call to that method. To complete the
example above, one can easily iterate over the decoded values like this:
<blockquote><pre class="prettyprint"><code>
IntDecoder d = e.createMatchingDecoder();
d.reInit(new ByteArrayInputStream(bytes));
long val;
while ((val = d.decode()) != IntDecoder.EOS) {
System.out.println(val);
}
</code></pre></blockquote>
<p>Some encoders don't perform any encoding at all, or do not include an
<p>
Some encoders don't perform any encoding at all, or do not include an
encoding logic. Those are called {@link
org.apache.lucene.util.encoding.IntEncoderFilter}s. A filter is an
encoder which delegates the encoding task to a given encoder, however
@ -76,91 +35,6 @@ example is {@link org.apache.lucene.util.encoding.DGapIntEncoder}
which encodes the gaps between values rather than the values themselves.
Another example is {@link
org.apache.lucene.util.encoding.SortingIntEncoder} which sorts all the
values in ascending order before they are sent for encoding. This
encoder aggregates the values in its {@link
org.apache.lucene.util.encoding.IntEncoder#encode(int) encode} implementation
and decoding only happens upon calling {@link
org.apache.lucene.util.encoding.IntEncoder#close() close}.
<h4>Extending IntEncoder</h4>
Extending {@link org.apache.lucene.util.encoding.IntEncoder} is a very
easy task. One only needs to implement {@link
org.apache.lucene.util.encoding.IntEncoder#encode(int) encode} and
{@link org.apache.lucene.util.encoding.IntEncoder#createMatchingDecoder()
createMatchingDecoder} as the base implementation takes care of
re-initializing the output stream and closing it. The following example
illustrates how can one write an encoder (and a matching decoder) which
'tags' the stream with type/ID of the encoder. Such tagging is important
in scenarios where an application uses different encoders for different
streams, and wants to manage some sort of mapping between an encoder ID
to an IntEncoder/Decoder implementation, so a proper decoder will be
initialized on the fly:
<blockquote><pre class="prettyprint"><code>
public class TaggingIntEncoder extends IntEncoderFilter {
public TaggingIntEncoder(IntEncoder encoder) {
super(encoder);
}
&#64;Override
public void encode(int value) throws IOException {
encoder.encode(value);
}
&#64;Override
public IntDecoder createMatchingDecoder() {
return new TaggingIntDecoder();
}
&#64;Override
public void reInit(OutputStream out) {
super.reInit(os);
// Assumes the application has a static EncodersMap class which is able to
// return a unique ID for a given encoder.
int encoderID = EncodersMap.getID(encoder);
this.out.write(encoderID);
}
&#64;Override
public String toString() {
return "Tagging (" + encoder.toString() + ")";
}
}
</code></pre></blockquote>
And the matching decoder:
<blockquote><pre class="prettyprint"><code>
public class TaggingIntDecoder extends IntDecoder {
// Will be initialized upon calling reInit.
private IntDecoder decoder;
&#64;Override
public void reInit(InputStream in) {
super.reInit(in);
// Read the ID of the encoder that tagged this stream.
int encoderID = in.read();
// Assumes EncodersMap can return the proper IntEncoder given the ID.
decoder = EncodersMap.getEncoder(encoderID).createMatchingDecoder();
}
&#64;Override
public long decode() throws IOException {
return decoder.decode();
}
&#64;Override
public String toString() {
return "Tagging (" + decoder == null ? "none" : decoder.toString() + ")";
}
}
</code></pre></blockquote>
The example implements <code>TaggingIntEncoder</code> as a filter over another
encoder. Even though it does not do any filtering on the actual values, it feels
right to present it as a filter. Anyway, this is just an example code and one
can choose to implement it however it makes sense to the application. For
simplicity, error checking was omitted from the sample code.
values in ascending order before they are sent for encoding.
</body>
</html>

View File

@ -4,7 +4,7 @@ import org.junit.Test;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.facet.example.ExampleResult;
import org.apache.lucene.facet.example.association.AssociationMain;
import org.apache.lucene.facet.example.association.CategoryAssociationsMain;
import org.apache.lucene.facet.search.results.FacetResultNode;
/*
@ -35,8 +35,8 @@ public class TestAssociationExample extends LuceneTestCase {
@Test
public void testAssociationExamples() throws Exception {
assertExampleResult(new AssociationMain().runSumIntAssociationSample(), EXPECTED_INT_SUM_RESULTS);
assertExampleResult(new AssociationMain().runSumFloatAssociationSample(), EXPECTED_FLOAT_SUM_RESULTS);
assertExampleResult(new CategoryAssociationsMain().runSumIntAssociationSample(), EXPECTED_INT_SUM_RESULTS);
assertExampleResult(new CategoryAssociationsMain().runSumFloatAssociationSample(), EXPECTED_FLOAT_SUM_RESULTS);
}
private void assertExampleResult(ExampleResult res, double[] expectedResults) {

View File

@ -19,8 +19,8 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.UnsafeByteArrayOutputStream;
import org.apache.lucene.util.encoding.DGapIntEncoder;
import org.apache.lucene.util.encoding.IntEncoder;
import org.apache.lucene.util.encoding.SortingIntEncoder;
@ -49,17 +49,19 @@ public class CategoryListIteratorTest extends LuceneTestCase {
private static final class DataTokenStream extends TokenStream {
private final PayloadAttribute payload = addAttribute(PayloadAttribute.class);
private final BytesRef buf;
private final IntEncoder encoder;
private final CharTermAttribute term = addAttribute(CharTermAttribute.class);
private int idx;
private PayloadAttribute payload = addAttribute(PayloadAttribute.class);
private byte[] buf = new byte[20];
UnsafeByteArrayOutputStream ubaos = new UnsafeByteArrayOutputStream(buf);
IntEncoder encoder;
private boolean exhausted = false;
private CharTermAttribute term = addAttribute(CharTermAttribute.class);
public DataTokenStream(String text, IntEncoder encoder) {
this.encoder = encoder;
term.setEmpty().append(text);
buf = new BytesRef();
payload.setPayload(buf);
}
public void setIdx(int idx) {
@ -73,30 +75,26 @@ public class CategoryListIteratorTest extends LuceneTestCase {
return false;
}
int[] values = data[idx];
ubaos.reInit(buf);
encoder.reInit(ubaos);
for (int val : values) {
encoder.encode(val);
}
encoder.close();
payload.setPayload(new BytesRef(buf, 0, ubaos.length()));
// must copy because encoders may change the buffer
encoder.encode(IntsRef.deepCopyOf(data[idx]), buf);
exhausted = true;
return true;
}
}
static final int[][] data = new int[][] {
new int[] { 1, 2 }, new int[] { 3, 4 }, new int[] { 1, 3 }, new int[] { 1, 2, 3, 4 },
static final IntsRef[] data = new IntsRef[] {
new IntsRef(new int[] { 1, 2 }, 0, 2),
new IntsRef(new int[] { 3, 4 }, 0, 2),
new IntsRef(new int[] { 1, 3 }, 0, 2),
new IntsRef(new int[] { 1, 2, 3, 4 }, 0, 4)
};
@Test
public void testPayloadIntDecodingIterator() throws Exception {
public void testPayloadCategoryListIteraor() throws Exception {
Directory dir = newDirectory();
DataTokenStream dts = new DataTokenStream("1",new SortingIntEncoder(
new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))));
final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
DataTokenStream dts = new DataTokenStream("1",encoder);
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)).setMergePolicy(newLogMergePolicy()));
for (int i = 0; i < data.length; i++) {
@ -108,21 +106,21 @@ public class CategoryListIteratorTest extends LuceneTestCase {
IndexReader reader = writer.getReader();
writer.close();
CategoryListIterator cli = new PayloadIntDecodingIterator(reader, new Term(
"f","1"), dts.encoder.createMatchingDecoder());
IntsRef ordinals = new IntsRef();
CategoryListIterator cli = new PayloadCategoryListIteraor(reader, new Term("f","1"), encoder.createMatchingDecoder());
cli.init();
int totalCategories = 0;
for (int i = 0; i < data.length; i++) {
Set<Integer> values = new HashSet<Integer>();
for (int j = 0; j < data[i].length; j++) {
values.add(data[i][j]);
values.add(data[i].ints[j]);
}
cli.skipTo(i);
long cat;
while ((cat = cli.nextCategory()) < Integer.MAX_VALUE) {
assertTrue("expected category not found: " + cat, values.contains((int) cat));
totalCategories ++;
cli.getOrdinals(i, ordinals);
assertTrue("no ordinals for document " + i, ordinals.length > 0);
for (int j = 0; j < ordinals.length; j++) {
assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
}
totalCategories += ordinals.length;
}
assertEquals("Missing categories!",10,totalCategories);
reader.close();
@ -135,8 +133,8 @@ public class CategoryListIteratorTest extends LuceneTestCase {
@Test
public void testPayloadIteratorWithInvalidDoc() throws Exception {
Directory dir = newDirectory();
DataTokenStream dts = new DataTokenStream("1",new SortingIntEncoder(
new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))));
final IntEncoder encoder = new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder())));
DataTokenStream dts = new DataTokenStream("1", encoder);
// this test requires that no payloads ever be randomly present!
final Analyzer noPayloadsAnalyzer = new Analyzer() {
@Override
@ -162,30 +160,27 @@ public class CategoryListIteratorTest extends LuceneTestCase {
IndexReader reader = writer.getReader();
writer.close();
CategoryListIterator cli = new PayloadIntDecodingIterator(reader, new Term(
"f","1"), dts.encoder.createMatchingDecoder());
IntsRef ordinals = new IntsRef();
CategoryListIterator cli = new PayloadCategoryListIteraor(reader, new Term("f","1"), encoder.createMatchingDecoder());
assertTrue("Failed to initialize payload iterator", cli.init());
int totalCats = 0;
int totalCategories = 0;
for (int i = 0; i < data.length; i++) {
// doc no. i
Set<Integer> values = new HashSet<Integer>();
for (int j = 0; j < data[i].length; j++) {
values.add(data[i][j]);
values.add(data[i].ints[j]);
}
boolean hasDoc = cli.skipTo(i);
if (hasDoc) {
assertTrue("Document " + i + " must not have a payload!", i == 0);
long cat;
while ((cat = cli.nextCategory()) < Integer.MAX_VALUE) {
assertTrue("expected category not found: " + cat, values.contains((int) cat));
++totalCats;
cli.getOrdinals(i, ordinals);
if (i == 0) {
assertTrue("document 0 must have a payload", ordinals.length > 0);
for (int j = 0; j < ordinals.length; j++) {
assertTrue("expected category not found: " + ordinals.ints[j], values.contains(ordinals.ints[j]));
}
totalCategories += ordinals.length;
} else {
assertFalse("Document " + i + " must have a payload!", i == 0);
assertTrue("only document 0 should have a payload", ordinals.length == 0);
}
}
assertEquals("Wrong number of total categories!", 2, totalCats);
assertEquals("Wrong number of total categories!", 2, totalCategories);
reader.close();
dir.close();

View File

@ -90,7 +90,9 @@ public class DrillDownTest extends LuceneTestCase {
paths.add(new CategoryPath("b"));
}
FacetFields facetFields = new FacetFields(taxoWriter);
facetFields.addFields(doc, paths);
if (paths.size() > 0) {
facetFields.addFields(doc, paths);
}
writer.addDocument(doc);
}

View File

@ -6,6 +6,7 @@ import java.util.List;
import java.util.Map;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.util.IntsRef;
import org.junit.After;
import org.junit.Before;
@ -118,18 +119,17 @@ public class TestCategoryListCache extends FacetTestBase {
@Override
public CategoryListIterator iterator(int partition) throws IOException {
final CategoryListIterator it = cld.iterator(partition);
return new CategoryListIterator() {
return new CategoryListIterator() {
@Override
public boolean skipTo(int docId) throws IOException {
return it.skipTo(docId);
}
@Override
public long nextCategory() throws IOException {
long res = it.nextCategory();
if (res>Integer.MAX_VALUE) {
return res;
public void getOrdinals(int docID, IntsRef ints) throws IOException {
it.getOrdinals(docID, ints);
for (int i = 0; i < ints.length; i++) {
if (ints.ints[i] > 1) {
ints.ints[i]--;
} else {
ints.ints[i]++;
}
}
return res>1 ? res-1 : res+1;
}
@Override
public boolean init() throws IOException {

View File

@ -0,0 +1,126 @@
package org.apache.lucene.facet.search.params;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Random;
import org.apache.lucene.document.Document;
import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.PerDimensionIndexingParams;
import org.apache.lucene.facet.search.CategoryListIterator;
import org.apache.lucene.facet.search.PayloadCategoryListIteraor;
import org.apache.lucene.facet.search.cache.CategoryListCache;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
import org.apache.lucene.facet.util.MultiCategoryListIterator;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.encoding.IntDecoder;
import org.junit.Test;
/*
* 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.
*/
public class MultiCategoryListIteratorTest extends LuceneTestCase {
@Test
public void testMultipleCategoryLists() throws Exception {
Random random = random();
int numDimensions = atLeast(random, 2); // at least 2 dimensions
String[] dimensions = new String[numDimensions];
for (int i = 0; i < numDimensions; i++) {
dimensions[i] = "dim" + i;
}
// build the PerDimensionIndexingParams
HashMap<CategoryPath,CategoryListParams> clps = new HashMap<CategoryPath,CategoryListParams>();
for (String dim : dimensions) {
CategoryPath cp = new CategoryPath(dim);
CategoryListParams clp = new CategoryListParams(new Term("$" + dim, CategoryListParams.DEFAULT_TERM.bytes()));
clps.put(cp, clp);
}
PerDimensionIndexingParams indexingParams = new PerDimensionIndexingParams(clps);
// index some documents
Directory indexDir = newDirectory();
Directory taxoDir = newDirectory();
IndexWriter indexWriter = new IndexWriter(indexDir, newIndexWriterConfig(TEST_VERSION_CURRENT, null).setMaxBufferedDocs(2));
TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
FacetFields facetFields = new FacetFields(taxoWriter, indexingParams);
int ndocs = atLeast(random, 10);
for (int i = 0; i < ndocs; i++) {
Document doc = new Document();
int numCategories = random.nextInt(numDimensions) + 1;
ArrayList<CategoryPath> categories = new ArrayList<CategoryPath>();
for (int j = 0; j < numCategories; j++) {
String dimension = dimensions[random.nextInt(dimensions.length)];
categories.add(new CategoryPath(dimension, Integer.toString(i)));
}
facetFields.addFields(doc, categories);
indexWriter.addDocument(doc);
}
IOUtils.close(indexWriter, taxoWriter);
// test the multi iterator
CategoryListCache clCache = null;
if (random.nextBoolean()) {
clCache = new CategoryListCache();
}
DirectoryReader indexReader = DirectoryReader.open(indexDir);
TaxonomyReader taxoReader = new DirectoryTaxonomyReader(taxoDir);
CategoryListIterator[] iterators = new CategoryListIterator[numDimensions];
for (int i = 0; i < iterators.length; i++) {
CategoryListParams clp = indexingParams.getCategoryListParams(new CategoryPath(dimensions[i]));
IntDecoder decoder = clp.createEncoder().createMatchingDecoder();
if (clCache != null && random.nextBoolean()) {
clCache.loadAndRegister(clp, indexReader, taxoReader, indexingParams);
iterators[i] = clCache.get(clp).iterator(0); // no partitions
} else {
iterators[i] = new PayloadCategoryListIteraor(indexReader, clp.getTerm(), decoder);
}
}
MultiCategoryListIterator cli = new MultiCategoryListIterator(iterators);
assertTrue("failed to init multi-iterator", cli.init());
IntsRef ordinals = new IntsRef();
int maxDoc = indexReader.maxDoc();
for (int i = 0; i < maxDoc; i++) {
cli.getOrdinals(i, ordinals);
assertTrue("document " + i + " does not have categories", ordinals.length > 0);
for (int j = 0; j < ordinals.length; j++) {
CategoryPath cp = taxoReader.getPath(ordinals.ints[j]);
assertNotNull("ordinal " + ordinals.ints[j] + " not found in taxonomy", cp);
if (cp.length == 2) {
assertEquals("invalid category for document " + i, i, Integer.parseInt(cp.components[1]));
}
}
}
IOUtils.close(indexReader, taxoReader);
IOUtils.close(indexDir, taxoDir);
}
}

View File

@ -1,271 +0,0 @@
package org.apache.lucene.facet.search.params;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.document.Document;
import org.apache.lucene.facet.index.FacetFields;
import org.apache.lucene.facet.index.params.CategoryListParams;
import org.apache.lucene.facet.index.params.FacetIndexingParams;
import org.apache.lucene.facet.search.CategoryListIterator;
import org.apache.lucene.facet.search.FacetArrays;
import org.apache.lucene.facet.search.FacetResultsHandler;
import org.apache.lucene.facet.search.FacetsAccumulator;
import org.apache.lucene.facet.search.ScoredDocIDs;
import org.apache.lucene.facet.search.StandardFacetsAccumulator;
import org.apache.lucene.facet.search.TopKFacetResultsHandler;
import org.apache.lucene.facet.search.cache.CategoryListCache;
import org.apache.lucene.facet.search.results.FacetResult;
import org.apache.lucene.facet.search.results.FacetResultNode;
import org.apache.lucene.facet.search.results.IntermediateFacetResult;
import org.apache.lucene.facet.taxonomy.CategoryPath;
import org.apache.lucene.facet.taxonomy.TaxonomyReader;
import org.apache.lucene.facet.taxonomy.TaxonomyWriter;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyReader;
import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
import org.apache.lucene.facet.util.ScoredDocIdsUtils;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.Test;
/*
* 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.
*/
/**
* Test faceted search with creation of multiple category list iterators by the
* same CLP, depending on the provided facet request
*/
public class MultiIteratorsPerCLParamsTest extends LuceneTestCase {
CategoryPath[][] perDocCategories = new CategoryPath[][] {
{ new CategoryPath("author", "Mark Twain"),
new CategoryPath("date", "2010") },
{ new CategoryPath("author", "Robert Frost"),
new CategoryPath("date", "2009") },
{ new CategoryPath("author", "Artur Miller"),
new CategoryPath("date", "2010") },
{ new CategoryPath("author", "Edgar Allan Poe"),
new CategoryPath("date", "2009") },
{ new CategoryPath("author", "Henry James"),
new CategoryPath("date", "2010") } };
String countForbiddenDimension;
@Test
public void testCLParamMultiIteratorsByRequest() throws Exception {
doTestCLParamMultiIteratorsByRequest(false);
}
@Test
public void testCLParamMultiIteratorsByRequestCacheCLI() throws Exception {
doTestCLParamMultiIteratorsByRequest(true);
}
private void doTestCLParamMultiIteratorsByRequest(boolean cacheCLI) throws Exception {
// Create a CLP which generates different CLIs according to the
// FacetRequest's dimension
CategoryListParams clp = new CategoryListParams();
FacetIndexingParams iParams = new FacetIndexingParams(clp);
Directory indexDir = newDirectory();
Directory taxoDir = newDirectory();
populateIndex(iParams, indexDir, taxoDir);
TaxonomyReader taxo = new DirectoryTaxonomyReader(taxoDir);
IndexReader reader = DirectoryReader.open(indexDir);
CategoryListCache clCache = null;
if (cacheCLI) {
// caching the iteratorr, so:
// 1: create the cached iterator, using original params
clCache = new CategoryListCache();
clCache.loadAndRegister(clp, reader, taxo, iParams);
}
ScoredDocIDs allDocs = ScoredDocIdsUtils
.createAllDocsScoredDocIDs(reader);
// Search index with 'author' should filter ONLY ordinals whose parent
// is 'author'
countForbiddenDimension = "date";
validateFacetedSearch(iParams, taxo, reader, clCache, allDocs, "author", 5, 5);
// Search index with 'date' should filter ONLY ordinals whose parent is
// 'date'
countForbiddenDimension = "author";
validateFacetedSearch(iParams, taxo, reader, clCache, allDocs, "date", 5, 2);
// Search index with both 'date' and 'author'
countForbiddenDimension = null;
validateFacetedSearch(iParams, taxo, reader, clCache, allDocs, new String[] {
"author", "date" }, new int[] { 5, 5 }, new int[] { 5, 2 });
taxo.close();
reader.close();
indexDir.close();
taxoDir.close();
}
private void validateFacetedSearch(FacetIndexingParams iParams,
TaxonomyReader taxo, IndexReader reader, CategoryListCache clCache,
ScoredDocIDs allDocs, String dimension, int expectedValue, int expectedNumDescendants) throws IOException {
validateFacetedSearch(iParams, taxo, reader, clCache, allDocs,
new String[] { dimension }, new int[] { expectedValue },
new int[] { expectedNumDescendants });
}
private void validateFacetedSearch(FacetIndexingParams iParams,
TaxonomyReader taxo, IndexReader reader, final CategoryListCache clCache,
ScoredDocIDs allDocs, String[] dimension, int[] expectedValue,
int[] expectedNumDescendants) throws IOException {
List<FacetRequest> facetRequests = new ArrayList<FacetRequest>();
for (String dim : dimension) {
facetRequests.add(new PerDimCountFacetRequest(new CategoryPath(dim), 10));
}
FacetSearchParams sParams = new FacetSearchParams(facetRequests, iParams) {
@Override
public CategoryListCache getCategoryListCache() {
return clCache;
}
};
FacetsAccumulator acc = new StandardFacetsAccumulator(sParams, reader, taxo);
// no use to test this with complement since at that mode all facets are taken
acc.setComplementThreshold(FacetsAccumulator.DISABLE_COMPLEMENT);
List<FacetResult> results = acc.accumulate(allDocs);
assertEquals("Wrong #results", dimension.length, results.size());
for (int i = 0; i < results.size(); i++) {
FacetResult res = results.get(i);
assertEquals("wrong num-descendants for dimension " + dimension[i],
expectedNumDescendants[i], res.getNumValidDescendants());
FacetResultNode resNode = res.getFacetResultNode();
assertEquals("wrong value for dimension " + dimension[i],
expectedValue[i], (int) resNode.getValue());
}
}
private void populateIndex(FacetIndexingParams iParams, Directory indexDir,
Directory taxoDir) throws Exception {
RandomIndexWriter writer = new RandomIndexWriter(random(), indexDir,
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random(), MockTokenizer.KEYWORD, false)));
TaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
FacetFields facetFields = new FacetFields(taxoWriter, iParams);
for (CategoryPath[] categories : perDocCategories) {
Document doc = new Document();
facetFields.addFields(doc, Arrays.asList(categories));
writer.addDocument(doc);
}
taxoWriter.commit();
writer.commit();
taxoWriter.close();
writer.close();
}
private class PerDimCountFacetRequest extends CountFacetRequest {
public PerDimCountFacetRequest(CategoryPath path, int num) {
super(path, num);
}
@Override
public CategoryListIterator createCategoryListIterator(IndexReader reader,
TaxonomyReader taxo, FacetSearchParams sParams, int partition) throws IOException {
// categories of certain dimension only
return new PerDimensionCLI(taxo, super.createCategoryListIterator(
reader, taxo, sParams, partition), getCategoryPath());
}
@Override
/** Override this method just for verifying that only specified facets are iterated.. */
public FacetResultsHandler createFacetResultsHandler(
TaxonomyReader taxonomyReader) {
return new TopKFacetResultsHandler(taxonomyReader, this) {
@Override
public IntermediateFacetResult fetchPartitionResult(
FacetArrays facetArrays, int offset) throws IOException {
final IntermediateFacetResult res = super.fetchPartitionResult(facetArrays, offset);
if (countForbiddenDimension!=null) {
int ord = taxonomyReader.getOrdinal(new CategoryPath(countForbiddenDimension));
assertEquals("Should not have accumulated for dimension '"+countForbiddenDimension+"'!",0,facetArrays.getIntArray()[ord]);
}
return res;
}
};
}
}
/**
* a CLI which filters another CLI for the dimension of the provided
* category-path
*/
private static class PerDimensionCLI implements CategoryListIterator {
private final CategoryListIterator superCLI;
private final int[] parentArray;
private final int parentOrdinal;
PerDimensionCLI(TaxonomyReader taxo, CategoryListIterator superCLI,
CategoryPath requestedPath) throws IOException {
this.superCLI = superCLI;
if (requestedPath == null) {
parentOrdinal = 0;
} else {
CategoryPath cp = new CategoryPath(requestedPath.components[0]);
parentOrdinal = taxo.getOrdinal(cp);
}
parentArray = taxo.getParallelTaxonomyArrays().parents();
}
@Override
public boolean init() throws IOException {
return superCLI.init();
}
@Override
public long nextCategory() throws IOException {
long next;
while ((next = superCLI.nextCategory()) <= Integer.MAX_VALUE
&& !isInDimension((int) next)) {
}
return next;
}
/** look for original parent ordinal, meaning same dimension */
private boolean isInDimension(int ordinal) {
while (ordinal > 0) {
if (ordinal == parentOrdinal) {
return true;
}
ordinal = parentArray[ordinal];
}
return false;
}
@Override
public boolean skipTo(int docId) throws IOException {
return superCLI.skipTo(docId);
}
}
}

View File

@ -1,141 +0,0 @@
package org.apache.lucene.util;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import org.junit.Test;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.Vint8;
/*
* 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.
*/
/**
* Tests the {@link Vint8} class.
*/
public class Vint8Test extends LuceneTestCase {
/**
* Tests the position wrapper.
* @throws Exception For any reason.
*/
@Test
public void testPosition() throws Exception {
Vint8.Position pos = new Vint8.Position();
assertEquals(0, pos.pos);
pos = new Vint8.Position(12345);
assertEquals(12345, pos.pos);
}
private static int[] testValues = {
-1000000000,
-1, 0, (1 << 7) - 1, 1 << 7, (1 << 14) - 1, 1 << 14,
(1 << 21) - 1, 1 << 21, (1 << 28) - 1, 1 << 28
};
private static int[] bytesNeededTestValues = {
5, 5, 1, 1, 2, 2, 3, 3, 4, 4, 5
};
/**
* Tests the {@code bytesNeeded} method.
*/
@Test
public void testBytesNeeded() {
assertEquals(5, Vint8.MAXIMUM_BYTES_NEEDED);
for (int j = 0; j < testValues.length; j++) {
assertEquals(bytesNeededTestValues[j], Vint8.bytesNeeded(testValues[j]));
}
}
/**
* Tests encoding and decoding to and from a stream.
*/
@Test
public void testStreamEncodingAndDecoding() throws IOException {
ByteArrayOutputStream baos = new ByteArrayOutputStream(256);
int expectedSize = 0;
for (int j = 0; j < testValues.length; j++) {
Vint8.encode(testValues[j], baos);
expectedSize += bytesNeededTestValues[j];
}
assertEquals(expectedSize, baos.size());
ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
for (int j = 0; j < testValues.length; j++) {
assertEquals(testValues[j], Vint8.decode(bais));
}
assertEquals(0, bais.available());
}
/**
* Tests encoding and decoding to and from an array.
*/
@Test
public void testArrayEncodingAndDecoding() throws IOException {
byte[] byteArray = new byte[256];
int position = 0, expectedSize = 0;
for (int j = 0; j < testValues.length; j++) {
position += Vint8.encode(testValues[j], byteArray, position);
expectedSize += bytesNeededTestValues[j];
}
assertEquals(expectedSize, position);
Vint8.Position pos = new Vint8.Position();
for (int j = 0; j < testValues.length; j++) {
assertEquals(testValues[j], Vint8.decode(byteArray, pos));
}
assertEquals(expectedSize, pos.pos);
}
/**
* The result of encoding the test values with the current algorithm. If these
* values are changed to match an algorithm change, compatibility with legacy
* data will be broken.
*/
private static final byte[] encodedTestValues = {
-4, -93, -108, -20, 0, -1, -1, -1, -1, 127, 0, 127, -127, 0, -1, 127,
-127, -128, 0, -1, -1, 127, -127, -128, -128, 0, -1, -1, -1, 127, -127,
-128, -128, -128, 0
};
/**
* Tests algorithm.
*/
@Test
public void testLegacyCompatibility() throws IOException {
/* To generate the encoded test values:
byte[] byteArray = new byte[256];
int position = 0, expectedSize = 0;
for (int j = 0; j < testValues.length; j++) {
position += Vint8.encode(testValues[j], byteArray, position);
expectedSize += bytesNeededTestValues[j];
}
assertEquals(expectedSize, position);
Vint8.Position pos = new Vint8.Position();
for (int j = 0; j < expectedSize; j++) {
System.out.print(byteArray[j] + ", ");
}
System.out.flush();
pos.pos = 0;
*/
Vint8.Position pos = new Vint8.Position();
for (int j = 0; j < testValues.length; j++) {
assertEquals(testValues[j], Vint8.decode(encodedTestValues, pos));
}
}
} // end class Vint8Test

View File

@ -1,21 +1,12 @@
package org.apache.lucene.util.encoding;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.text.NumberFormat;
import java.util.Arrays;
import java.util.Locale;
import org.apache.lucene.util.encoding.DGapIntEncoder;
import org.apache.lucene.util.encoding.EightFlagsIntEncoder;
import org.apache.lucene.util.encoding.FourFlagsIntEncoder;
import org.apache.lucene.util.encoding.IntDecoder;
import org.apache.lucene.util.encoding.IntEncoder;
import org.apache.lucene.util.encoding.NOnesIntEncoder;
import org.apache.lucene.util.encoding.SortingIntEncoder;
import org.apache.lucene.util.encoding.UniqueValuesIntEncoder;
import org.apache.lucene.util.encoding.VInt8IntEncoder;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -40,8 +31,8 @@ public class EncodingSpeed {
private static int[] data9910 = null;
private static int[] data501871 = null;
private static int[] data10k = null;
private static String resultsFormat = "%-20s %10s %20d %26s %20d %26s";
private static String headerFormat = "%-20s %10s %20s %26s %20s %26s";
private static String resultsFormat = "%-60s %10s %20d %26s %20d %26s";
private static String headerFormat = "%-60s %10s %20s %26s %20s %26s";
private static int integers = 100000000;
private static NumberFormat nf;
@ -53,8 +44,14 @@ public class EncodingSpeed {
testFacetIDs(data501871, 501871);
}
private static void testFacetIDs(int[] facetIDs, int docID)
throws IOException {
private static IntsRef newIntsRef(int[] data) {
IntsRef res = new IntsRef(data.length);
System.arraycopy(data, 0, res.ints, 0, data.length);
res.length = data.length;
return res;
}
private static void testFacetIDs(int[] facetIDs, int docID) throws IOException {
int loopFactor = integers / facetIDs.length;
System.out
.println("\nEstimating ~"
@ -88,68 +85,53 @@ public class EncodingSpeed {
System.out.println();
}
private static void encoderTest(IntEncoder encoder, int[] data,
int loopFactor) throws IOException {
private static void encoderTest(IntEncoder encoder, int[] values, int loopFactor) throws IOException {
long startTime, endTime;
ByteArrayOutputStream baos = new ByteArrayOutputStream();
BytesRef bytes = new BytesRef(values.length); // at least one byte per value
// -- Looping 100 times as a warm up --------------------------
for (int i = 100; i != 0; --i) {
baos.reset();
encoder.reInit(baos);
for (int value : data) {
encoder.encode(value);
}
encoder.close();
IntsRef data = newIntsRef(values);
encoder.encode(data, bytes);
}
// -----------------------------------------------------------
startTime = System.currentTimeMillis();
long encodeTime = 0;
for (int factor = loopFactor; factor > 0; --factor) {
baos.reset();
encoder.reInit(baos);
for (int value : data) {
encoder.encode(value);
}
encoder.close();
IntsRef data = newIntsRef(values);
long start = System.currentTimeMillis();
encoder.encode(data, bytes);
encodeTime += System.currentTimeMillis() - start;
}
endTime = System.currentTimeMillis();
long encodeTime = endTime - startTime;
ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
IntsRef decoded = new IntsRef(values.length);
int encodedSize = bytes.length;
IntDecoder decoder = encoder.createMatchingDecoder();
decoder.reInit(bais);
// -- Looping 100 times as a warm up --------------------------
for (int i = 100; i != 0; --i) {
bais.mark(baos.size());
while (decoder.decode() != IntDecoder.EOS) {
}
bais.reset();
decoder.reInit(bais);
decoder.decode(bytes, decoded);
}
// -----------------------------------------------------------
decoder.reInit(bais);
startTime = System.currentTimeMillis();
long decodeTime = 0;
for (int i = loopFactor; i > 0; --i) {
bais.mark(baos.size());
while (decoder.decode() != IntDecoder.EOS) {
}
bais.reset();
decoder.reInit(bais);
long start = System.currentTimeMillis();
decoder.decode(bytes, decoded);
decodeTime += System.currentTimeMillis() - start;
}
if (decoded.length != values.length) {
throw new RuntimeException("wrong num values. expected=" + values.length + " actual=" + decoded.length +
" decoder=" + decoder);
}
endTime = System.currentTimeMillis();
long decodeTime = endTime - startTime;
System.out.println(String.format(Locale.ROOT, resultsFormat, encoder, nf.format(baos
.size()
* 8.0 / data.length), encodeTime, nf.format(encodeTime
* 1000000.0 / (loopFactor * data.length)), decodeTime, nf
.format(decodeTime * 1000000.0 / (loopFactor * data.length))));
System.out.println(String.format(Locale.ROOT, resultsFormat, encoder,
nf.format(encodedSize * 8.0 / values.length),
encodeTime,
nf.format(encodeTime * 1000000.0 / (loopFactor * values.length)),
decodeTime,
nf.format(decodeTime * 1000000.0 / (loopFactor * values.length))));
}
static {

View File

@ -1,14 +1,13 @@
package org.apache.lucene.util.encoding;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.HashSet;
import java.util.TreeSet;
import org.junit.Test;
import java.util.Arrays;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.BeforeClass;
import org.junit.Test;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -29,65 +28,33 @@ import org.apache.lucene.util.LuceneTestCase;
public class EncodingTest extends LuceneTestCase {
static int[] data = null;
private static TreeSet<Long> dataSet = new TreeSet<Long>();
static {
setData();
}
@Test
public void testVInt8() throws Exception {
encoderTest(new VInt8IntEncoder());
private static IntsRef uniqueSortedData, data;
@BeforeClass
public static void beforeClassEncodingTest() throws Exception {
int capacity = atLeast(10000);
data = new IntsRef(capacity);
for (int i = 0; i < 10; i++) {
data.ints[i] = i + 1; // small values
}
for (int i = 10; i < data.ints.length; i++) {
data.ints[i] = random().nextInt(Integer.MAX_VALUE - 1) + 1; // some encoders don't allow 0
}
data.length = data.ints.length;
// cover negative numbers;
IntEncoder enc = new VInt8IntEncoder();
ByteArrayOutputStream baos = new ByteArrayOutputStream();
enc.reInit(baos);
enc.encode(-1);
IntDecoder dec = enc.createMatchingDecoder();
dec.reInit(new ByteArrayInputStream(baos.toByteArray()));
assertEquals(-1, dec.decode());
uniqueSortedData = IntsRef.deepCopyOf(data);
Arrays.sort(uniqueSortedData.ints);
uniqueSortedData.length = 0;
int prev = -1;
for (int i = 0; i < uniqueSortedData.ints.length; i++) {
if (uniqueSortedData.ints[i] != prev) {
uniqueSortedData.ints[uniqueSortedData.length++] = uniqueSortedData.ints[i];
prev = uniqueSortedData.ints[i];
}
}
}
@Test
public void testSimpleInt() {
encoderTest(new SimpleIntEncoder());
}
@Test
public void testSortingUniqueValues() {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new VInt8IntEncoder())));
}
@Test
public void testSortingUniqueDGap() {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))));
}
@Test
public void testSortingUniqueDGapEightFlags() {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new EightFlagsIntEncoder()))));
}
@Test
public void testSortingUniqueDGapFourFlags() {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new FourFlagsIntEncoder()))));
}
@Test
public void testSortingUniqueDGapNOnes4() {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(4)))));
}
@Test
public void testSortingUniqueDGapNOnes3() {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(3)))));
}
private static void encoderTest(IntEncoder encoder) {
private static void encoderTest(IntEncoder encoder, IntsRef data, IntsRef expected) throws IOException {
// ensure toString is implemented
String toString = encoder.toString();
assertFalse(toString.startsWith(encoder.getClass().getName() + "@"));
@ -95,320 +62,90 @@ public class EncodingTest extends LuceneTestCase {
toString = decoder.toString();
assertFalse(toString.startsWith(decoder.getClass().getName() + "@"));
ByteArrayOutputStream baos = new ByteArrayOutputStream();
try {
encoding(encoder, baos);
decoding(baos, encoder.createMatchingDecoder());
} catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
baos.reset();
try {
encoding(encoder, baos);
decoding(baos, encoder.createMatchingDecoder());
} catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
}
BytesRef bytes = new BytesRef(100); // some initial capacity - encoders should grow the byte[]
IntsRef values = new IntsRef(100); // some initial capacity - decoders should grow the int[]
encoding(encoder, data, bytes);
decoding(bytes, values, encoder.createMatchingDecoder());
assertTrue(expected.intsEquals(values));
}
private static void encoding(IntEncoder encoder, ByteArrayOutputStream baos) throws IOException {
encoder.reInit(baos);
for (int value : data) {
encoder.encode(value);
private static void encoding(IntEncoder encoder, IntsRef data, BytesRef bytes) throws IOException {
final IntsRef values;
if (random().nextBoolean()) { // randomly set the offset
values = new IntsRef(data.length + 1);
System.arraycopy(data.ints, 0, values.ints, 1, data.length);
values.offset = 1; // ints start at index 1
values.length = data.length;
} else {
// need to copy the array because it may be modified by encoders (e.g. sorting)
values = IntsRef.deepCopyOf(data);
}
encoder.close();
baos.reset();
encoder.reInit(baos);
for (int value : data) {
encoder.encode(value);
}
encoder.close();
encoder.encode(values, bytes);
}
private static void decoding(ByteArrayOutputStream baos, IntDecoder decoder)
throws IOException {
ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
decoder.reInit(bais);
HashSet<Long> set = new HashSet<Long>();
long value = 0;
while ((value = decoder.decode()) != IntDecoder.EOS) {
set.add(value);
private static void decoding(BytesRef bytes, IntsRef values, IntDecoder decoder) throws IOException {
int offset = 0;
if (random().nextBoolean()) { // randomly set the offset and length to other than 0,0
bytes.grow(bytes.length + 1); // ensure that we have enough capacity to shift values by 1
bytes.offset = 1; // bytes start at index 1 (must do that after grow)
System.arraycopy(bytes.bytes, 0, bytes.bytes, 1, bytes.length);
offset = 1;
}
assertEquals(dataSet.size(), set.size());
assertTrue(set.equals(dataSet));
set.clear();
bais.reset();
decoder.reInit(bais);
value = 0;
while ((value = decoder.decode()) != IntDecoder.EOS) {
set.add(value);
}
assertEquals(dataSet.size(), set.size());
assertTrue(set.equals(dataSet));
decoder.decode(bytes, values);
assertEquals(offset, bytes.offset); // decoders should not mess with offsets
}
private static void setData() {
data = new int[] { 2, 4, 86133, 11, 16505, 86134, 86135, 86136, 1290,
86137, 86138, 32473, 19346, 32474, 4922, 32475, 86139, 16914,
86140, 86141, 86142, 86143, 32478, 86144, 86145, 32480, 4884,
4887, 32481, 86146, 16572, 86147, 16295, 165, 86148, 3183,
21920, 21921, 21922, 555, 4006, 32484, 21925, 21926, 13775,
86149, 13777, 85833, 85834, 13779, 13773, 13780, 75266, 17674,
13784, 13785, 13786, 13787, 13788, 6258, 86150, 13790, 75267,
13793, 13794, 13795, 312, 4914, 4915, 6222, 86151, 4845, 4883,
4918, 4894, 4919, 86152, 4921, 6223, 6224, 6225, 6226, 67909,
6229, 18170, 6230, 5198, 25625, 6231, 6232, 6233, 1808, 6234,
6235, 6236, 41376, 6238, 6239, 67911, 6240, 86153, 6243, 6244,
83549, 6246, 6247, 6248, 6249, 782, 444, 6251, 6250, 19863,
28963, 310, 2234, 144, 2236, 2309, 69437, 2311, 2325, 2241,
69438, 69439, 2244, 2245, 2246, 23504, 2314, 69440, 36603,
2250, 2268, 2271, 2251, 2254, 2255, 2257, 2240, 36604, 84726,
36605, 84727, 2262, 2263, 18431, 38853, 2317, 2149, 2326, 2327,
2329, 3980, 2275, 2277, 2258, 84728, 2260, 84729, 84730, 13766,
36607, 2282, 2283, 84731, 2284, 2286, 2287, 2337, 7424, 2288,
2338, 3522, 2290, 84733, 32902, 371, 37708, 2096, 3065, 3066,
375, 377, 374, 378, 2100, 86154, 381, 382, 58795, 379, 383,
384, 385, 4449, 387, 388, 389, 390, 9052, 391, 18358, 2107,
394, 2111, 2108, 393, 2109, 395, 86155, 86156, 397, 2113, 398,
399, 400, 273, 274, 275, 40980, 276, 277, 31716, 279, 280,
31717, 281, 282, 1628, 1623, 1624, 1625, 2052, 1626, 725, 727,
728, 729, 730, 731, 1633, 733, 734, 735, 86157, 737, 738, 739,
1634, 3563, 3564, 3565, 1667, 12461, 76276, 3567, 5413, 77622,
5415, 5416, 5417, 5418, 107, 86158, 7784, 15363, 153, 3723,
2713, 7786, 3835, 7787, 86159, 7789, 7791, 7792, 7794, 86160,
7796, 86161, 6708, 7798, 7799, 7800, 7801, 7802, 7803, 1665,
43150, 15365, 1581, 5656, 43152, 80258, 7450, 39922, 86162,
51587, 9059, 4606, 396, 86163, 86164, 7250, 401, 403, 2860,
33281, 2964, 408, 9119, 409, 86165, 7669, 2861, 410, 413,
86166, 414, 415, 33282, 405, 33283, 7498, 2865, 7230, 33284,
2866, 86167, 2867, 47518, 2868, 86168, 2869, 2870, 4712, 7096,
28484, 6913, 6914, 6915, 6916, 37169, 37170, 7103, 28269, 6919,
86169, 45431, 6922, 7104, 6923, 7108, 6924, 6925, 6926, 6927,
6928, 86170, 86171, 86172, 6930, 6931, 6932, 6934, 6935, 6936,
451, 6937, 6938, 4756, 3554, 5309, 8145, 3586, 16417, 9767,
14126, 25854, 6580, 10174, 86173, 5519, 21309, 8561, 20938,
10386, 86174, 781, 2030, 16419, 30323, 16420, 16421, 16424,
86175, 86176, 86177, 28871, 86178, 28872, 63980, 6329, 49561,
4271, 38778, 86179, 86180, 20126, 16245, 193, 195, 196, 197,
56973, 199, 200, 201, 202, 203, 204, 56974, 56975, 205, 206,
4662, 207, 208, 209, 210, 211, 212, 47901, 641, 642, 643, 1380,
1079, 47902, 1381, 1081, 1082, 1083, 47903, 1382, 47904, 1087,
47905, 965, 966, 1298, 968, 1387, 1300, 50288, 971, 972, 973,
974, 23974, 22183, 1390, 23313, 1389, 1391, 902, 23029, 296,
1304, 1395, 1303, 1309, 1308, 50289, 1312, 50290, 50291, 1315,
1317, 9270, 19796, 3605, 1320, 1321, 44946, 1322, 1323, 50292,
967, 1587, 1326, 1331, 17482, 633, 29115, 53858, 29118, 29119,
62624, 44494, 6965, 6966, 6959, 6967, 71562, 6969, 23459,
23460, 17464, 4225, 23461, 23462, 23463, 5893, 23464, 17467,
17468, 23465, 12562, 1405, 1406, 1407, 960, 961, 962, 687, 963,
86181, 86182, 5997, 10812, 11976, 11977, 1850, 577, 13393,
10810, 13394, 65040, 86183, 3935, 3936, 3937, 710, 86184, 5785,
5786, 29949, 5787, 5788, 283, 284, 2687, 285, 286, 287, 2689,
288, 289, 8880, 290, 2690, 13899, 991, 292, 295, 42007, 35616,
63103, 298, 299, 3520, 297, 9024, 303, 301, 302, 300, 31345,
3719, 304, 305, 306, 307, 308, 368, 364, 85002, 9026, 63105,
367, 39596, 25835, 19746, 293, 294, 26505, 85003, 18377, 56785,
10122, 10123, 10124, 86185, 39863, 86186, 10125, 39865, 4066,
4067, 24257, 4068, 4070, 86187, 4073, 4074, 86188, 4076, 7538,
4077, 86189, 4078, 4079, 7540, 7541, 4084, 4085, 7542, 86190,
4086, 86191, 4087, 4088, 86192, 7545, 44874, 7821, 44875,
86193, 4286, 86194, 51470, 17609, 1408, 47486, 1411, 1412,
47487, 1413, 1414, 1417, 1415, 47488, 1416, 1418, 1420, 470,
1422, 1423, 1424, 5001, 5002, 47489, 1427, 1429, 1430, 31811,
1432, 1433, 47490, 1435, 3753, 1437, 1439, 1440, 47491, 1443,
47492, 1446, 5004, 5005, 1450, 47493, 353, 1452, 42145, 3103,
3402, 3104, 3105, 4780, 3106, 3107, 3108, 12157, 3111, 42146,
42147, 3114, 4782, 42148, 3116, 3117, 42149, 42150, 3407, 3121,
3122, 18154, 3126, 3127, 3128, 3410, 3130, 3411, 3412, 3415,
24241, 3417, 3418, 3449, 42151, 3421, 3422, 7587, 42152, 3424,
3427, 3428, 3448, 3430, 3432, 42153, 42154, 41648, 1991, 407,
57234, 411, 2862, 57235, 2863, 18368, 57236, 2874, 7350, 4115,
2876, 2877, 17975, 86195, 4116, 2881, 2882, 2883, 2886, 463,
870, 872, 873, 874, 875, 8783, 8784, 877, 1480, 1481, 459,
2778, 881, 8785, 2779, 8786, 8787, 8788, 886, 887, 8789, 889,
8790, 86196, 6920, 86197, 5080, 5081, 7395, 7396, 9395, 9396,
1528, 42737, 805, 86198, 1209, 13595, 4126, 9680, 34368, 9682,
86199, 86200, 174, 175, 176, 177, 178, 179, 180, 182, 183,
1477, 31138, 186, 172, 187, 188, 189, 190, 191, 458, 871,
31294, 31295, 27604, 31296, 31297, 882, 883, 884, 31298, 890,
1089, 1488, 1489, 1092, 1093, 1094, 1095, 1096, 1097, 1490,
1098, 1495, 1502, 1099, 1100, 1101, 1493, 2997, 12223, 1103,
2654, 1498, 1499, 1500, 80615, 80616, 80617, 33359, 86201,
9294, 1501, 86202, 1506, 1507, 23454, 38802, 38803, 1014,
86203, 5583, 5584, 651, 74717, 5586, 5587, 5588, 5589, 74720,
5590, 38808, 33527, 78330, 10930, 5119, 10931, 1000, 10928,
10932, 10933, 10934, 10935, 5863, 10936, 86204, 10938, 10939,
86205, 192, 194, 38754, 38755, 198, 38756, 38757, 38758, 2842,
640, 22780, 22781, 1080, 86206, 86207, 1084, 1086, 1088, 63916,
9412, 970, 9413, 9414, 9415, 9416, 9417, 1310, 7168, 7169,
1318, 9418, 1324, 39159, 1804, 1557, 24850, 41499, 1560, 41500,
1562, 1563, 1565, 1927, 1928, 1566, 1569, 1570, 1571, 1572,
1573, 1574, 1575, 1576, 2674, 2677, 2678, 2679, 2946, 2682,
2676, 2683, 2947, 1156, 1157, 1158, 1467, 1160, 1468, 1469,
1161, 1162, 1163, 4369, 1165, 1166, 1167, 12923, 2917, 1169,
1170, 1171, 1172, 1173, 1174, 1175, 1176, 1177, 18153, 8359,
1178, 1164, 1191, 1180, 12924, 86208, 86209, 54817, 66962,
2476, 86210, 86211, 41820, 41821, 41822, 41824, 1130, 1131,
1132, 32692, 1134, 34848, 1136, 1133, 1137, 1138, 1139, 1140,
1141, 1143, 1144, 1145, 34849, 2639, 34850, 1146, 1147, 1148,
34851, 1150, 1151, 1152, 1153, 1154, 1155, 1678, 1679, 1680,
1681, 40870, 2059, 1685, 1686, 32686, 14970, 1688, 1689, 86212,
1692, 1682, 1693, 1695, 1696, 1698, 12955, 8909, 41690, 1700,
41691, 86213, 30949, 41692, 1703, 1704, 1705, 41693, 14976,
1708, 2071, 1709, 1710, 1711, 1712, 1727, 86214, 86215, 86216,
1715, 86217, 1714, 1717, 1690, 41697, 86218, 1720, 86219, 2073,
41699, 1724, 2075, 1726, 1729, 1730, 1732, 2078, 2223, 1735,
1713, 41700, 1737, 14977, 1739, 1740, 1741, 2080, 1743, 1744,
1745, 1746, 1747, 1748, 1749, 1750, 1751, 41701, 1752, 1753,
1909, 86220, 2085, 1754, 19548, 86221, 19551, 5733, 3856, 5190,
4581, 25145, 86222, 86223, 4846, 86224, 4861, 86225, 86226,
86227, 25150, 86228, 86229, 13820, 2027, 4898, 4899, 4901,
2135, 4902, 4868, 4904, 86230, 4905, 25155, 4907, 86231, 4909,
4910, 4911, 4912, 86232, 6220, 81357, 86233, 2589, 73877,
29706, 6227, 6228, 86234, 6237, 86235, 6241, 6242, 1812, 13808,
13809, 70908, 2293, 2294, 86236, 2295, 2296, 2297, 22947,
16511, 2299, 2300, 2301, 13097, 73079, 86237, 13099, 50121,
86238, 86239, 13101, 86240, 2424, 4725, 4726, 4727, 4728, 4729,
4730, 86241, 26881, 10944, 4734, 4735, 4736, 26239, 26240,
71408, 86242, 57401, 71410, 26244, 5344, 26245, 86243, 4102,
71414, 11091, 6736, 86244, 6737, 6738, 38152, 6740, 6741, 6742,
6298, 6743, 6745, 6746, 20867, 6749, 20616, 86245, 9801, 65297,
20617, 65298, 20619, 5629, 65299, 20621, 20622, 8385, 20623,
20624, 5191, 20625, 20626, 442, 443, 445, 27837, 77681, 86246,
27839, 86247, 86248, 41435, 66511, 2478, 2479, 2480, 2481,
2482, 2483, 2484, 2485, 2486, 2487, 2488, 2489, 2490, 2494,
2493, 33025, 12084, 2542, 2497, 2499, 2501, 2503, 2504, 2505,
33026, 2506, 2507, 2508, 2509, 2511, 1787, 12080, 2513, 2514,
3988, 3176, 3989, 2518, 2521, 9285, 2522, 2524, 2525, 3990,
2527, 2528, 27499, 2529, 2530, 3991, 2532, 2534, 2535, 18038,
2536, 2538, 2495, 46077, 61493, 61494, 1006, 713, 4971, 4972,
4973, 4975, 4976, 650, 170, 7549, 7550, 7551, 7552, 7553,
86249, 7936, 956, 11169, 11170, 1249, 1244, 1245, 1247, 2544,
1250, 2545, 1252, 2547, 1253, 1254, 2549, 39636, 1259, 1257,
1258, 39637, 1260, 1261, 2551, 1262, 1263, 848, 86250, 86251,
854, 74596, 856, 1957, 86252, 855, 1959, 1961, 857, 86253, 851,
859, 860, 862, 1964, 864, 865, 866, 867, 1965, 1966, 1967,
1968, 1969, 86254, 1971, 1972, 1973, 1974, 1975, 1976, 1977,
841, 1954, 842, 2978, 846, 847, 849, 850, 852, 1956, 17452,
71941, 86255, 86256, 73665, 1471, 13690, 185, 503, 504, 2342,
505, 506, 4378, 508, 4379, 17313, 510, 511, 512, 520, 513,
4384, 17314, 514, 515, 46158, 17317, 518, 34269, 519, 4386,
523, 524, 525, 46159, 528, 529, 17319, 531, 532, 533, 534, 535,
7482, 537, 538, 5267, 536, 539, 541, 540, 19858, 17320, 17321,
906, 907, 908, 17322, 910, 17323, 912, 15850, 913, 4398, 17324,
86257, 278, 2948, 2949, 2950, 3007, 2951, 2952, 2953, 2954,
2955, 3013, 35352, 3014, 3015, 2962, 3016, 33505, 39118, 3017,
3018, 20492, 4000, 3021, 3022, 35353, 39293, 3024, 18443, 3029,
9467, 20529, 39119, 8380, 2965, 3030, 3043, 22714, 39120, 2956,
3035, 39121, 3037, 3038, 2688, 86258, 36675, 30894, 24505,
8888, 13541, 49728, 27660, 9082, 27661, 365, 366, 2232, 76098,
7233, 1494, 17391, 606, 607, 611, 610, 612, 614, 615, 613, 616,
9117, 617, 618, 21155, 1789, 619, 620, 7636, 12019, 621, 622,
1793, 623, 625, 624, 631, 626, 627, 21578, 21103, 628, 21579,
629, 9122, 9123, 12189, 9289, 3168, 3169, 630, 632, 634, 21580,
9121, 635, 636, 637, 21581, 12781, 1801, 638, 639, 1559, 24343,
9419, 9420, 795, 796, 1611, 86259, 1612, 21551, 21552, 3741,
1617, 3742, 1615, 1619, 1620, 6301, 3744, 1622, 67685, 8521,
55937, 9025, 27663, 8881, 13581, 86260, 11592, 44720, 86261,
63231, 50873, 42925, 52332, 86262, 72706, 17705, 17707, 17708,
3401, 40217, 1248, 40218, 86263, 7098, 86264, 86265, 1264,
86266, 1266, 1267, 1268, 1269, 86267, 1271, 1272, 1273, 1274,
2556, 1275, 1276, 1277, 1278, 1279, 1280, 1282, 1283, 22680,
11889, 86268, 45662, 7038, 86269, 19315, 45663, 45664, 86270,
5855, 34002, 49245, 10447, 5663, 86271, 15429, 53877, 49249,
86272, 86273, 86274, 60128, 60453, 60129, 5552, 31923, 43407,
4287, 17980, 64977, 86275, 86276, 8234, 86277, 3649, 8240,
1330, 11999, 1332, 27618, 1334, 1335, 340, 3651, 25640, 18165,
1343, 4618, 1474, 3653, 75921, 1349, 53519, 1779, 45454, 22778,
40153, 67677, 63826, 45455, 15128, 67678, 67679, 1792, 67680,
3171, 47816, 45457, 9288, 59891, 67681, 25703, 35731, 35732,
369, 35713, 35714, 35715, 34652, 35716, 31681, 35717, 12779,
35718, 35719, 11992, 806, 807, 808, 43499, 43500, 810, 776,
812, 813, 814, 241, 43501, 43502, 816, 755, 43503, 818, 819,
820, 43504, 821, 822, 823, 824, 825, 826, 43505, 43506, 43507,
828, 829, 20083, 43508, 43509, 832, 833, 834, 835, 86278,
19984, 19985, 86279, 24125, 19986, 86280, 19988, 86281, 5414,
86282, 85808, 5479, 5420, 5421, 5422, 5423, 63800, 86283,
86284, 30965, 86285, 416, 1510, 5740, 5741, 81991, 86286,
28938, 50149, 1003, 55512, 14306, 6960, 688, 86287, 14307,
5399, 5400, 17783, 24118, 720, 86288, 44913, 24557, 667, 24876,
6529, 24877, 24878, 24879, 24880, 31847, 20671, 4011, 171, 580,
86289, 3863, 914, 2202, 916, 917, 918, 919, 921, 922, 923,
7585, 925, 7586, 926, 927, 928, 7588, 929, 930, 931, 932, 933,
934, 1875, 1876, 7589, 7590, 1878, 1879, 7591, 7592, 1882,
1883, 1884, 2212, 7593, 1887, 1888, 1889, 1890, 1891, 1892,
1893, 1894, 1895, 1896, 1897, 1898, 2217, 1900, 7594, 1902,
2219, 7595, 1905, 1906, 1907, 3323, 7596, 1911, 1912, 7597,
1914, 1915, 1916, 2226, 1919, 7598, 2227, 1920, 1921, 7599,
7600, 4708, 1923, 355, 356, 1549, 358, 32077, 360, 32078,
21117, 362, 19043, 71677, 5716, 86290, 49790, 86291, 86292,
86293, 49792, 86294, 86295, 49794, 86296, 86297, 86298, 86299,
11882, 86300, 49798, 86301, 49800, 49801, 49802, 49803, 453,
49804, 8591, 6794, 49806, 18989, 49807, 49808, 16308, 49809,
86302, 86303, 10105, 86304, 5285, 10106, 10107, 6557, 86305,
23571, 10109, 38883, 10110, 5401, 86306, 67557, 16430, 67558,
40171, 16433, 25878, 86307, 21762, 23, 86308, 86309, 21766,
86310, 86311, 5149, 3926, 21768, 21769, 47826, 942, 46985,
6588, 58867, 6589, 6590, 86312, 6592, 6006, 53855, 9565, 359,
86313, 2845, 876, 879, 27556, 27557, 885, 27558, 888, 2847,
27559, 2115, 2116, 2117, 53962, 57839, 315, 316, 317, 318, 319,
86314, 321, 322, 2122, 323, 2123, 324, 325, 328, 326, 327,
40542, 329, 330, 18079, 18080, 331, 1790, 7382, 332, 7380,
7236, 23413, 23414, 18924, 18925, 333, 335, 336, 39750, 337,
86315, 339, 341, 342, 343, 16264, 16265, 6615, 86316, 86317,
86318, 86319, 16269, 10538, 33226, 86320, 16272, 5824, 16273,
16274, 16276, 16277, 16278, 16279, 16280, 14517, 1547, 6463,
3394, 49677, 659, 10380, 30013, 10382, 10378, 10379, 10383,
10384, 10385, 86321, 4139, 13370, 13371, 86322, 86323, 11878,
64509, 15141, 15142, 15143, 32737, 14183, 15144, 39101, 42768,
5645, 32738, 801, 803, 804, 86324, 14707, 86325, 6601, 12402,
712, 12403, 2936, 1447, 15477, 1410, 44872, 1550, 8614, 15478,
15479, 15480, 15481, 4811, 3752, 1442, 15482, 8818, 1445, 5006,
16304, 32277, 16305, 16306, 86326, 16307, 53691, 69305, 809,
86327, 815, 26724, 69307, 43484, 63904, 86328, 13498, 827,
86329, 831, 2857, 836, 86330, 86331, 837, 838, 839, 840, 228,
229, 43722, 230, 231, 43723, 234, 235, 236, 237, 238, 239,
2745, 2746, 240, 242, 243, 244, 43724, 19788, 246, 247, 21134,
248, 250, 251, 252, 253, 254, 255, 256, 257, 258, 43725, 43726,
41, 43727, 262, 43728, 2751, 264, 265, 266, 267, 268, 269, 270,
271, 272, 1024, 1025, 1026, 1027, 1028, 1029, 1030, 1031, 1032,
1033, 1034, 43729, 1035, 43730, 1037, 21821, 2926, 14388,
10432, 14389, 14390, 14391, 14392, 86332, 14394, 14395, 2035,
2169, 86333, 14397, 14398, 14399, 14400, 52, 14401, 14402,
7077, 21822, 14405, 14406, 14396, 86334, 17356, 17357, 84679,
84680, 76383, 17360, 17361, 86335, 38801, 2060, 30850, 12963,
1684, 1687, 2061, 14978, 1694, 43387, 1697, 1699, 2067, 1701,
1702, 1706, 43388, 43389, 76325, 1716, 1718, 26832, 1719, 1723,
2081, 2063, 1728, 39059, 76326, 1731, 86336, 1736, 76327, 1738,
19657, 6579, 6581, 6582, 6583, 6584, 6585, 29979, 1818, 28239,
68, 69, 3391, 86337, 10266, 63528, 86338, 10269, 10270, 10271,
10272, 86339, 86340, 63530, 63531, 63532, 63533, 10273, 63534,
86341, 10681, 10682, 86342, 9673, 86343, 10683, 460, 461, 462,
467, 4464, 4466, 3729, 471, 472, 468, 81634, 474, 81635, 475,
476, 477, 479, 480, 81636, 81637, 482, 17442, 81638, 81639,
484, 485, 486, 4473, 488, 489, 490, 493, 466, 494, 495, 496,
497, 499, 500, 501, 502, 34376, 86344, 63836, 56281, 1707,
20416, 61452, 56282, 1755, 56283, 56284, 18508, 53650, 63444,
86345, 3579, 63445, 3677, 1979, 1980, 1981, 3132, 3147, 34090,
1987, 12770, 1329, 80818, 80819, 1988, 23522, 1986, 15880,
1985, 32975, 1992, 1993, 7165, 3141, 3143, 86346, 1982, 1984,
3145, 86347, 78064, 55453, 2656, 2657, 35634, 35635, 2167,
43479,
// ensure there is a representative number for any # of int bytes
1, 1 << 8 + 1, 1 << 16 + 1, 1 << 24 + 1 };
// data = new int[]{1, 2, 3, 4};
for (int value : data) {
dataSet.add(new Long(value));
}
@Test
public void testVInt8() throws Exception {
encoderTest(new VInt8IntEncoder(), data, data);
// cover negative numbers;
BytesRef bytes = new BytesRef(5);
IntEncoder enc = new VInt8IntEncoder();
IntsRef values = new IntsRef(1);
values.ints[values.length++] = -1;
enc.encode(values, bytes);
IntDecoder dec = enc.createMatchingDecoder();
values.length = 0;
dec.decode(bytes, values);
assertEquals(1, values.length);
assertEquals(-1, values.ints[0]);
}
@Test
public void testSimpleInt() throws Exception {
encoderTest(new SimpleIntEncoder(), data, data);
}
@Test
public void testSortingUniqueValues() throws Exception {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new VInt8IntEncoder())), data, uniqueSortedData);
}
@Test
public void testSortingUniqueDGap() throws Exception {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new VInt8IntEncoder()))), data, uniqueSortedData);
}
@Test
public void testSortingUniqueDGapEightFlags() throws Exception {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new EightFlagsIntEncoder()))), data, uniqueSortedData);
}
@Test
public void testSortingUniqueDGapFourFlags() throws Exception {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new FourFlagsIntEncoder()))), data, uniqueSortedData);
}
@Test
public void testSortingUniqueDGapNOnes4() throws Exception {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(4)))), data, uniqueSortedData);
}
@Test
public void testSortingUniqueDGapNOnes3() throws Exception {
encoderTest(new SortingIntEncoder(new UniqueValuesIntEncoder(new DGapIntEncoder(new NOnesIntEncoder(3)))), data, uniqueSortedData);
}
}

View File

@ -0,0 +1,54 @@
package org.apache.lucene.util.encoding;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.Test;
/*
* 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.
*/
/**
* Tests the {@link VInt8} class.
*/
public class Vint8Test extends LuceneTestCase {
private static final int[] TEST_VALUES = {
-1000000000,
-1, 0, (1 << 7) - 1, 1 << 7, (1 << 14) - 1, 1 << 14,
(1 << 21) - 1, 1 << 21, (1 << 28) - 1, 1 << 28
};
private static int[] BYTES_NEEDED_TEST_VALUES = {
5, 5, 1, 1, 2, 2, 3, 3, 4, 4, 5
};
@Test
public void testBytesRef() throws Exception {
BytesRef bytes = new BytesRef(256);
int expectedSize = 0;
for (int j = 0; j < TEST_VALUES.length; j++) {
VInt8.encode(TEST_VALUES[j], bytes);
expectedSize += BYTES_NEEDED_TEST_VALUES[j];
}
assertEquals(expectedSize, bytes.length);
for (int j = 0; j < TEST_VALUES.length; j++) {
assertEquals(TEST_VALUES[j], VInt8.decode(bytes));
}
assertEquals(bytes.offset, bytes.length);
}
}

View File

@ -73,7 +73,7 @@ public class NativeUnixDirectory extends FSDirectory {
private final static long ALIGN = 512;
private final static long ALIGN_NOT_MASK = ~(ALIGN-1);
/** Default buffer size before writing to disk (256 MB);
/** Default buffer size before writing to disk (256 KB);
* larger means less IO load but more RAM and direct
* buffer storage space consumed during merging. */

View File

@ -237,7 +237,7 @@ public class FSTCompletionBuilder {
final Object empty = outputs.getNoOutput();
final Builder<Object> builder = new Builder<Object>(
FST.INPUT_TYPE.BYTE1, 0, 0, true, true,
shareMaxTailLength, outputs, null, false);
shareMaxTailLength, outputs, null, false, true);
BytesRef scratch = new BytesRef();
BytesRef entry;

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs.asserting;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
@ -29,6 +30,7 @@ public final class AssertingCodec extends FilterCodec {
private final PostingsFormat postings = new AssertingPostingsFormat();
private final TermVectorsFormat vectors = new AssertingTermVectorsFormat();
private final StoredFieldsFormat storedFields = new AssertingStoredFieldsFormat();
public AssertingCodec() {
super("Asserting", new Lucene41Codec());
@ -44,4 +46,8 @@ public final class AssertingCodec extends FilterCodec {
return vectors;
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return storedFields;
}
}

View File

@ -0,0 +1,136 @@
package org.apache.lucene.codecs.asserting;
/*
* 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 org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.StoredFieldVisitor;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
/**
* Just like {@link Lucene41StoredFieldsFormat} but with additional asserts.
*/
public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
private final StoredFieldsFormat in = new Lucene41StoredFieldsFormat();
@Override
public StoredFieldsReader fieldsReader(Directory directory, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
return new AssertingStoredFieldsReader(in.fieldsReader(directory, si, fn, context), si.getDocCount());
}
@Override
public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOContext context) throws IOException {
return new AssertingStoredFieldsWriter(in.fieldsWriter(directory, si, context));
}
static class AssertingStoredFieldsReader extends StoredFieldsReader {
private final StoredFieldsReader in;
private final int maxDoc;
AssertingStoredFieldsReader(StoredFieldsReader in, int maxDoc) {
this.in = in;
this.maxDoc = maxDoc;
}
@Override
public void close() throws IOException {
in.close();
}
@Override
public void visitDocument(int n, StoredFieldVisitor visitor) throws IOException {
assert n >= 0 && n < maxDoc;
in.visitDocument(n, visitor);
}
@Override
public StoredFieldsReader clone() {
return new AssertingStoredFieldsReader(in.clone(), maxDoc);
}
}
enum Status {
UNDEFINED, STARTED, FINISHED;
}
static class AssertingStoredFieldsWriter extends StoredFieldsWriter {
private final StoredFieldsWriter in;
private int numWritten;
private int fieldCount;
private Status docStatus;
AssertingStoredFieldsWriter(StoredFieldsWriter in) {
this.in = in;
this.docStatus = Status.UNDEFINED;
}
@Override
public void startDocument(int numStoredFields) throws IOException {
assert docStatus != Status.STARTED;
in.startDocument(numStoredFields);
assert fieldCount == 0;
fieldCount = numStoredFields;
numWritten++;
docStatus = Status.STARTED;
}
@Override
public void finishDocument() throws IOException {
assert docStatus == Status.STARTED;
assert fieldCount == 0;
in.finishDocument();
docStatus = Status.FINISHED;
}
@Override
public void writeField(FieldInfo info, StorableField field) throws IOException {
assert docStatus == Status.STARTED;
in.writeField(info, field);
assert fieldCount > 0;
fieldCount--;
}
@Override
public void abort() {
in.abort();
}
@Override
public void finish(FieldInfos fis, int numDocs) throws IOException {
assert docStatus == (numDocs > 0 ? Status.FINISHED : Status.UNDEFINED);
in.finish(fis, numDocs);
assert fieldCount == 0;
assert numDocs == numWritten;
}
@Override
public void close() throws IOException {
in.close();
assert docStatus != Status.STARTED;
}
}
}

View File

@ -18,17 +18,20 @@ package org.apache.lucene.codecs.asserting;
*/
import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
import org.apache.lucene.index.AssertingAtomicReader;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
/**
* Just like {@link Lucene40TermVectorsFormat} but with additional asserts.
@ -43,16 +46,16 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
@Override
public TermVectorsWriter vectorsWriter(Directory directory, SegmentInfo segmentInfo, IOContext context) throws IOException {
return in.vectorsWriter(directory, segmentInfo, context);
return new AssertingTermVectorsWriter(in.vectorsWriter(directory, segmentInfo, context));
}
static class AssertingTermVectorsReader extends TermVectorsReader {
private final TermVectorsReader in;
AssertingTermVectorsReader(TermVectorsReader in) {
this.in = in;
}
@Override
public void close() throws IOException {
in.close();
@ -68,5 +71,120 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
public TermVectorsReader clone() {
return new AssertingTermVectorsReader(in.clone());
}
}
}
enum Status {
UNDEFINED, STARTED, FINISHED;
}
static class AssertingTermVectorsWriter extends TermVectorsWriter {
private final TermVectorsWriter in;
private Status docStatus, fieldStatus, termStatus;
private int fieldCount, termCount, positionCount;
boolean hasPositions;
AssertingTermVectorsWriter(TermVectorsWriter in) {
this.in = in;
docStatus = Status.UNDEFINED;
fieldStatus = Status.UNDEFINED;
termStatus = Status.UNDEFINED;
fieldCount = termCount = positionCount = 0;
}
@Override
public void startDocument(int numVectorFields) throws IOException {
assert fieldCount == 0;
assert docStatus != Status.STARTED;
in.startDocument(numVectorFields);
docStatus = Status.STARTED;
fieldCount = numVectorFields;
}
@Override
public void finishDocument() throws IOException {
assert fieldCount == 0;
assert docStatus == Status.STARTED;
in.finishDocument();
docStatus = Status.FINISHED;
}
@Override
public void startField(FieldInfo info, int numTerms, boolean positions,
boolean offsets, boolean payloads) throws IOException {
assert termCount == 0;
assert docStatus == Status.STARTED;
assert fieldStatus != Status.STARTED;
in.startField(info, numTerms, positions, offsets, payloads);
fieldStatus = Status.STARTED;
termCount = numTerms;
hasPositions = positions || offsets || payloads;
}
@Override
public void finishField() throws IOException {
assert termCount == 0;
assert fieldStatus == Status.STARTED;
in.finishField();
fieldStatus = Status.FINISHED;
--fieldCount;
}
@Override
public void startTerm(BytesRef term, int freq) throws IOException {
assert docStatus == Status.STARTED;
assert fieldStatus == Status.STARTED;
assert termStatus != Status.STARTED;
in.startTerm(term, freq);
termStatus = Status.STARTED;
positionCount = hasPositions ? freq : 0;
}
@Override
public void finishTerm() throws IOException {
assert positionCount == 0;
assert docStatus == Status.STARTED;
assert fieldStatus == Status.STARTED;
assert termStatus == Status.STARTED;
in.finishTerm();
termStatus = Status.FINISHED;
--termCount;
}
@Override
public void addPosition(int position, int startOffset, int endOffset,
BytesRef payload) throws IOException {
assert docStatus == Status.STARTED;
assert fieldStatus == Status.STARTED;
assert termStatus == Status.STARTED;
in.addPosition(position, startOffset, endOffset, payload);
--positionCount;
}
@Override
public void abort() {
in.abort();
}
@Override
public void finish(FieldInfos fis, int numDocs) throws IOException {
assert docStatus == (numDocs > 0 ? Status.FINISHED : Status.UNDEFINED);
assert fieldStatus != Status.STARTED;
assert termStatus != Status.STARTED;
in.finish(fis, numDocs);
}
@Override
public Comparator<BytesRef> getComparator() throws IOException {
return in.getComparator();
}
@Override
public void close() throws IOException {
in.close();
assert docStatus != Status.STARTED;
assert fieldStatus != Status.STARTED;
assert termStatus != Status.STARTED;
}
}
}

Some files were not shown because too many files have changed in this diff Show More