cutover to Iterable for SortedDV

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4547@1430090 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael McCandless 2013-01-07 23:41:09 +00:00
parent f0ecefa78b
commit 9bfe0af20b
15 changed files with 353 additions and 939 deletions

View File

@ -123,8 +123,6 @@ public class MemoryDocValuesFormat extends SimpleDocValuesFormat {
public SortedDocValues getSorted(FieldInfo field) throws IOException {
SortedDocValues valuesIn = producer.getSorted(field);
final int maxDoc = valuesIn.size();
final int maxLength = valuesIn.maxLength();
final boolean fixedLength = valuesIn.isFixedLength();
final int valueCount = valuesIn.getValueCount();
// nocommit used packed ints and so on
@ -163,16 +161,6 @@ public class MemoryDocValuesFormat extends SimpleDocValuesFormat {
public int size() {
return maxDoc;
}
@Override
public boolean isFixedLength() {
return fixedLength;
}
@Override
public int maxLength() {
return maxLength;
}
};
}

View File

@ -32,7 +32,6 @@ import java.util.Set;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.codecs.SimpleDVProducer;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
import org.apache.lucene.codecs.SortedDocValuesConsumer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues;
@ -268,21 +267,24 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
}
@Override
public SortedDocValuesConsumer addSortedField(FieldInfo field, final int valueCount, boolean fixedLength, final int maxLength) throws IOException {
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
assert fieldSeen(field.name);
assert DocValues.isSortedBytes(field.getDocValuesType());
assert !isNorms;
writeFieldEntry(field);
int valueCount = 0;
int maxLength = -1;
for(BytesRef value : values) {
maxLength = Math.max(maxLength, value.length);
valueCount++;
}
// write numValues
SimpleTextUtil.write(data, NUMVALUES);
SimpleTextUtil.write(data, Integer.toString(valueCount), scratch);
SimpleTextUtil.writeNewline(data);
// write fixedlength
SimpleTextUtil.write(data, FIXEDLENGTH);
SimpleTextUtil.write(data, Boolean.toString(fixedLength), scratch);
SimpleTextUtil.writeNewline(data);
// write maxLength
SimpleTextUtil.write(data, MAXLENGTH);
SimpleTextUtil.write(data, Integer.toString(maxLength), scratch);
@ -312,40 +314,34 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
SimpleTextUtil.writeNewline(data);
final DecimalFormat ordEncoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
return new SortedDocValuesConsumer() {
// for asserts:
int valuesSeen = 0;
// for asserts:
private int valuesSeen;
@Override
public void addValue(BytesRef value) throws IOException {
// write length
SimpleTextUtil.write(data, LENGTH);
SimpleTextUtil.write(data, encoder.format(value.length), scratch);
SimpleTextUtil.writeNewline(data);
for(BytesRef value : values) {
// write length
SimpleTextUtil.write(data, LENGTH);
SimpleTextUtil.write(data, encoder.format(value.length), scratch);
SimpleTextUtil.writeNewline(data);
// write bytes -- don't use SimpleText.write
// because it escapes:
data.writeBytes(value.bytes, value.offset, value.length);
// write bytes -- don't use SimpleText.write
// because it escapes:
data.writeBytes(value.bytes, value.offset, value.length);
// pad to fit
for (int i = value.length; i < maxLength; i++) {
data.writeByte((byte)' ');
}
SimpleTextUtil.writeNewline(data);
valuesSeen++;
assert valuesSeen <= valueCount;
// pad to fit
for (int i = value.length; i < maxLength; i++) {
data.writeByte((byte)' ');
}
SimpleTextUtil.writeNewline(data);
valuesSeen++;
assert valuesSeen <= valueCount;
}
@Override
public void addDoc(int ord) throws IOException {
SimpleTextUtil.write(data, ordEncoder.format(ord), scratch);
SimpleTextUtil.writeNewline(data);
}
assert valuesSeen == valueCount;
@Override
public void finish() throws IOException {}
};
for(Number ord : docToOrd) {
SimpleTextUtil.write(data, ordEncoder.format(ord.intValue()), scratch);
SimpleTextUtil.writeNewline(data);
}
}
/** write the header for this field */
@ -450,9 +446,6 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
assert startsWith(NUMVALUES);
field.numValues = Integer.parseInt(stripPrefix(NUMVALUES));
readLine();
assert startsWith(FIXEDLENGTH);
field.fixedLength = Boolean.parseBoolean(stripPrefix(FIXEDLENGTH));
readLine();
assert startsWith(MAXLENGTH);
field.maxLength = Integer.parseInt(stripPrefix(MAXLENGTH));
readLine();
@ -635,16 +628,6 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
public int size() {
return maxDoc;
}
@Override
public boolean isFixedLength() {
return field.fixedLength;
}
@Override
public int maxLength() {
return field.maxLength;
}
};
}

View File

@ -19,6 +19,7 @@ package org.apache.lucene.codecs;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
@ -31,6 +32,8 @@ import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.PriorityQueue;
// prototype streaming DV api
public abstract class SimpleDVConsumer implements Closeable {
@ -42,8 +45,7 @@ public abstract class SimpleDVConsumer implements Closeable {
public abstract void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException;
// nocommit: figure out whats fair here.
public abstract SortedDocValuesConsumer addSortedField(FieldInfo field, int valueCount, boolean fixedLength, int maxLength) throws IOException;
public abstract void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException;
// dead simple impl: codec can optimize
public void mergeNumericField(FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge) throws IOException {
@ -183,10 +185,238 @@ public abstract class SimpleDVConsumer implements Closeable {
});
}
public void mergeSortedField(FieldInfo fieldInfo, MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
SortedDocValuesConsumer.Merger merger = new SortedDocValuesConsumer.Merger();
public static class SortedBytesMerger {
public int numMergedTerms;
final List<BytesRef> mergedTerms = new ArrayList<BytesRef>();
final List<SegmentState> segStates = new ArrayList<SegmentState>();
private static class SegmentState {
AtomicReader reader;
FixedBitSet liveTerms;
int ord = -1;
SortedDocValues values;
BytesRef scratch = new BytesRef();
// nocommit can we factor out the compressed fields
// compression? ie we have a good idea "roughly" what
// the ord should be (linear projection) so we only
// need to encode the delta from that ...:
int[] segOrdToMergedOrd;
public BytesRef nextTerm() {
while (ord < values.getValueCount()-1) {
ord++;
if (liveTerms == null || liveTerms.get(ord)) {
values.lookupOrd(ord, scratch);
return scratch;
} else {
// Skip "deleted" terms (ie, terms that were not
// referenced by any live docs):
values.lookupOrd(ord, scratch);
}
}
return null;
}
}
private static class TermMergeQueue extends PriorityQueue<SegmentState> {
public TermMergeQueue(int maxSize) {
super(maxSize);
}
@Override
protected boolean lessThan(SegmentState a, SegmentState b) {
return a.scratch.compareTo(b.scratch) <= 0;
}
}
public void merge(MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
// First pass: mark "live" terms
for (int readerIDX=0;readerIDX<toMerge.size();readerIDX++) {
AtomicReader reader = mergeState.readers.get(readerIDX);
// nocommit what if this is null...? need default source?
int maxDoc = reader.maxDoc();
SegmentState state = new SegmentState();
state.reader = reader;
state.values = toMerge.get(readerIDX);
segStates.add(state);
assert state.values.getValueCount() < Integer.MAX_VALUE;
if (reader.hasDeletions()) {
state.liveTerms = new FixedBitSet(state.values.getValueCount());
Bits liveDocs = reader.getLiveDocs();
for(int docID=0;docID<maxDoc;docID++) {
if (liveDocs.get(docID)) {
state.liveTerms.set(state.values.getOrd(docID));
}
}
}
// nocommit we can unload the bits to disk to reduce
// transient ram spike...
}
// Second pass: merge only the live terms
TermMergeQueue q = new TermMergeQueue(segStates.size());
for(SegmentState segState : segStates) {
if (segState.nextTerm() != null) {
// nocommit we could defer this to 3rd pass (and
// reduce transient RAM spike) but then
// we'd spend more effort computing the mapping...:
segState.segOrdToMergedOrd = new int[segState.values.getValueCount()];
q.add(segState);
}
}
BytesRef lastTerm = null;
int ord = 0;
while (q.size() != 0) {
SegmentState top = q.top();
if (lastTerm == null || !lastTerm.equals(top.scratch)) {
lastTerm = BytesRef.deepCopyOf(top.scratch);
// nocommit we could spill this to disk instead of
// RAM, and replay on finish...
mergedTerms.add(lastTerm);
ord++;
}
top.segOrdToMergedOrd[top.ord] = ord-1;
if (top.nextTerm() == null) {
q.pop();
} else {
q.updateTop();
}
}
numMergedTerms = ord;
}
/*
public void finish(SortedDocValuesConsumer consumer) throws IOException {
// Third pass: write merged result
for(BytesRef term : mergedTerms) {
consumer.addValue(term);
}
for(SegmentState segState : segStates) {
Bits liveDocs = segState.reader.getLiveDocs();
int maxDoc = segState.reader.maxDoc();
for(int docID=0;docID<maxDoc;docID++) {
if (liveDocs == null || liveDocs.get(docID)) {
int segOrd = segState.values.getOrd(docID);
int mergedOrd = segState.segOrdToMergedOrd[segOrd];
consumer.addDoc(mergedOrd);
}
}
}
}
*/
}
public void mergeSortedField(FieldInfo fieldInfo, final MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
final SortedBytesMerger merger = new SortedBytesMerger();
// Does the heavy lifting to merge sort all "live" ords:
merger.merge(mergeState, toMerge);
SortedDocValuesConsumer consumer = addSortedField(fieldInfo, merger.numMergedTerms, merger.fixedLength >= 0, merger.maxLength);
consumer.merge(mergeState, merger);
addSortedField(fieldInfo,
// ord -> value
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new Iterator<BytesRef>() {
int ordUpto;
@Override
public boolean hasNext() {
return ordUpto < merger.mergedTerms.size();
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() {
return merger.mergedTerms.get(ordUpto++);
}
};
}
},
// doc -> ord
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
int nextValue;
SortedBytesMerger.SegmentState currentReader;
Bits currentLiveDocs;
boolean nextIsSet;
@Override
public boolean hasNext() {
return nextIsSet || setNext();
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public Number next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
assert nextIsSet;
nextIsSet = false;
// nocommit make a mutable number
return nextValue;
}
private boolean setNext() {
while (true) {
if (readerUpto == merger.segStates.size()) {
return false;
}
if (currentReader == null || docIDUpto == currentReader.reader.maxDoc()) {
readerUpto++;
if (readerUpto < merger.segStates.size()) {
currentReader = merger.segStates.get(readerUpto);
currentLiveDocs = currentReader.reader.getLiveDocs();
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
int segOrd = currentReader.values.getOrd(docIDUpto);
nextValue = currentReader.segOrdToMergedOrd[segOrd];
docIDUpto++;
return true;
}
docIDUpto++;
}
}
};
}
});
}
}

View File

@ -1,196 +0,0 @@
package org.apache.lucene.codecs;
/*
* 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.ArrayList;
import java.util.List;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.PriorityQueue;
//nocommit - this needs an abort() method? to free opened files?
public abstract class SortedDocValuesConsumer {
/** This is called, in value sort order, once per unique
* value. */
public abstract void addValue(BytesRef value) throws IOException;
/** This is called once per document after all values are
* added. */
public abstract void addDoc(int ord) throws IOException;
public abstract void finish() throws IOException;
public static class Merger {
public int fixedLength = -2;
public int maxLength;
public int numMergedTerms;
private final List<BytesRef> mergedTerms = new ArrayList<BytesRef>();
private final List<SegmentState> segStates = new ArrayList<SegmentState>();
private static class SegmentState {
AtomicReader reader;
FixedBitSet liveTerms;
int ord = -1;
SortedDocValues values;
BytesRef scratch = new BytesRef();
// nocommit can we factor out the compressed fields
// compression? ie we have a good idea "roughly" what
// the ord should be (linear projection) so we only
// need to encode the delta from that ...:
int[] segOrdToMergedOrd;
public BytesRef nextTerm() {
while (ord < values.getValueCount()-1) {
ord++;
if (liveTerms == null || liveTerms.get(ord)) {
values.lookupOrd(ord, scratch);
return scratch;
} else {
// Skip "deleted" terms (ie, terms that were not
// referenced by any live docs):
values.lookupOrd(ord, scratch);
}
}
return null;
}
}
private static class TermMergeQueue extends PriorityQueue<SegmentState> {
public TermMergeQueue(int maxSize) {
super(maxSize);
}
@Override
protected boolean lessThan(SegmentState a, SegmentState b) {
return a.scratch.compareTo(b.scratch) <= 0;
}
}
public void merge(MergeState mergeState, List<SortedDocValues> toMerge) throws IOException {
// First pass: mark "live" terms
for (int readerIDX=0;readerIDX<toMerge.size();readerIDX++) {
AtomicReader reader = mergeState.readers.get(readerIDX);
// nocommit what if this is null...? need default source?
int maxDoc = reader.maxDoc();
SegmentState state = new SegmentState();
state.reader = reader;
state.values = toMerge.get(readerIDX);
segStates.add(state);
assert state.values.getValueCount() < Integer.MAX_VALUE;
if (reader.hasDeletions()) {
state.liveTerms = new FixedBitSet(state.values.getValueCount());
Bits liveDocs = reader.getLiveDocs();
for(int docID=0;docID<maxDoc;docID++) {
if (liveDocs.get(docID)) {
state.liveTerms.set(state.values.getOrd(docID));
}
}
}
// nocommit we can unload the bits to disk to reduce
// transient ram spike...
}
// Second pass: merge only the live terms
TermMergeQueue q = new TermMergeQueue(segStates.size());
for(SegmentState segState : segStates) {
if (segState.nextTerm() != null) {
// nocommit we could defer this to 3rd pass (and
// reduce transient RAM spike) but then
// we'd spend more effort computing the mapping...:
segState.segOrdToMergedOrd = new int[segState.values.getValueCount()];
q.add(segState);
}
}
BytesRef lastTerm = null;
int ord = 0;
while (q.size() != 0) {
SegmentState top = q.top();
if (lastTerm == null || !lastTerm.equals(top.scratch)) {
lastTerm = BytesRef.deepCopyOf(top.scratch);
// nocommit we could spill this to disk instead of
// RAM, and replay on finish...
mergedTerms.add(lastTerm);
ord++;
if (fixedLength == -2) {
fixedLength = lastTerm.length;
} else {
if (lastTerm.length != fixedLength) {
fixedLength = -1;
}
}
maxLength = Math.max(maxLength, lastTerm.length);
}
top.segOrdToMergedOrd[top.ord] = ord-1;
if (top.nextTerm() == null) {
q.pop();
} else {
q.updateTop();
}
}
numMergedTerms = ord;
}
public void finish(SortedDocValuesConsumer consumer) throws IOException {
// Third pass: write merged result
for(BytesRef term : mergedTerms) {
consumer.addValue(term);
}
for(SegmentState segState : segStates) {
Bits liveDocs = segState.reader.getLiveDocs();
int maxDoc = segState.reader.maxDoc();
for(int docID=0;docID<maxDoc;docID++) {
if (liveDocs == null || liveDocs.get(docID)) {
int segOrd = segState.values.getOrd(docID);
int mergedOrd = segState.segOrdToMergedOrd[segOrd];
consumer.addDoc(mergedOrd);
}
}
}
}
}
// nocommit why return int...?
public void merge(MergeState mergeState, Merger merger) throws IOException {
merger.finish(this);
this.finish();
}
}

View File

@ -25,7 +25,6 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.codecs.SimpleDVProducer;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
import org.apache.lucene.codecs.SortedDocValuesConsumer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
@ -128,8 +127,8 @@ public class Lucene41SimpleDocValuesFormat extends SimpleDocValuesFormat {
}
@Override
public SortedDocValuesConsumer addSortedField(FieldInfo field, int valueCount, boolean fixedLength, int maxLength) throws IOException {
return null;
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
// nocommit todo
}
@Override

View File

@ -1,129 +0,0 @@
package org.apache.lucene.codecs.lucene41.values;
/*
* 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.SimpleDVConsumer;
import org.apache.lucene.codecs.SortedDocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
public class Lucene41DocValuesConsumer extends SimpleDVConsumer {
/**
* Filename extension for index files
*/
static final String INDEX_EXTENSION = "idx";
static final String OFFSET_EXTENSION = "off";
/**
* Filename extension for data files.
*/
static final String DATA_EXTENSION = "dat";
static final String DV_SEGMENT_SUFFIX = "sdv"; // nocommit change to dv
private final SegmentInfo info;
private final Directory dir;
private Directory cfs;
private final IOContext context;
private final String segmentSuffix;
Lucene41DocValuesConsumer(SegmentWriteState state) throws IOException {
this.dir = state.directory;
this.info = state.segmentInfo;
this.context = state.context;
this.segmentSuffix = state.segmentSuffix;
}
private synchronized Directory getDirectory() throws IOException {
if (cfs == null) {
final String suffix;
if (segmentSuffix.length() == 0) {
suffix = Lucene41DocValuesConsumer.DV_SEGMENT_SUFFIX;
} else {
suffix = segmentSuffix + "_" + Lucene41DocValuesConsumer.DV_SEGMENT_SUFFIX;
}
String fileName = IndexFileNames.segmentFileName(info.name,
suffix,
IndexFileNames.COMPOUND_FILE_EXTENSION);
cfs = new CompoundFileDirectory(dir, fileName, context, true);
}
return cfs;
}
@Override
public void close() throws IOException {
IOUtils.close(cfs);
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
// ncommit
}
@Override
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) {
// ncommit
}
// nocommit: bogus to put segmentName in here. think about copySegmentAsIs!!!!!!
static String getDocValuesFileName(SegmentInfo info, FieldInfo field, String extension) {
return IndexFileNames.segmentFileName(info.name + "_"
+ field.number, DV_SEGMENT_SUFFIX, extension);
}
@Override
public SortedDocValuesConsumer addSortedField(FieldInfo field,
int valueCount, boolean fixedLength, int maxLength)
throws IOException {
String nameData = getDocValuesFileName(info, field, DATA_EXTENSION);
String idxOut = getDocValuesFileName(info, field, INDEX_EXTENSION);
String offOut = getDocValuesFileName(info, field, OFFSET_EXTENSION);
boolean success = false;
IndexOutput dataOut = null;
IndexOutput indexOut = null;
IndexOutput offsetOut = null;
try {
dataOut = getDirectory().createOutput(nameData, context);
indexOut = getDirectory().createOutput(idxOut, context);
if (fixedLength) {
offsetOut = null;
} else {
offsetOut = getDirectory().createOutput(offOut, context);
}
Lucene41SortedDocValuesConsumer consumer = new Lucene41SortedDocValuesConsumer(
dataOut, indexOut, offsetOut, valueCount, maxLength, this.info.getDocCount());
success = true;
return consumer;
} finally {
if (!success) {
IOUtils.close(dataOut, indexOut);
}
}
}
}

View File

@ -1,46 +0,0 @@
package org.apache.lucene.codecs.lucene41.values;
import java.io.IOException;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.codecs.SimpleDVProducer;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/*
* 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 Lucene41DocValuesFormat extends SimpleDocValuesFormat {
public Lucene41DocValuesFormat() {
super("Lucene41");
}
@Override
public SimpleDVConsumer fieldsConsumer(SegmentWriteState state)
throws IOException {
return new Lucene41DocValuesConsumer(state);
}
@Override
public SimpleDVProducer fieldsProducer(SegmentReadState state)
throws IOException {
return new Lucene41DocValuesProducer(state);
}
}

View File

@ -1,95 +0,0 @@
package org.apache.lucene.codecs.lucene41.values;
/*
* 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.Closeable;
import java.io.IOException;
import org.apache.lucene.codecs.SimpleDVProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
// nocommit
public class Lucene41DocValuesProducer extends SimpleDVProducer {
private final CompoundFileDirectory cfs;
// nocommit: remove this
private final SegmentInfo info;
private final IOContext context;
public Lucene41DocValuesProducer(SegmentReadState state) throws IOException {
final String suffix;
if (state.segmentSuffix.length() == 0) {
suffix = Lucene41DocValuesConsumer.DV_SEGMENT_SUFFIX;
} else {
suffix = state.segmentSuffix + "_" + Lucene41DocValuesConsumer.DV_SEGMENT_SUFFIX;
}
String cfsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, suffix,
IndexFileNames.COMPOUND_FILE_EXTENSION);
this.cfs = new CompoundFileDirectory(state.directory, cfsFileName, state.context, false);
this.info = state.segmentInfo;
this.context = state.context;
}
@Override
public void close() throws IOException {
IOUtils.close(cfs);
}
@Override
public SimpleDVProducer clone() {
return this; // nocommit ? actually safe since we open new each time from cfs?
}
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
// nocommit
return null;
}
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
// nocommit
return null;
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
if (DocValues.isSortedBytes(field.getDocValuesType())) {
return new Lucene41SortedDocValues.Factory(this.cfs, this.info, field, context).getDirect();
} else {
return null;
}
}
public static abstract class DocValuesFactory<T> implements Closeable {
public abstract T getDirect() throws IOException;
public abstract T getInMemory() throws IOException;
}
}

View File

@ -1,202 +0,0 @@
package org.apache.lucene.codecs.lucene41.values;
/*
* 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.CodecUtil;
import org.apache.lucene.codecs.lucene41.values.Lucene41DocValuesProducer.DocValuesFactory;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.codecs.lucene41.values.Lucene41SortedDocValuesConsumer.*;
public class Lucene41SortedDocValues extends SortedDocValues {
private final PackedInts.Reader docToOrdIndex;
private final PackedInts.Reader ordToOffsetIndex;
private final IndexInput data;
private final long baseOffset;
private final int valueCount;
private int size;
private int maxLength;
private final DocValuesFactory<SortedDocValues> factory;
public Lucene41SortedDocValues(IndexInput dataIn, long dataOffset, int size,
int maxLength, int valueCount, PackedInts.Reader index, PackedInts.Reader offsets, DocValuesFactory<SortedDocValues> factory)
throws IOException {
this.data = dataIn;
this.size = size;
this.maxLength = maxLength;
this.baseOffset = dataOffset;
this.valueCount = valueCount;
this.docToOrdIndex = index;
ordToOffsetIndex = offsets;
this.factory = factory;
}
@Override
public int getOrd(int docID) {
return (int) docToOrdIndex.get(docID);
}
@Override
public void lookupOrd(int ord, BytesRef result) {
try {
assert ord < valueCount;
final long offset;
final int length;
if (ordToOffsetIndex != null) {
offset = ordToOffsetIndex.get(ord);
// 1+ord is safe because we write a sentinel at the end
final long nextOffset = ordToOffsetIndex.get(1 + ord);
assert offset <= nextOffset : "offset: " + offset + " nextOffset: " + nextOffset + " ord: " + ord + " numValues: " + valueCount;
length = (int) (nextOffset - offset);
} else {
length = size;
offset = size * ord;
}
data.seek(baseOffset + offset);
result.offset = 0;
result.grow(length);
data.readBytes(result.bytes, 0, length);
result.length = length;
} catch (IOException ex) {
throw new IllegalStateException("failed", ex);
}
}
@Override
public int getValueCount() {
return valueCount;
}
@Override
public int size() {
return size;
}
@Override
public boolean isFixedLength() {
return ordToOffsetIndex == null;
}
@Override
public int maxLength() {
return maxLength;
}
public static final class Factory extends
DocValuesFactory<SortedDocValues> {
private final IndexInput datIn;
private final IndexInput offsetIn;
private final PackedInts.Header offsetHeader;
private final IndexInput indexIn;
private final PackedInts.Header indexHeader;
private final int size;
private final int maxLength;
private final long baseOffset;
private final int valueCount;
public Factory(Directory dir,
SegmentInfo segmentInfo, FieldInfo field, IOContext context)
throws IOException {
boolean success = false;
IndexInput datIn = null;
IndexInput indexIn = null;
IndexInput offsetIn = null;
try {
datIn = dir.openInput(Lucene41DocValuesConsumer.getDocValuesFileName(
segmentInfo, field, Lucene41DocValuesConsumer.DATA_EXTENSION),
context);
CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
indexIn = dir.openInput(Lucene41DocValuesConsumer.getDocValuesFileName(
segmentInfo, field, Lucene41DocValuesConsumer.INDEX_EXTENSION),
context);
CodecUtil
.checkHeader(indexIn, CODEC_NAME, VERSION_START, VERSION_START);
indexHeader = PackedInts.readHeader(indexIn);
this.size = datIn.readInt();
this.maxLength = datIn.readInt();
this.valueCount = datIn.readInt();
this.baseOffset = datIn.getFilePointer();
//if (size == Lucene41BinaryDocValuesConsumer.VALUE_SIZE_VAR) {
if (size == -1) {
offsetIn = dir.openInput(Lucene41DocValuesConsumer
.getDocValuesFileName(segmentInfo, field,
Lucene41DocValuesConsumer.OFFSET_EXTENSION), context);
CodecUtil.checkHeader(offsetIn, CODEC_NAME, VERSION_START,
VERSION_START);
this.offsetHeader = PackedInts.readHeader(offsetIn);
} else {
offsetIn = null;
this.offsetHeader = null;
}
this.offsetIn = offsetIn;
this.indexIn = indexIn;
this.datIn = datIn;
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(datIn, indexIn, offsetIn);
}
}
}
public SortedDocValues getDirect() throws IOException {
return new Lucene41SortedDocValues(datIn.clone(), this.baseOffset, size,
maxLength, valueCount, PackedInts.getDirectReaderNoHeader(indexIn.clone(),
indexHeader), offsetHeader == null ? null
: PackedInts.getDirectReaderNoHeader(offsetIn.clone(),
offsetHeader), this);
}
public Lucene41SortedDocValues getInMemory() throws IOException {
// nocommit simple in memory impl
PackedInts.Reader offsetReader = offsetHeader == null ? null : PackedInts
.getReaderNoHeader(offsetIn.clone(), offsetHeader);
PackedInts.Reader indexReader = PackedInts.getReaderNoHeader(
indexIn.clone(), indexHeader);
PagedBytes bytes = new PagedBytes(15);
bytes.copy(
datIn.clone(),
offsetReader == null ? size * indexReader.size() : offsetReader
.get(offsetReader.size() - 1));
bytes.freeze(true);
return new Lucene41SortedDocValues(bytes.getDataInput(), 0, size,
maxLength, valueCount, indexReader, offsetReader, this);
}
@Override
public void close() throws IOException {
IOUtils.close(datIn, indexIn, offsetIn);
}
}
}

View File

@ -1,104 +0,0 @@
package org.apache.lucene.codecs.lucene41.values;
/*
* 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.CodecUtil;
import org.apache.lucene.codecs.SortedDocValuesConsumer;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.AppendingLongBuffer;
import org.apache.lucene.util.packed.AppendingLongBuffer.Iterator;
import org.apache.lucene.util.packed.PackedInts;
public class Lucene41SortedDocValuesConsumer extends SortedDocValuesConsumer {
static final int VERSION_START = -1;
static final String CODEC_NAME = "Lucene41Sorted";
static final int VALUE_SIZE_VAR = -1;
private final IndexOutput data;
private final IndexOutput index;
private final IndexOutput offsets;
private final AppendingLongBuffer offsetBuffer;
private final PackedInts.Writer ords;
private long offset = 0;
private final int valueCount;
public Lucene41SortedDocValuesConsumer(IndexOutput dataOut,
IndexOutput indexOut, IndexOutput offsetOut, int valueCount, int maxLength, int docCount)
throws IOException {
int size;
if (offsetOut != null) {
size = VALUE_SIZE_VAR;
offsetBuffer = new AppendingLongBuffer();
this.offsets = offsetOut;
CodecUtil.writeHeader(offsetOut, CODEC_NAME, VERSION_START);
} else {
size = maxLength;
offsetBuffer = null;
this.offsets = null;
}
CodecUtil.writeHeader(dataOut, CODEC_NAME, VERSION_START);
dataOut.writeInt(size);
dataOut.writeInt(maxLength);
dataOut.writeInt(valueCount);
CodecUtil.writeHeader(indexOut, CODEC_NAME, VERSION_START);
this.data = dataOut;
this.index = indexOut;
this.valueCount = valueCount;
ords = PackedInts.getWriter(index, docCount,
PackedInts.bitsRequired(valueCount-1), PackedInts.DEFAULT);
}
@Override
public void addValue(BytesRef value) throws IOException {
data.writeBytes(value.bytes, value.offset, value.length);
if (offsetBuffer != null) {
offsetBuffer.add(offset);
offset += value.length;
}
}
@Override
public void addDoc(int ord) throws IOException {
ords.add(ord);
}
@Override
public void finish() throws IOException {
try {
ords.finish();
if (offsetBuffer != null) {
final int bitsRequired = PackedInts.bitsRequired(offset);
Iterator iterator = offsetBuffer.iterator();
PackedInts.Writer writer = PackedInts.getWriter(offsets, valueCount+1,
bitsRequired, PackedInts.DEFAULT);
while (iterator.hasNext()) {
writer.add(iterator.next());
}
writer.add(offset); // total # bytes
writer.finish();
}
} finally {
IOUtils.close(data, index, offsets);
}
}
}

View File

@ -29,7 +29,6 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.codecs.SimpleDVProducer;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
import org.apache.lucene.codecs.SortedDocValuesConsumer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
@ -111,8 +110,8 @@ public abstract class PerFieldDocValuesFormat extends SimpleDocValuesFormat {
}
@Override
public SortedDocValuesConsumer addSortedField(FieldInfo field, int valueCount, boolean fixedLength, int maxLength) throws IOException {
return getInstance(field).addSortedField(field, valueCount, fixedLength, maxLength);
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
getInstance(field).addSortedField(field, values, docToOrd);
}
private SimpleDVConsumer getInstance(FieldInfo field) throws IOException {

View File

@ -18,9 +18,9 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.Iterator;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.codecs.SortedDocValuesConsumer;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
@ -44,11 +44,6 @@ class SortedBytesDVWriter extends DocValuesWriter {
private static final BytesRef EMPTY = new BytesRef(BytesRef.EMPTY_BYTES);
private static final int DEFAULT_PENDING_SIZE = 16;
// -2 means not set yet; -1 means length isn't fixed;
// -otherwise it's the fixed length seen so far:
int fixedLength = -2;
int maxLength;
public SortedBytesDVWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
this.fieldInfo = fieldInfo;
this.iwBytesUsed = iwBytesUsed;
@ -81,12 +76,10 @@ class SortedBytesDVWriter extends DocValuesWriter {
public void finish(int maxDoc) {
if (pendingIndex < maxDoc) {
addOneValue(EMPTY);
mergeLength(0);
}
}
private void addOneValue(BytesRef value) {
mergeLength(value.length);
int ord = hash.add(value);
if (ord < 0) {
ord = -ord-1;
@ -100,54 +93,102 @@ class SortedBytesDVWriter extends DocValuesWriter {
pending[pendingIndex++] = ord;
}
private void mergeLength(int length) {
if (fixedLength == -2) {
fixedLength = length;
} else if (fixedLength != length) {
fixedLength = -1;
}
maxLength = Math.max(maxLength, length);
}
@Override
public void flush(SegmentWriteState state, SimpleDVConsumer dvConsumer) throws IOException {
SortedDocValuesConsumer consumer = dvConsumer.addSortedField(fieldInfo,
hash.size(),
fixedLength >= 0,
maxLength);
final int maxDoc = state.segmentInfo.getDocCount();
int emptyOrd = -1;
final int emptyOrd;
if (pendingIndex < maxDoc) {
// Make sure we added EMPTY value before sorting:
emptyOrd = hash.add(EMPTY);
if (emptyOrd < 0) {
emptyOrd = -emptyOrd-1;
int ord = hash.add(EMPTY);
if (ord < 0) {
emptyOrd = -ord-1;
} else {
emptyOrd = ord;
}
} else {
emptyOrd = -1;
}
int valueCount = hash.size();
final int valueCount = hash.size();
int[] sortedValues = hash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
final int[] sortedValues = hash.sort(BytesRef.getUTF8SortedAsUnicodeComparator());
final int sortedValueRamUsage = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + RamUsageEstimator.NUM_BYTES_INT*valueCount;
iwBytesUsed.addAndGet(sortedValueRamUsage);
final int[] ordMap = new int[valueCount];
// Write values, in sorted order:
BytesRef scratch = new BytesRef();
for(int ord=0;ord<valueCount;ord++) {
consumer.addValue(hash.get(sortedValues[ord], scratch));
ordMap[sortedValues[ord]] = ord;
}
final int bufferedDocCount = pendingIndex;
for(int docID=0;docID<bufferedDocCount;docID++) {
consumer.addDoc(ordMap[pending[docID]]);
}
for(int docID=bufferedDocCount;docID<maxDoc;docID++) {
consumer.addDoc(ordMap[emptyOrd]);
}
dvConsumer.addSortedField(fieldInfo,
// ord -> value
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new Iterator<BytesRef>() {
int ordUpto;
BytesRef scratch = new BytesRef();
@Override
public boolean hasNext() {
return ordUpto < valueCount;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() {
hash.get(sortedValues[ordUpto], scratch);
ordUpto++;
return scratch;
}
};
}
},
// doc -> ord
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new Iterator<Number>() {
int docUpto;
@Override
public boolean hasNext() {
return docUpto < maxDoc;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public Number next() {
int ord;
if (docUpto < bufferedDocCount) {
ord = pending[docUpto];
} else {
ord = emptyOrd;
}
docUpto++;
// nocommit make
// resuable Number?
return ordMap[ord];
}
};
}
});
iwBytesUsed.addAndGet(-sortedValueRamUsage);
reset();
consumer.finish();
}
public void abort() {
@ -159,7 +200,5 @@ class SortedBytesDVWriter extends DocValuesWriter {
pending = ArrayUtil.shrink(pending, DEFAULT_PENDING_SIZE);
pendingIndex = 0;
hash.clear();
fixedLength = -2;
maxLength = 0;
}
}

View File

@ -30,10 +30,6 @@ public abstract class SortedDocValues extends BinaryDocValues {
public abstract int getValueCount();
public abstract boolean isFixedLength();
public abstract int maxLength();
@Override
public void get(int docID, BytesRef result) {
int ord = getOrd(docID);
@ -174,16 +170,6 @@ public abstract class SortedDocValues extends BinaryDocValues {
public int size() {
return size;
}
@Override
public boolean isFixedLength() {
return true;
}
@Override
public int maxLength() {
return 0;
}
}
/** If {@code key} exists, returns its ordinal, else

View File

@ -937,16 +937,12 @@ class FieldCacheImpl implements FieldCache {
private final PackedInts.Reader termOrdToBytesOffset;
private final PackedInts.Reader docToTermOrd;
private final int numOrd;
private final int maxLength;
private final boolean isFixedLength;
public SortedDocValuesImpl(PagedBytes.Reader bytes, PackedInts.Reader termOrdToBytesOffset, PackedInts.Reader docToTermOrd, int numOrd, int maxLength, boolean isFixedLength) {
public SortedDocValuesImpl(PagedBytes.Reader bytes, PackedInts.Reader termOrdToBytesOffset, PackedInts.Reader docToTermOrd, int numOrd) {
this.bytes = bytes;
this.docToTermOrd = docToTermOrd;
this.termOrdToBytesOffset = termOrdToBytesOffset;
this.numOrd = numOrd;
this.maxLength = maxLength;
this.isFixedLength = isFixedLength;
}
@Override
@ -975,16 +971,6 @@ class FieldCacheImpl implements FieldCache {
bytes.fill(ret, termOrdToBytesOffset.get(ord));
}
@Override
public boolean isFixedLength() {
return isFixedLength;
}
@Override
public int maxLength() {
return maxLength;
}
@Override
public TermsEnum getTermsEnum() {
return this.new SortedDocValuesEnum();
@ -1207,9 +1193,6 @@ class FieldCacheImpl implements FieldCache {
int termOrd = 0;
int sameLength = -2;
int maxLength = -1;
// TODO: use Uninvert?
if (terms != null) {
@ -1221,12 +1204,6 @@ class FieldCacheImpl implements FieldCache {
if (term == null) {
break;
}
if (sameLength == -2) {
sameLength = term.length;
} else if (sameLength != term.length) {
sameLength = -1;
}
maxLength = Math.max(maxLength, term.length);
if (termOrd >= termCountHardLimit) {
break;
}
@ -1256,7 +1233,7 @@ class FieldCacheImpl implements FieldCache {
}
// maybe an int-only impl?
return new SortedDocValuesImpl(bytes.freeze(true), termOrdToBytesOffset.getMutable(), docToTermOrd.getMutable(), termOrd, maxLength, sameLength >= 0);
return new SortedDocValuesImpl(bytes.freeze(true), termOrdToBytesOffset.getMutable(), docToTermOrd.getMutable(), termOrd);
}
}
}
@ -1264,14 +1241,10 @@ class FieldCacheImpl implements FieldCache {
private static class BinaryDocValuesImpl extends BinaryDocValues {
private final PagedBytes.Reader bytes;
private final PackedInts.Reader docToOffset;
private final int maxLength;
private final boolean isFixedLength;
public BinaryDocValuesImpl(PagedBytes.Reader bytes, PackedInts.Reader docToOffset, int maxLength, boolean isFixedLength) {
public BinaryDocValuesImpl(PagedBytes.Reader bytes, PackedInts.Reader docToOffset) {
this.bytes = bytes;
this.docToOffset = docToOffset;
this.maxLength = maxLength;
this.isFixedLength = isFixedLength;
}
@Override
@ -1353,9 +1326,6 @@ class FieldCacheImpl implements FieldCache {
// pointer==0 means not set
bytes.copyUsingLengthPrefix(new BytesRef());
int sameLength = -2;
int maxLength = -1;
if (terms != null) {
int termCount = 0;
final TermsEnum termsEnum = terms.iterator(null);
@ -1372,12 +1342,6 @@ class FieldCacheImpl implements FieldCache {
if (term == null) {
break;
}
if (sameLength == -2) {
sameLength = term.length;
} else if (sameLength != term.length) {
sameLength = -1;
}
maxLength = Math.max(maxLength, term.length);
final long pointer = bytes.copyUsingLengthPrefix(term);
docs = termsEnum.docs(null, docs, 0);
while (true) {
@ -1391,7 +1355,7 @@ class FieldCacheImpl implements FieldCache {
}
// maybe an int-only impl?
return new BinaryDocValuesImpl(bytes.freeze(true), docToOffset.getMutable(), maxLength, sameLength >= 0);
return new BinaryDocValuesImpl(bytes.freeze(true), docToOffset.getMutable());
}
}
}

View File

@ -44,7 +44,6 @@ import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase;
import org.junit.Ignore;
/**
* A very simple demo used in the API documentation (src/java/overview.html).
@ -594,7 +593,6 @@ public class TestDemoDocValue extends LuceneTestCase {
IndexReader ireader = DirectoryReader.open(directory); // read-only=true
assert ireader.leaves().size() == 1;
SortedDocValues dv = ireader.leaves().get(0).reader().getSortedDocValues("dv");
assertTrue(dv.isFixedLength()); // "hello world 1" length == "hello world 2" length
assertEquals(2, dv.getValueCount()); // 2 ords
BytesRef scratch = new BytesRef();
dv.lookupOrd(0, scratch);