LUCENE-4547: DocValues improvements

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1443717 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2013-02-07 20:48:21 +00:00
commit a181dc0735
383 changed files with 16143 additions and 19545 deletions

View File

@ -43,6 +43,19 @@ Changes in backwards compatibility policy
PathPolicy was removed, you should extend FacetFields and DrillDownStream
to control which categories are added as drill-down terms. (Shai Erera)
* LUCENE-4547: DocValues improvements:
- Simplified codec API: codecs are now only responsible for encoding and
decoding docvalues, they do not need to do buffering or RAM accounting.
- Per-Field support: added PerFieldDocValuesFormat, which allows you to
use a different DocValuesFormat per field (like postings).
- Unified with FieldCache api: DocValues can be accessed via FieldCache API,
so it works automatically with grouping/join/sort/function queries, etc.
- Simplified types: There are only 3 types (NUMERIC, BINARY, SORTED), so its
not necessary to specify for example that all of your binary values have
the same length. Instead its easy for the Codec API to optimize encoding
based on any properties of the content.
(Simon Willnauer, Adrien Grand, Mike McCandless, Robert Muir)
Optimizations
* LUCENE-4687: BloomFilterPostingsFormat now lazily initializes delegate
@ -73,6 +86,12 @@ Optimizations
TermVectorsFormat (Lucene42TermVectorsFormat) based on
CompressingTermVectorsFormat. (Adrien Grand)
* LUCENE-3729: The default Lucene 4.2 codec now uses a more compact
DocValuesFormat (Lucene42DocValuesFormat). Sorted values are stored in an
FST, Numerics and Ordinals use a number of strategies (delta-compression,
table-compression, etc), and memory addresses use MonotonicBlockPackedWriter.
(Simon Willnauer, Adrien Grand, Mike McCandless, Robert Muir)
New Features
* LUCENE-4686: New specialized DGapVInt8IntEncoder for facets (now the
@ -100,6 +119,26 @@ New Features
MemoryIndex only once even if multiple fields are highlighted.
(Simon Willnauer)
* LUCENE-4035: Added ICUCollationDocValuesField, more efficient
support for Locale-sensitive sort and range queries for
single-valued fields. (Robert Muir)
* LUCENE-4547: Added MonotonicBlockPacked(Reader/Writer), which provide
efficient random access to large amounts of monotonically increasing
positive values (e.g. file offsets). Each block stores the minimum value
and the average gap, and values are encoded as signed deviations from
the expected value. (Adrien Grand)
* LUCENE-4547: Added AppendingLongBuffer, an append-only buffer that packs
signed long values in memory and provides an efficient iterator API.
(Adrien Grand)
* LUCENE-4540: It is now possible for a codec to represent norms with
less than 8 bits per value. For performance reasons this is not done
by default, but you can customize your codec (e.g. pass PackedInts.DEFAULT
to Lucene42DocValuesConsumer) if you want to make this tradeoff.
(Adrien Grand, Robert Muir)
API Changes
* LUCENE-4709: FacetResultNode no longer has a residue field. (Shai Erera)
@ -139,6 +178,15 @@ Bug Fixes
* LUCENE-4739: Fixed bugs that prevented FSTs more than ~1.1GB from
being saved and loaded (Adrien Grand, Mike McCandless)
* LUCENE-4717: Fixed bug where Lucene40DocValuesFormat would sometimes write
an extra unused ordinal for sorted types. The bug is detected and corrected
on-the-fly for old indexes. (Robert Muir)
* LUCENE-4547: Fixed bug where Lucene40DocValuesFormat was unable to encode
segments that would exceed 2GB total data. This could happen in some surprising
cases, for example if you had an index with more than 260M documents and a
VAR_INT field. (Simon Willnauer, Adrien Grand, Mike McCandless, Robert Muir)
Documentation
* LUCENE-4718: Fixed documentation of oal.queryparser.classic.

View File

@ -0,0 +1,77 @@
package org.apache.lucene.collation;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.document.Field;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.search.FieldCacheRangeFilter;
import org.apache.lucene.util.BytesRef;
import com.ibm.icu.text.Collator;
import com.ibm.icu.text.RawCollationKey;
/**
* Indexes collation keys as a single-valued {@link SortedDocValuesField}.
* <p>
* This is more efficient that {@link ICUCollationKeyAnalyzer} if the field
* only has one value: no uninversion is necessary to sort on the field,
* locale-sensitive range queries can still work via {@link FieldCacheRangeFilter},
* and the underlying data structures built at index-time are likely more efficient
* and use less memory than FieldCache.
*/
public final class ICUCollationDocValuesField extends Field {
private final String name;
private final Collator collator;
private final BytesRef bytes = new BytesRef();
private final RawCollationKey key = new RawCollationKey();
/**
* Create a new ICUCollationDocValuesField.
* <p>
* NOTE: you should not create a new one for each document, instead
* just make one and reuse it during your indexing process, setting
* the value via {@link #setStringValue(String)}.
* @param name field name
* @param collator Collator for generating collation keys.
*/
// TODO: can we make this trap-free? maybe just synchronize on the collator
// instead?
public ICUCollationDocValuesField(String name, Collator collator) {
super(name, SortedDocValuesField.TYPE);
this.name = name;
try {
this.collator = (Collator) collator.clone();
} catch (CloneNotSupportedException e) {
throw new RuntimeException(e);
}
fieldsData = bytes; // so wrong setters cannot be called
}
@Override
public String name() {
return name;
}
@Override
public void setStringValue(String value) {
collator.getRawCollationKey(value, key);
bytes.bytes = key.bytes;
bytes.offset = 0;
bytes.length = key.size;
}
}

View File

@ -0,0 +1,143 @@
package org.apache.lucene.collation;
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.FieldCacheRangeFilter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryUtils;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.SortField;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import com.ibm.icu.text.Collator;
import com.ibm.icu.util.ULocale;
/**
* trivial test of ICUCollationDocValuesField
*/
@SuppressCodecs("Lucene3x")
public class TestICUCollationDocValuesField extends LuceneTestCase {
public void testBasic() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
Document doc = new Document();
Field field = newField("field", "", StringField.TYPE_STORED);
ICUCollationDocValuesField collationField = new ICUCollationDocValuesField("collated", Collator.getInstance(ULocale.ENGLISH));
doc.add(field);
doc.add(collationField);
field.setStringValue("ABC");
collationField.setStringValue("ABC");
iw.addDocument(doc);
field.setStringValue("abc");
collationField.setStringValue("abc");
iw.addDocument(doc);
IndexReader ir = iw.getReader();
iw.close();
IndexSearcher is = newSearcher(ir);
SortField sortField = new SortField("collated", SortField.Type.STRING);
TopDocs td = is.search(new MatchAllDocsQuery(), 5, new Sort(sortField));
assertEquals("abc", ir.document(td.scoreDocs[0].doc).get("field"));
assertEquals("ABC", ir.document(td.scoreDocs[1].doc).get("field"));
ir.close();
dir.close();
}
public void testRanges() throws Exception {
Directory dir = newDirectory();
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
Document doc = new Document();
Field field = newField("field", "", StringField.TYPE_STORED);
Collator collator = Collator.getInstance(); // uses -Dtests.locale
if (random().nextBoolean()) {
collator.setStrength(Collator.PRIMARY);
}
ICUCollationDocValuesField collationField = new ICUCollationDocValuesField("collated", collator);
doc.add(field);
doc.add(collationField);
int numDocs = atLeast(500);
for (int i = 0; i < numDocs; i++) {
String value = _TestUtil.randomSimpleString(random());
field.setStringValue(value);
collationField.setStringValue(value);
iw.addDocument(doc);
}
IndexReader ir = iw.getReader();
iw.close();
IndexSearcher is = newSearcher(ir);
int numChecks = atLeast(100);
for (int i = 0; i < numChecks; i++) {
String start = _TestUtil.randomSimpleString(random());
String end = _TestUtil.randomSimpleString(random());
BytesRef lowerVal = new BytesRef(collator.getCollationKey(start).toByteArray());
BytesRef upperVal = new BytesRef(collator.getCollationKey(end).toByteArray());
Query query = new ConstantScoreQuery(FieldCacheRangeFilter.newBytesRefRange("collated", lowerVal, upperVal, true, true));
doTestRanges(is, start, end, query, collator);
}
ir.close();
dir.close();
}
private void doTestRanges(IndexSearcher is, String startPoint, String endPoint, Query query, Collator collator) throws Exception {
QueryUtils.check(query);
// positive test
TopDocs docs = is.search(query, is.getIndexReader().maxDoc());
for (ScoreDoc doc : docs.scoreDocs) {
String value = is.doc(doc.doc).get("field");
assertTrue(collator.compare(value, startPoint) >= 0);
assertTrue(collator.compare(value, endPoint) <= 0);
}
// negative test
BooleanQuery bq = new BooleanQuery();
bq.add(new MatchAllDocsQuery(), Occur.SHOULD);
bq.add(query, Occur.MUST_NOT);
docs = is.search(bq, is.getIndexReader().maxDoc());
for (ScoreDoc doc : docs.scoreDocs) {
String value = is.doc(doc.doc).get("field");
assertTrue(collator.compare(value, startPoint) < 0 || collator.compare(value, endPoint) > 0);
}
}
}

View File

@ -53,10 +53,10 @@ import org.apache.lucene.index.MultiFields;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.FieldCache.DocTermsIndex;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BytesRef;
@ -342,12 +342,11 @@ public class TestPerfTasksLogic extends BenchmarkTestCase {
Benchmark benchmark = execBenchmark(algLines);
DirectoryReader r = DirectoryReader.open(benchmark.getRunData().getDirectory());
DocTermsIndex idx = FieldCache.DEFAULT.getTermsIndex(new SlowCompositeReaderWrapper(r), "country");
SortedDocValues idx = FieldCache.DEFAULT.getTermsIndex(new SlowCompositeReaderWrapper(r), "country");
final int maxDoc = r.maxDoc();
assertEquals(1000, maxDoc);
BytesRef br = new BytesRef();
for(int i=0;i<1000;i++) {
assertNotNull("doc " + i + " has null country", idx.getTerm(i, br));
assertTrue("doc " + i + " has null country", idx.getOrd(i) != -1);
}
r.close();
}

View File

@ -163,7 +163,7 @@ public final class BloomFilteringPostingsFormat extends PostingsFormat {
IndexInput bloomIn = null;
boolean success = false;
try {
bloomIn = state.dir.openInput(bloomFileName, state.context);
bloomIn = state.directory.openInput(bloomFileName, state.context);
CodecUtil.checkHeader(bloomIn, BLOOM_CODEC_NAME, BLOOM_CODEC_VERSION,
BLOOM_CODEC_VERSION);
// // Load the hash function used in the BloomFilter

View File

@ -0,0 +1,142 @@
package org.apache.lucene.codecs.diskdv;
/*
* 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.DocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
/** writer for {@link DiskDocValuesFormat} */
public class DiskDocValuesConsumer extends DocValuesConsumer {
static final int BLOCK_SIZE = 16384;
final IndexOutput data, meta;
final int maxDoc;
public DiskDocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
boolean success = false;
try {
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = state.directory.createOutput(dataName, state.context);
CodecUtil.writeHeader(data, dataCodec, DiskDocValuesFormat.VERSION_CURRENT);
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
meta = state.directory.createOutput(metaName, state.context);
CodecUtil.writeHeader(meta, metaCodec, DiskDocValuesFormat.VERSION_CURRENT);
maxDoc = state.segmentInfo.getDocCount();
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this);
}
}
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
int count = 0;
for (@SuppressWarnings("unused") Number nv : values) {
++count;
}
meta.writeVInt(field.number);
meta.writeByte(DiskDocValuesFormat.NUMERIC);
meta.writeVInt(PackedInts.VERSION_CURRENT);
meta.writeLong(data.getFilePointer());
meta.writeVInt(count);
meta.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
writer.add(nv.longValue());
}
writer.finish();
}
@Override
public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
// write the byte[] data
meta.writeVInt(field.number);
meta.writeByte(DiskDocValuesFormat.BINARY);
int minLength = Integer.MAX_VALUE;
int maxLength = Integer.MIN_VALUE;
final long startFP = data.getFilePointer();
int count = 0;
for(BytesRef v : values) {
minLength = Math.min(minLength, v.length);
maxLength = Math.max(maxLength, v.length);
data.writeBytes(v.bytes, v.offset, v.length);
count++;
}
meta.writeVInt(minLength);
meta.writeVInt(maxLength);
meta.writeVInt(count);
meta.writeLong(startFP);
// if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
// otherwise, we need to record the length fields...
if (minLength != maxLength) {
meta.writeLong(data.getFilePointer());
meta.writeVInt(PackedInts.VERSION_CURRENT);
meta.writeVInt(BLOCK_SIZE);
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
long addr = 0;
for (BytesRef v : values) {
addr += v.length;
writer.add(addr);
}
writer.finish();
}
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(DiskDocValuesFormat.SORTED);
addBinaryField(field, values);
addNumericField(field, docToOrd);
}
@Override
public void close() throws IOException {
boolean success = false;
try {
if (meta != null) {
meta.writeVInt(-1); // write EOF marker
}
success = true;
} finally {
if (success) {
IOUtils.close(data, meta);
} else {
IOUtils.closeWhileHandlingException(data, meta);
}
}
}
}

View File

@ -0,0 +1,61 @@
package org.apache.lucene.codecs.diskdv;
/*
* 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.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* DocValues format that keeps most things on disk.
* <p>
* Things like ordinals and disk offsets are loaded into ram,
* for single-seek access to all the types.
* <p>
* @lucene.experimental
*/
public final class DiskDocValuesFormat extends DocValuesFormat {
public DiskDocValuesFormat() {
super("Disk");
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new DiskDocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new DiskDocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
public static final String DATA_CODEC = "DiskDocValuesData";
public static final String DATA_EXTENSION = "dvdd";
public static final String META_CODEC = "DiskDocValuesMetadata";
public static final String META_EXTENSION = "dvdm";
public static final int VERSION_START = 0;
public static final int VERSION_CURRENT = VERSION_START;
public static final byte NUMERIC = 0;
public static final byte BINARY = 1;
public static final byte SORTED = 2;
}

View File

@ -0,0 +1,277 @@
package org.apache.lucene.codecs.diskdv;
/*
* 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.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.BlockPackedReader;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
class DiskDocValuesProducer extends DocValuesProducer {
private final Map<Integer,NumericEntry> numerics;
private final Map<Integer,BinaryEntry> binaries;
private final Map<Integer,NumericEntry> ords;
private final IndexInput data;
// memory-resident structures
private final Map<Integer,BlockPackedReader> ordinalInstances = new HashMap<Integer,BlockPackedReader>();
private final Map<Integer,MonotonicBlockPackedReader> addressInstances = new HashMap<Integer,MonotonicBlockPackedReader>();
DiskDocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
// read in the entries from the metadata file.
IndexInput in = state.directory.openInput(metaName, state.context);
boolean success = false;
try {
CodecUtil.checkHeader(in, metaCodec,
DiskDocValuesFormat.VERSION_START,
DiskDocValuesFormat.VERSION_START);
numerics = new HashMap<Integer,NumericEntry>();
ords = new HashMap<Integer,NumericEntry>();
binaries = new HashMap<Integer,BinaryEntry>();
readFields(in, state.fieldInfos);
success = true;
} finally {
if (success) {
IOUtils.close(in);
} else {
IOUtils.closeWhileHandlingException(in);
}
}
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = state.directory.openInput(dataName, state.context);
CodecUtil.checkHeader(data, dataCodec,
DiskDocValuesFormat.VERSION_START,
DiskDocValuesFormat.VERSION_START);
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
byte type = meta.readByte();
if (type == DiskDocValuesFormat.NUMERIC) {
numerics.put(fieldNumber, readNumericEntry(meta));
} else if (type == DiskDocValuesFormat.BINARY) {
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
} else if (type == DiskDocValuesFormat.SORTED) {
// sorted = binary + numeric
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
}
if (meta.readByte() != DiskDocValuesFormat.BINARY) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
}
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
if (meta.readVInt() != fieldNumber) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
}
if (meta.readByte() != DiskDocValuesFormat.NUMERIC) {
throw new CorruptIndexException("sorted entry for field: " + fieldNumber + " is corrupt");
}
NumericEntry n = readNumericEntry(meta);
ords.put(fieldNumber, n);
}
fieldNumber = meta.readVInt();
}
}
static NumericEntry readNumericEntry(IndexInput meta) throws IOException {
NumericEntry entry = new NumericEntry();
entry.packedIntsVersion = meta.readVInt();
entry.offset = meta.readLong();
entry.count = meta.readVInt();
entry.blockSize = meta.readVInt();
return entry;
}
static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
BinaryEntry entry = new BinaryEntry();
entry.minLength = meta.readVInt();
entry.maxLength = meta.readVInt();
entry.count = meta.readVInt();
entry.offset = meta.readLong();
if (entry.minLength != entry.maxLength) {
entry.addressesOffset = meta.readLong();
entry.packedIntsVersion = meta.readVInt();
entry.blockSize = meta.readVInt();
}
return entry;
}
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericEntry entry = numerics.get(field.number);
final IndexInput data = this.data.clone();
data.seek(entry.offset);
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, true);
return new NumericDocValues() {
@Override
public long get(int docID) {
return reader.get(docID);
}
};
}
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryEntry bytes = binaries.get(field.number);
if (bytes.minLength == bytes.maxLength) {
return getFixedBinary(field, bytes);
} else {
return getVariableBinary(field, bytes);
}
}
private BinaryDocValues getFixedBinary(FieldInfo field, final BinaryEntry bytes) {
final IndexInput data = this.data.clone();
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
long address = bytes.offset + docID * (long)bytes.maxLength;
try {
data.seek(address);
// NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource)
// assume "they" own the bytes after calling this!
final byte[] buffer = new byte[bytes.maxLength];
data.readBytes(buffer, 0, buffer.length);
result.bytes = buffer;
result.offset = 0;
result.length = buffer.length;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
};
}
private BinaryDocValues getVariableBinary(FieldInfo field, final BinaryEntry bytes) throws IOException {
final IndexInput data = this.data.clone();
final MonotonicBlockPackedReader addresses;
synchronized (addressInstances) {
MonotonicBlockPackedReader addrInstance = addressInstances.get(field.number);
if (addrInstance == null) {
data.seek(bytes.addressesOffset);
addrInstance = new MonotonicBlockPackedReader(data, bytes.packedIntsVersion, bytes.blockSize, bytes.count, false);
addressInstances.put(field.number, addrInstance);
}
addresses = addrInstance;
}
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
long startAddress = bytes.offset + (docID == 0 ? 0 : + addresses.get(docID-1));
long endAddress = bytes.offset + addresses.get(docID);
int length = (int) (endAddress - startAddress);
try {
data.seek(startAddress);
// NOTE: we could have one buffer, but various consumers (e.g. FieldComparatorSource)
// assume "they" own the bytes after calling this!
final byte[] buffer = new byte[length];
data.readBytes(buffer, 0, buffer.length);
result.bytes = buffer;
result.offset = 0;
result.length = length;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
};
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
final int valueCount = binaries.get(field.number).count;
final BinaryDocValues binary = getBinary(field);
final BlockPackedReader ordinals;
synchronized (ordinalInstances) {
BlockPackedReader ordsInstance = ordinalInstances.get(field.number);
if (ordsInstance == null) {
NumericEntry entry = ords.get(field.number);
IndexInput data = this.data.clone();
data.seek(entry.offset);
ordsInstance = new BlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, entry.count, false);
ordinalInstances.put(field.number, ordsInstance);
}
ordinals = ordsInstance;
}
return new SortedDocValues() {
@Override
public int getOrd(int docID) {
return (int) ordinals.get(docID);
}
@Override
public void lookupOrd(int ord, BytesRef result) {
binary.get(ord, result);
}
@Override
public int getValueCount() {
return valueCount;
}
};
}
@Override
public void close() throws IOException {
data.close();
}
static class NumericEntry {
long offset;
int packedIntsVersion;
int count;
int blockSize;
}
static class BinaryEntry {
long offset;
int count;
int minLength;
int maxLength;
long addressesOffset;
int packedIntsVersion;
int blockSize;
}
}

View File

@ -20,6 +20,6 @@
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Default DocValues implementation for Lucene 4.0 indexes.
DocValuesFormat that accesses values directly from disk.
</body>
</html>
</html>

View File

@ -842,7 +842,7 @@ public final class MemoryPostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, EXTENSION);
final IndexInput in = state.dir.openInput(fileName, IOContext.READONCE);
final IndexInput in = state.directory.openInput(fileName, IOContext.READONCE);
final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();

View File

@ -100,7 +100,7 @@ public abstract class PulsingPostingsFormat extends PostingsFormat {
docsReader = wrappedPostingsBaseFormat.postingsReaderBase(state);
pulsingReader = new PulsingPostingsReader(docsReader);
FieldsProducer ret = new BlockTreeTermsReader(
state.dir, state.fieldInfos, state.segmentInfo,
state.directory, state.fieldInfos, state.segmentInfo,
pulsingReader,
state.context,
state.segmentSuffix,

View File

@ -1,91 +0,0 @@
package org.apache.lucene.codecs.sep;
/*
* 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 java.util.Collection;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.codecs.lucene40.values.Bytes;
import org.apache.lucene.codecs.lucene40.values.Floats;
import org.apache.lucene.codecs.lucene40.values.Ints;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
/**
* Implementation of PerDocProducer that uses separate files.
* @lucene.experimental
*/
public class SepDocValuesProducer extends PerDocProducerBase {
private final TreeMap<String, DocValues> docValues;
/**
* Creates a new {@link SepDocValuesProducer} instance and loads all
* {@link DocValues} instances for this segment and codec.
*/
public SepDocValuesProducer(SegmentReadState state) throws IOException {
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.getDocCount(), state.dir, state.context);
}
@Override
protected Map<String,DocValues> docValues() {
return docValues;
}
@Override
protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
IOUtils.close(closeables);
}
@Override
protected DocValues loadDocValues(int docCount, Directory dir, String id,
Type type, IOContext context) throws IOException {
switch (type) {
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
return Ints.getValues(dir, id, docCount, type, context);
case FLOAT_32:
return Floats.getValues(dir, id, docCount, context, type);
case FLOAT_64:
return Floats.getValues(dir, id, docCount, context, type);
case BYTES_FIXED_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
case BYTES_FIXED_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
case BYTES_FIXED_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
case BYTES_VAR_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
case BYTES_VAR_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
case BYTES_VAR_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
default:
throw new IllegalStateException("unrecognized index values mode " + type);
}
}
}

View File

@ -18,12 +18,12 @@ package org.apache.lucene.codecs.simpletext;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
@ -39,11 +39,9 @@ public final class SimpleTextCodec extends Codec {
private final SegmentInfoFormat segmentInfos = new SimpleTextSegmentInfoFormat();
private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
// TODO: need a plain-text impl
private final DocValuesFormat docValues = new SimpleTextDocValuesFormat();
// TODO: need a plain-text impl (using the above)
private final NormsFormat normsFormat = new SimpleTextNormsFormat();
private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
private final DocValuesFormat dvFormat = new SimpleTextDocValuesFormat();
public SimpleTextCodec() {
super("SimpleText");
@ -54,11 +52,6 @@ public final class SimpleTextCodec extends Codec {
return postings;
}
@Override
public DocValuesFormat docValuesFormat() {
return docValues;
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return storedFields;
@ -88,4 +81,9 @@ public final class SimpleTextCodec extends Codec {
public LiveDocsFormat liveDocsFormat() {
return liveDocs;
}
@Override
public DocValuesFormat docValuesFormat() {
return dvFormat;
}
}

View File

@ -1,295 +0,0 @@
package org.apache.lucene.codecs.simpletext;
/*
* 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.DocValuesArraySource;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.IOUtils;
/**
* Writes plain-text DocValues.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
*/
public class SimpleTextDocValuesConsumer extends DocValuesConsumer {
static final BytesRef ZERO_DOUBLE = new BytesRef(Double.toString(0d));
static final BytesRef ZERO_INT = new BytesRef(Integer.toString(0));
static final BytesRef HEADER = new BytesRef("SimpleTextDocValues");
static final BytesRef END = new BytesRef("END");
static final BytesRef VALUE_SIZE = new BytesRef("valuesize ");
static final BytesRef DOC = new BytesRef(" doc ");
static final BytesRef VALUE = new BytesRef(" value ");
protected BytesRef scratch = new BytesRef();
protected int maxDocId = -1;
protected final String segment;
protected final Directory dir;
protected final IOContext ctx;
protected final Type type;
protected final BytesRefHash hash;
private int[] ords;
private int valueSize = Integer.MIN_VALUE;
private BytesRef zeroBytes;
private final String segmentSuffix;
public SimpleTextDocValuesConsumer(String segment, Directory dir,
IOContext ctx, Type type, String segmentSuffix) {
this.ctx = ctx;
this.dir = dir;
this.segment = segment;
this.type = type;
hash = new BytesRefHash();
ords = new int[0];
this.segmentSuffix = segmentSuffix;
}
@Override
public void add(int docID, StorableField value) throws IOException {
assert docID >= 0;
final int ord, vSize;
switch (type) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
vSize = value.binaryValue().length;
ord = hash.add(value.binaryValue());
break;
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
vSize = -1;
ord = hash.add(value.binaryValue());
break;
case FIXED_INTS_16:
vSize = 2;
scratch.grow(2);
DocValuesArraySource.copyShort(scratch, value.numericValue().shortValue());
ord = hash.add(scratch);
break;
case FIXED_INTS_32:
vSize = 4;
scratch.grow(4);
DocValuesArraySource.copyInt(scratch, value.numericValue().intValue());
ord = hash.add(scratch);
break;
case FIXED_INTS_8:
vSize = 1;
scratch.grow(1);
scratch.bytes[scratch.offset] = value.numericValue().byteValue();
scratch.length = 1;
ord = hash.add(scratch);
break;
case FIXED_INTS_64:
vSize = 8;
scratch.grow(8);
DocValuesArraySource.copyLong(scratch, value.numericValue().longValue());
ord = hash.add(scratch);
break;
case VAR_INTS:
vSize = -1;
scratch.grow(8);
DocValuesArraySource.copyLong(scratch, value.numericValue().longValue());
ord = hash.add(scratch);
break;
case FLOAT_32:
vSize = 4;
scratch.grow(4);
DocValuesArraySource.copyInt(scratch,
Float.floatToRawIntBits(value.numericValue().floatValue()));
ord = hash.add(scratch);
break;
case FLOAT_64:
vSize = 8;
scratch.grow(8);
DocValuesArraySource.copyLong(scratch,
Double.doubleToRawLongBits(value.numericValue().doubleValue()));
ord = hash.add(scratch);
break;
default:
throw new RuntimeException("should not reach this line");
}
if (valueSize == Integer.MIN_VALUE) {
assert maxDocId == -1;
valueSize = vSize;
} else {
if (valueSize != vSize) {
throw new IllegalArgumentException("value size must be " + valueSize + " but was: " + vSize);
}
}
maxDocId = Math.max(docID, maxDocId);
ords = grow(ords, docID);
ords[docID] = (ord < 0 ? (-ord)-1 : ord) + 1;
}
protected BytesRef getHeader() {
return HEADER;
}
private int[] grow(int[] array, int upto) {
if (array.length <= upto) {
return ArrayUtil.grow(array, 1 + upto);
}
return array;
}
private void prepareFlush(int docCount) {
assert ords != null;
ords = grow(ords, docCount);
}
@Override
public void finish(int docCount) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segment, "",
segmentSuffix);
IndexOutput output = dir.createOutput(fileName, ctx);
boolean success = false;
BytesRef spare = new BytesRef();
try {
SimpleTextUtil.write(output, getHeader());
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, VALUE_SIZE);
SimpleTextUtil.write(output, Integer.toString(this.valueSize), scratch);
SimpleTextUtil.writeNewline(output);
prepareFlush(docCount);
for (int i = 0; i < docCount; i++) {
SimpleTextUtil.write(output, DOC);
SimpleTextUtil.write(output, Integer.toString(i), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, VALUE);
writeDoc(output, i, spare);
SimpleTextUtil.writeNewline(output);
}
SimpleTextUtil.write(output, END);
SimpleTextUtil.writeNewline(output);
success = true;
} finally {
hash.close();
if (success) {
IOUtils.close(output);
} else {
IOUtils.closeWhileHandlingException(output);
dir.deleteFile(fileName);
}
}
}
protected void writeDoc(IndexOutput output, int docId, BytesRef spare) throws IOException {
int ord = ords[docId] - 1;
if (ord != -1) {
assert ord >= 0;
hash.get(ord, spare);
switch (type) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
SimpleTextUtil.write(output, spare);
break;
case FIXED_INTS_16:
SimpleTextUtil.write(output,
Short.toString(DocValuesArraySource.asShort(spare)), scratch);
break;
case FIXED_INTS_32:
SimpleTextUtil.write(output,
Integer.toString(DocValuesArraySource.asInt(spare)), scratch);
break;
case VAR_INTS:
case FIXED_INTS_64:
SimpleTextUtil.write(output,
Long.toString(DocValuesArraySource.asLong(spare)), scratch);
break;
case FIXED_INTS_8:
assert spare.length == 1 : spare.length;
SimpleTextUtil.write(output,
Integer.toString(spare.bytes[spare.offset]), scratch);
break;
case FLOAT_32:
float valueFloat = Float.intBitsToFloat(DocValuesArraySource.asInt(spare));
SimpleTextUtil.write(output, Float.toString(valueFloat), scratch);
break;
case FLOAT_64:
double valueDouble = Double.longBitsToDouble(DocValuesArraySource
.asLong(spare));
SimpleTextUtil.write(output, Double.toString(valueDouble), scratch);
break;
default:
throw new IllegalArgumentException("unsupported type: " + type);
}
} else {
switch (type) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
if(zeroBytes == null) {
assert valueSize > 0;
zeroBytes = new BytesRef(new byte[valueSize]);
}
SimpleTextUtil.write(output, zeroBytes);
break;
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
scratch.length = 0;
SimpleTextUtil.write(output, scratch);
break;
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
SimpleTextUtil.write(output, ZERO_INT);
break;
case FLOAT_32:
case FLOAT_64:
SimpleTextUtil.write(output, ZERO_DOUBLE);
break;
default:
throw new IllegalArgumentException("unsupported type: " + type);
}
}
}
@Override
protected Type getType() {
return type;
}
@Override
public int getValueSize() {
return valueSize;
}
}

View File

@ -2,50 +2,104 @@ package org.apache.lucene.codecs.simpletext;
/*
* 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
*
* 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.
* 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.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.index.SegmentWriteState;
/**
* Plain-text DocValues format.
* plain text doc values format.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
* <p>
* the .dat file contains the data.
* for numbers this is a "fixed-width" file, for example a single byte range:
* <pre>
* field myField
* type NUMERIC
* minvalue 0
* pattern 000
* 005
* 234
* 123
* ...
* </pre>
* so a document's value (delta encoded from minvalue) can be retrieved by
* seeking to startOffset + (1+pattern.length())*docid. The extra 1 is the newline.
*
* for bytes this is also a "fixed-width" file, for example:
* <pre>
* field myField
* type BINARY
* maxlength 6
* pattern 0
* length 6
* foobar[space][space]
* length 3
* baz[space][space][space][space][space]
* ...
* </pre>
* so a doc's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*doc
* the extra 9 is 2 newlines, plus "length " itself.
*
* for sorted bytes this is a fixed-width file, for example:
* <pre>
* field myField
* type SORTED
* numvalues 10
* maxLength 8
* pattern 0
* ordpattern 00
* length 6
* foobar[space][space]
* length 3
* baz[space][space][space][space][space]
* ...
* 03
* 06
* 01
* 10
* ...
* </pre>
* so the "ord section" begins at startOffset + (9+pattern.length+maxlength)*numValues.
* a document's ord can be retrieved by seeking to "ord section" + (1+ordpattern.length())*docid
* an ord's value can be retrieved by seeking to startOffset + (9+pattern.length+maxlength)*ord
*
* the reader can just scan this file when it opens, skipping over the data blocks
* and saving the offset/etc for each field.
* @lucene.experimental
*/
public class SimpleTextDocValuesFormat extends DocValuesFormat {
private static final String DOC_VALUES_SEG_SUFFIX = "dv";
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new SimpleTextPerDocConsumer(state, DOC_VALUES_SEG_SUFFIX);
public SimpleTextDocValuesFormat() {
super("SimpleText");
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new SimpleTextPerDocProducer(state, BytesRef.getUTF8SortedAsUnicodeComparator(), DOC_VALUES_SEG_SUFFIX);
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new SimpleTextDocValuesWriter(state, "dat");
}
static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId;
@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new SimpleTextDocValuesReader(state, "dat");
}
}

View File

@ -0,0 +1,307 @@
package org.apache.lucene.codecs.simpletext;
/*
* 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.math.BigDecimal;
import java.math.BigInteger;
import java.text.DecimalFormat;
import java.text.DecimalFormatSymbols;
import java.text.ParseException;
import java.util.HashMap;
import java.util.Locale;
import java.util.Map;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.END;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.FIELD;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.LENGTH;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MAXLENGTH;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.MINVALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.NUMVALUES;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.ORDPATTERN;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.PATTERN;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesWriter.TYPE;
class SimpleTextDocValuesReader extends DocValuesProducer {
static class OneField {
long dataStartFilePointer;
String pattern;
String ordPattern;
int maxLength;
boolean fixedLength;
long minValue;
int numValues;
};
final int maxDoc;
final IndexInput data;
final BytesRef scratch = new BytesRef();
final Map<String,OneField> fields = new HashMap<String,OneField>();
public SimpleTextDocValuesReader(SegmentReadState state, String ext) throws IOException {
//System.out.println("dir=" + state.directory + " seg=" + state.segmentInfo.name + " ext=" + ext);
data = state.directory.openInput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
maxDoc = state.segmentInfo.getDocCount();
while(true) {
readLine();
//System.out.println("READ field=" + scratch.utf8ToString());
if (scratch.equals(END)) {
break;
}
assert startsWith(FIELD) : scratch.utf8ToString();
String fieldName = stripPrefix(FIELD);
//System.out.println(" field=" + fieldName);
FieldInfo fieldInfo = state.fieldInfos.fieldInfo(fieldName);
assert fieldInfo != null;
OneField field = new OneField();
fields.put(fieldName, field);
readLine();
assert startsWith(TYPE) : scratch.utf8ToString();
DocValuesType dvType = DocValuesType.valueOf(stripPrefix(TYPE));
assert dvType != null;
if (dvType == DocValuesType.NUMERIC) {
readLine();
assert startsWith(MINVALUE): "got " + scratch.utf8ToString() + " field=" + fieldName + " ext=" + ext;
field.minValue = Long.parseLong(stripPrefix(MINVALUE));
readLine();
assert startsWith(PATTERN);
field.pattern = stripPrefix(PATTERN);
field.dataStartFilePointer = data.getFilePointer();
data.seek(data.getFilePointer() + (1+field.pattern.length()) * maxDoc);
} else if (dvType == DocValuesType.BINARY) {
readLine();
assert startsWith(MAXLENGTH);
field.maxLength = Integer.parseInt(stripPrefix(MAXLENGTH));
readLine();
assert startsWith(PATTERN);
field.pattern = stripPrefix(PATTERN);
field.dataStartFilePointer = data.getFilePointer();
data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * maxDoc);
} else if (dvType == DocValuesType.SORTED) {
readLine();
assert startsWith(NUMVALUES);
field.numValues = Integer.parseInt(stripPrefix(NUMVALUES));
readLine();
assert startsWith(MAXLENGTH);
field.maxLength = Integer.parseInt(stripPrefix(MAXLENGTH));
readLine();
assert startsWith(PATTERN);
field.pattern = stripPrefix(PATTERN);
readLine();
assert startsWith(ORDPATTERN);
field.ordPattern = stripPrefix(ORDPATTERN);
field.dataStartFilePointer = data.getFilePointer();
data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * field.numValues + (1+field.ordPattern.length())*maxDoc);
} else {
throw new AssertionError();
}
}
// We should only be called from above if at least one
// field has DVs:
assert !fields.isEmpty();
}
@Override
public NumericDocValues getNumeric(FieldInfo fieldInfo) throws IOException {
final OneField field = fields.get(fieldInfo.name);
assert field != null;
// SegmentCoreReaders already verifies this field is
// valid:
assert field != null: "field=" + fieldInfo.name + " fields=" + fields;
final IndexInput in = data.clone();
final BytesRef scratch = new BytesRef();
final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
decoder.setParseBigDecimal(true);
return new NumericDocValues() {
@Override
public long get(int docID) {
try {
//System.out.println(Thread.currentThread().getName() + ": get docID=" + docID + " in=" + in);
if (docID < 0 || docID >= maxDoc) {
throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
}
in.seek(field.dataStartFilePointer + (1+field.pattern.length())*docID);
SimpleTextUtil.readLine(in, scratch);
//System.out.println("parsing delta: " + scratch.utf8ToString());
BigDecimal bd;
try {
bd = (BigDecimal) decoder.parse(scratch.utf8ToString());
} catch (ParseException pe) {
CorruptIndexException e = new CorruptIndexException("failed to parse BigDecimal value");
e.initCause(pe);
throw e;
}
return BigInteger.valueOf(field.minValue).add(bd.toBigIntegerExact()).longValue();
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
};
}
@Override
public BinaryDocValues getBinary(FieldInfo fieldInfo) throws IOException {
final OneField field = fields.get(fieldInfo.name);
// SegmentCoreReaders already verifies this field is
// valid:
assert field != null;
final IndexInput in = data.clone();
final BytesRef scratch = new BytesRef();
final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
try {
if (docID < 0 || docID >= maxDoc) {
throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
}
in.seek(field.dataStartFilePointer + (9+field.pattern.length() + field.maxLength)*docID);
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, LENGTH);
int len;
try {
len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
} catch (ParseException pe) {
CorruptIndexException e = new CorruptIndexException("failed to parse int length");
e.initCause(pe);
throw e;
}
result.bytes = new byte[len];
result.offset = 0;
result.length = len;
in.readBytes(result.bytes, 0, len);
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
};
}
@Override
public SortedDocValues getSorted(FieldInfo fieldInfo) throws IOException {
final OneField field = fields.get(fieldInfo.name);
// SegmentCoreReaders already verifies this field is
// valid:
assert field != null;
final IndexInput in = data.clone();
final BytesRef scratch = new BytesRef();
final DecimalFormat decoder = new DecimalFormat(field.pattern, new DecimalFormatSymbols(Locale.ROOT));
final DecimalFormat ordDecoder = new DecimalFormat(field.ordPattern, new DecimalFormatSymbols(Locale.ROOT));
return new SortedDocValues() {
@Override
public int getOrd(int docID) {
if (docID < 0 || docID >= maxDoc) {
throw new IndexOutOfBoundsException("docID must be 0 .. " + (maxDoc-1) + "; got " + docID);
}
try {
in.seek(field.dataStartFilePointer + field.numValues * (9 + field.pattern.length() + field.maxLength) + docID * (1 + field.ordPattern.length()));
SimpleTextUtil.readLine(in, scratch);
try {
return ordDecoder.parse(scratch.utf8ToString()).intValue();
} catch (ParseException pe) {
CorruptIndexException e = new CorruptIndexException("failed to parse ord");
e.initCause(pe);
throw e;
}
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
@Override
public void lookupOrd(int ord, BytesRef result) {
try {
if (ord < 0 || ord >= field.numValues) {
throw new IndexOutOfBoundsException("ord must be 0 .. " + (field.numValues-1) + "; got " + ord);
}
in.seek(field.dataStartFilePointer + ord * (9 + field.pattern.length() + field.maxLength));
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, LENGTH): "got " + scratch.utf8ToString() + " in=" + in;
int len;
try {
len = decoder.parse(new String(scratch.bytes, scratch.offset + LENGTH.length, scratch.length - LENGTH.length, "UTF-8")).intValue();
} catch (ParseException pe) {
CorruptIndexException e = new CorruptIndexException("failed to parse int length");
e.initCause(pe);
throw e;
}
result.bytes = new byte[len];
result.offset = 0;
result.length = len;
in.readBytes(result.bytes, 0, len);
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
@Override
public int getValueCount() {
return field.numValues;
}
};
}
@Override
public void close() throws IOException {
data.close();
}
/** Used only in ctor: */
private void readLine() throws IOException {
SimpleTextUtil.readLine(data, scratch);
//System.out.println("line: " + scratch.utf8ToString());
}
/** Used only in ctor: */
private boolean startsWith(BytesRef prefix) {
return StringHelper.startsWith(scratch, prefix);
}
/** Used only in ctor: */
private String stripPrefix(BytesRef prefix) throws IOException {
return new String(scratch.bytes, scratch.offset + prefix.length, scratch.length - prefix.length, "UTF-8");
}
}

View File

@ -0,0 +1,281 @@
package org.apache.lucene.codecs.simpletext;
/*
* 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.math.BigInteger;
import java.text.DecimalFormat;
import java.text.DecimalFormatSymbols;
import java.util.HashSet;
import java.util.Locale;
import java.util.Set;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
class SimpleTextDocValuesWriter extends DocValuesConsumer {
final static BytesRef END = new BytesRef("END");
final static BytesRef FIELD = new BytesRef("field ");
final static BytesRef TYPE = new BytesRef(" type ");
// used for numerics
final static BytesRef MINVALUE = new BytesRef(" minvalue ");
final static BytesRef PATTERN = new BytesRef(" pattern ");
// used for bytes
final static BytesRef LENGTH = new BytesRef("length ");
final static BytesRef MAXLENGTH = new BytesRef(" maxlength ");
// used for sorted bytes
final static BytesRef NUMVALUES = new BytesRef(" numvalues ");
final static BytesRef ORDPATTERN = new BytesRef(" ordpattern ");
final IndexOutput data;
final BytesRef scratch = new BytesRef();
final int numDocs;
private final Set<String> fieldsSeen = new HashSet<String>(); // for asserting
public SimpleTextDocValuesWriter(SegmentWriteState state, String ext) throws IOException {
//System.out.println("WRITE: " + IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext) + " " + state.segmentInfo.getDocCount() + " docs");
data = state.directory.createOutput(IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, ext), state.context);
numDocs = state.segmentInfo.getDocCount();
}
// for asserting
private boolean fieldSeen(String field) {
assert !fieldsSeen.contains(field): "field \"" + field + "\" was added more than once during flush";
fieldsSeen.add(field);
return true;
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
assert fieldSeen(field.name);
assert (field.getDocValuesType() == FieldInfo.DocValuesType.NUMERIC ||
field.getNormType() == FieldInfo.DocValuesType.NUMERIC);
writeFieldEntry(field, FieldInfo.DocValuesType.NUMERIC);
// first pass to find min/max
long minValue = Long.MAX_VALUE;
long maxValue = Long.MIN_VALUE;
for(Number n : values) {
long v = n.longValue();
minValue = Math.min(minValue, v);
maxValue = Math.max(maxValue, v);
}
// write our minimum value to the .dat, all entries are deltas from that
SimpleTextUtil.write(data, MINVALUE);
SimpleTextUtil.write(data, Long.toString(minValue), scratch);
SimpleTextUtil.writeNewline(data);
// build up our fixed-width "simple text packed ints"
// format
BigInteger maxBig = BigInteger.valueOf(maxValue);
BigInteger minBig = BigInteger.valueOf(minValue);
BigInteger diffBig = maxBig.subtract(minBig);
int maxBytesPerValue = diffBig.toString().length();
StringBuilder sb = new StringBuilder();
for (int i = 0; i < maxBytesPerValue; i++) {
sb.append('0');
}
// write our pattern to the .dat
SimpleTextUtil.write(data, PATTERN);
SimpleTextUtil.write(data, sb.toString(), scratch);
SimpleTextUtil.writeNewline(data);
final String patternString = sb.toString();
final DecimalFormat encoder = new DecimalFormat(patternString, new DecimalFormatSymbols(Locale.ROOT));
int numDocsWritten = 0;
// second pass to write the values
for(Number n : values) {
long value = n.longValue();
assert value >= minValue;
Number delta = BigInteger.valueOf(value).subtract(BigInteger.valueOf(minValue));
String s = encoder.format(delta);
assert s.length() == patternString.length();
SimpleTextUtil.write(data, s, scratch);
SimpleTextUtil.writeNewline(data);
numDocsWritten++;
assert numDocsWritten <= numDocs;
}
assert numDocs == numDocsWritten: "numDocs=" + numDocs + " numDocsWritten=" + numDocsWritten;
}
@Override
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
assert fieldSeen(field.name);
assert field.getDocValuesType() == DocValuesType.BINARY;
int maxLength = 0;
for(BytesRef value : values) {
maxLength = Math.max(maxLength, value.length);
}
writeFieldEntry(field, FieldInfo.DocValuesType.BINARY);
// write maxLength
SimpleTextUtil.write(data, MAXLENGTH);
SimpleTextUtil.write(data, Integer.toString(maxLength), scratch);
SimpleTextUtil.writeNewline(data);
int maxBytesLength = Long.toString(maxLength).length();
StringBuilder sb = new StringBuilder();
for (int i = 0; i < maxBytesLength; i++) {
sb.append('0');
}
// write our pattern for encoding lengths
SimpleTextUtil.write(data, PATTERN);
SimpleTextUtil.write(data, sb.toString(), scratch);
SimpleTextUtil.writeNewline(data);
final DecimalFormat encoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
int numDocsWritten = 0;
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);
// pad to fit
for (int i = value.length; i < maxLength; i++) {
data.writeByte((byte)' ');
}
SimpleTextUtil.writeNewline(data);
numDocsWritten++;
}
assert numDocs == numDocsWritten;
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
assert fieldSeen(field.name);
assert field.getDocValuesType() == DocValuesType.SORTED;
writeFieldEntry(field, FieldInfo.DocValuesType.SORTED);
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 maxLength
SimpleTextUtil.write(data, MAXLENGTH);
SimpleTextUtil.write(data, Integer.toString(maxLength), scratch);
SimpleTextUtil.writeNewline(data);
int maxBytesLength = Integer.toString(maxLength).length();
StringBuilder sb = new StringBuilder();
for (int i = 0; i < maxBytesLength; i++) {
sb.append('0');
}
// write our pattern for encoding lengths
SimpleTextUtil.write(data, PATTERN);
SimpleTextUtil.write(data, sb.toString(), scratch);
SimpleTextUtil.writeNewline(data);
final DecimalFormat encoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
int maxOrdBytes = Integer.toString(valueCount).length();
sb.setLength(0);
for (int i = 0; i < maxOrdBytes; i++) {
sb.append('0');
}
// write our pattern for ords
SimpleTextUtil.write(data, ORDPATTERN);
SimpleTextUtil.write(data, sb.toString(), scratch);
SimpleTextUtil.writeNewline(data);
final DecimalFormat ordEncoder = new DecimalFormat(sb.toString(), new DecimalFormatSymbols(Locale.ROOT));
// for asserts:
int valuesSeen = 0;
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);
// pad to fit
for (int i = value.length; i < maxLength; i++) {
data.writeByte((byte)' ');
}
SimpleTextUtil.writeNewline(data);
valuesSeen++;
assert valuesSeen <= valueCount;
}
assert valuesSeen == valueCount;
for(Number ord : docToOrd) {
SimpleTextUtil.write(data, ordEncoder.format(ord.intValue()), scratch);
SimpleTextUtil.writeNewline(data);
}
}
/** write the header for this field */
private void writeFieldEntry(FieldInfo field, FieldInfo.DocValuesType type) throws IOException {
SimpleTextUtil.write(data, FIELD);
SimpleTextUtil.write(data, field.name, scratch);
SimpleTextUtil.writeNewline(data);
SimpleTextUtil.write(data, TYPE);
SimpleTextUtil.write(data, type.toString(), scratch);
SimpleTextUtil.writeNewline(data);
}
@Override
public void close() throws IOException {
boolean success = false;
try {
assert !fieldsSeen.isEmpty();
// TODO: sheisty to do this here?
SimpleTextUtil.write(data, END);
SimpleTextUtil.writeNewline(data);
success = true;
} finally {
if (success) {
IOUtils.close(data);
} else {
IOUtils.closeWhileHandlingException(data);
}
}
}
}

View File

@ -25,10 +25,10 @@ import java.util.Map;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@ -97,12 +97,12 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, NORMS_TYPE);
String nrmType = readString(NORMS_TYPE.length, scratch);
final DocValues.Type normsType = docValuesType(nrmType);
final DocValuesType normsType = docValuesType(nrmType);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, DOCVALUES);
String dvType = readString(DOCVALUES.length, scratch);
final DocValues.Type docValuesType = docValuesType(dvType);
final DocValuesType docValuesType = docValuesType(dvType);
SimpleTextUtil.readLine(input, scratch);
assert StringHelper.startsWith(scratch, NUM_ATTS);
@ -140,11 +140,11 @@ public class SimpleTextFieldInfosReader extends FieldInfosReader {
}
}
public DocValues.Type docValuesType(String dvType) {
public DocValuesType docValuesType(String dvType) {
if ("false".equals(dvType)) {
return null;
} else {
return DocValues.Type.valueOf(dvType);
return DocValuesType.valueOf(dvType);
}
}

View File

@ -20,8 +20,8 @@ import java.io.IOException;
import java.util.Map;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.FieldInfo.IndexOptions;
@ -137,7 +137,7 @@ public class SimpleTextFieldInfosWriter extends FieldInfosWriter {
}
}
private static String getDocValuesType(DocValues.Type type) {
private static String getDocValuesType(DocValuesType type) {
return type == null ? "false" : type.toString();
}
}

View File

@ -68,7 +68,7 @@ class SimpleTextFieldsReader extends FieldsProducer {
public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
fieldInfos = state.fieldInfos;
in = state.dir.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix), state.context);
in = state.directory.openInput(SimpleTextPostingsFormat.getPostingsFileName(state.segmentInfo.name, state.segmentSuffix), state.context);
boolean success = false;
try {
fields = readFields(in.clone());

View File

@ -18,19 +18,12 @@ package org.apache.lucene.codecs.simpletext;
*/
import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.index.SegmentWriteState;
/**
* plain-text norms format.
@ -40,17 +33,16 @@ import org.apache.lucene.util.BytesRef;
* @lucene.experimental
*/
public class SimpleTextNormsFormat extends NormsFormat {
private static final String NORMS_SEG_SUFFIX = "len";
private static final String NORMS_SEG_EXTENSION = "len";
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new SimpleTextNormsPerDocConsumer(state);
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
return new SimpleTextNormsConsumer(state);
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new SimpleTextNormsPerDocProducer(state,
BytesRef.getUTF8SortedAsUnicodeComparator());
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
return new SimpleTextNormsProducer(state);
}
/**
@ -60,29 +52,12 @@ public class SimpleTextNormsFormat extends NormsFormat {
*
* @lucene.experimental
*/
public static class SimpleTextNormsPerDocProducer extends
SimpleTextPerDocProducer {
public SimpleTextNormsPerDocProducer(SegmentReadState state,
Comparator<BytesRef> comp) throws IOException {
super(state, comp, NORMS_SEG_SUFFIX);
public static class SimpleTextNormsProducer extends SimpleTextDocValuesReader {
public SimpleTextNormsProducer(SegmentReadState state) throws IOException {
// All we do is change the extension from .dat -> .len;
// otherwise this is a normal simple doc values file:
super(state, NORMS_SEG_EXTENSION);
}
@Override
protected boolean canLoad(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
@Override
protected boolean anyDocValuesFields(FieldInfos infos) {
return infos.hasNorms();
}
}
/**
@ -92,33 +67,11 @@ public class SimpleTextNormsFormat extends NormsFormat {
*
* @lucene.experimental
*/
public static class SimpleTextNormsPerDocConsumer extends
SimpleTextPerDocConsumer {
public SimpleTextNormsPerDocConsumer(PerDocWriteState state) {
super(state, NORMS_SEG_SUFFIX);
}
@Override
protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
throws IOException {
return reader.normValues(info.name);
}
@Override
protected boolean canMerge(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
@Override
public void abort() {
// We don't have to remove files here: IndexFileDeleter
// will do so
public static class SimpleTextNormsConsumer extends SimpleTextDocValuesWriter {
public SimpleTextNormsConsumer(SegmentWriteState state) throws IOException {
// All we do is change the extension from .dat -> .len;
// otherwise this is a normal simple doc values file:
super(state, NORMS_SEG_EXTENSION);
}
}
}

View File

@ -1,61 +0,0 @@
package org.apache.lucene.codecs.simpletext;
/*
* 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.DocValuesConsumer;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.DocValues.Type;
/**
* @lucene.experimental
*/
class SimpleTextPerDocConsumer extends PerDocConsumer {
protected final PerDocWriteState state;
protected final String segmentSuffix;
public SimpleTextPerDocConsumer(PerDocWriteState state, String segmentSuffix) {
this.state = state;
this.segmentSuffix = segmentSuffix;
}
@Override
public void close() throws IOException {
}
@Override
public DocValuesConsumer addValuesField(Type type, FieldInfo field)
throws IOException {
return new SimpleTextDocValuesConsumer(SimpleTextDocValuesFormat.docValuesId(state.segmentInfo.name,
field.number), state.directory, state.context, type, segmentSuffix);
}
@Override
public void abort() {
// We don't have to remove files here: IndexFileDeleter
// will do so
}
static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId;
}
}

View File

@ -1,447 +0,0 @@
package org.apache.lucene.codecs.simpletext;
/*
* 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.codecs.simpletext.SimpleTextDocValuesConsumer.DOC;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.END;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.HEADER;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.VALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.VALUE_SIZE;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Comparator;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.codecs.DocValuesArraySource;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
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.BytesRefHash;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.packed.PackedInts.Reader;
/**
* Reads plain-text DocValues.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
*/
public class SimpleTextPerDocProducer extends PerDocProducerBase {
protected final TreeMap<String, DocValues> docValues;
private Comparator<BytesRef> comp;
private final String segmentSuffix;
/**
* Creates a new {@link SimpleTextPerDocProducer} instance and loads all
* {@link DocValues} instances for this segment and codec.
*/
public SimpleTextPerDocProducer(SegmentReadState state,
Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
this.comp = comp;
this.segmentSuffix = segmentSuffix;
if (anyDocValuesFields(state.fieldInfos)) {
docValues = load(state.fieldInfos, state.segmentInfo.name,
state.segmentInfo.getDocCount(), state.dir, state.context);
} else {
docValues = new TreeMap<String, DocValues>();
}
}
@Override
protected Map<String, DocValues> docValues() {
return docValues;
}
@Override
protected DocValues loadDocValues(int docCount, Directory dir, String id,
DocValues.Type type, IOContext context) throws IOException {
return new SimpleTextDocValues(dir, context, type, id, docCount, comp, segmentSuffix);
}
@Override
protected void closeInternal(Collection<? extends Closeable> closeables)
throws IOException {
IOUtils.close(closeables);
}
private static class SimpleTextDocValues extends DocValues {
private int docCount;
@Override
public void close() throws IOException {
boolean success = false;
try {
super.close();
success = true;
} finally {
if (success) {
IOUtils.close(input);
} else {
IOUtils.closeWhileHandlingException(input);
}
}
}
private Type type;
private Comparator<BytesRef> comp;
private int valueSize;
private final IndexInput input;
public SimpleTextDocValues(Directory dir, IOContext ctx, Type type,
String id, int docCount, Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
this.type = type;
this.docCount = docCount;
this.comp = comp;
final String fileName = IndexFileNames.segmentFileName(id, "", segmentSuffix);
boolean success = false;
IndexInput in = null;
try {
in = dir.openInput(fileName, ctx);
valueSize = readHeader(in);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(in);
}
}
input = in;
}
@Override
protected Source loadSource() throws IOException {
boolean success = false;
IndexInput in = input.clone();
try {
Source source = null;
switch (type) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
source = read(in, new ValueReader(type, docCount, comp));
break;
case FIXED_INTS_16:
case FIXED_INTS_32:
case VAR_INTS:
case FIXED_INTS_64:
case FIXED_INTS_8:
case FLOAT_32:
case FLOAT_64:
source = read(in, new ValueReader(type, docCount, null));
break;
default:
throw new IllegalArgumentException("unknown type: " + type);
}
assert source != null;
success = true;
return source;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(in);
} else {
IOUtils.close(in);
}
}
}
private int readHeader(IndexInput in) throws IOException {
BytesRef scratch = new BytesRef();
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, HEADER);
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, VALUE_SIZE);
return Integer.parseInt(readString(scratch.offset + VALUE_SIZE.length,
scratch));
}
private Source read(IndexInput in, ValueReader reader) throws IOException {
BytesRef scratch = new BytesRef();
for (int i = 0; i < docCount; i++) {
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, DOC) : scratch.utf8ToString();
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, VALUE);
reader.fromString(i, scratch, scratch.offset + VALUE.length);
}
SimpleTextUtil.readLine(in, scratch);
assert scratch.equals(END);
return reader.getSource();
}
@Override
public Source getDirectSource() throws IOException {
return this.getSource(); // don't cache twice
}
@Override
protected Source loadDirectSource() throws IOException {
return this.getSource();
}
@Override
public int getValueSize() {
return valueSize;
}
@Override
public Type getType() {
return type;
}
}
public static String readString(int offset, BytesRef scratch) {
return new String(scratch.bytes, scratch.offset + offset, scratch.length
- offset, IOUtils.CHARSET_UTF_8);
}
private static final class ValueReader {
private final Type type;
private byte[] bytes;
private short[] shorts;
private int[] ints;
private long[] longs;
private float[] floats;
private double[] doubles;
private Source source;
private BytesRefHash hash;
private BytesRef scratch;
public ValueReader(Type type, int maxDocs, Comparator<BytesRef> comp) {
super();
this.type = type;
Source docValuesArray = null;
switch (type) {
case FIXED_INTS_16:
shorts = new short[maxDocs];
docValuesArray = DocValuesArraySource.forType(type)
.newFromArray(shorts);
break;
case FIXED_INTS_32:
ints = new int[maxDocs];
docValuesArray = DocValuesArraySource.forType(type).newFromArray(ints);
break;
case FIXED_INTS_64:
longs = new long[maxDocs];
docValuesArray = DocValuesArraySource.forType(type)
.newFromArray(longs);
break;
case VAR_INTS:
longs = new long[maxDocs];
docValuesArray = new VarIntsArraySource(type, longs);
break;
case FIXED_INTS_8:
bytes = new byte[maxDocs];
docValuesArray = DocValuesArraySource.forType(type).newFromArray(bytes);
break;
case FLOAT_32:
floats = new float[maxDocs];
docValuesArray = DocValuesArraySource.forType(type)
.newFromArray(floats);
break;
case FLOAT_64:
doubles = new double[maxDocs];
docValuesArray = DocValuesArraySource.forType(type).newFromArray(
doubles);
break;
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
assert comp != null;
hash = new BytesRefHash();
BytesSource bytesSource = new BytesSource(type, comp, maxDocs, hash);
ints = bytesSource.docIdToEntry;
source = bytesSource;
scratch = new BytesRef();
break;
}
if (docValuesArray != null) {
assert source == null;
this.source = docValuesArray;
}
}
public void fromString(int ord, BytesRef ref, int offset) {
switch (type) {
case FIXED_INTS_16:
assert shorts != null;
shorts[ord] = Short.parseShort(readString(offset, ref));
break;
case FIXED_INTS_32:
assert ints != null;
ints[ord] = Integer.parseInt(readString(offset, ref));
break;
case FIXED_INTS_64:
case VAR_INTS:
assert longs != null;
longs[ord] = Long.parseLong(readString(offset, ref));
break;
case FIXED_INTS_8:
assert bytes != null;
bytes[ord] = (byte) Integer.parseInt(readString(offset, ref));
break;
case FLOAT_32:
assert floats != null;
floats[ord] = Float.parseFloat(readString(offset, ref));
break;
case FLOAT_64:
assert doubles != null;
doubles[ord] = Double.parseDouble(readString(offset, ref));
break;
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
scratch.bytes = ref.bytes;
scratch.length = ref.length - offset;
scratch.offset = ref.offset + offset;
int key = hash.add(scratch);
ints[ord] = key < 0 ? (-key) - 1 : key;
break;
}
}
public Source getSource() {
if (source instanceof BytesSource) {
((BytesSource) source).maybeSort();
}
return source;
}
}
private static final class BytesSource extends SortedSource {
private final BytesRefHash hash;
int[] docIdToEntry;
int[] sortedEntries;
int[] adresses;
private final boolean isSorted;
protected BytesSource(Type type, Comparator<BytesRef> comp, int maxDoc,
BytesRefHash hash) {
super(type, comp);
docIdToEntry = new int[maxDoc];
this.hash = hash;
isSorted = type == Type.BYTES_FIXED_SORTED
|| type == Type.BYTES_VAR_SORTED;
}
void maybeSort() {
if (isSorted) {
adresses = new int[hash.size()];
sortedEntries = hash.sort(getComparator());
for (int i = 0; i < adresses.length; i++) {
int entry = sortedEntries[i];
adresses[entry] = i;
}
}
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
if (isSorted) {
return hash.get(sortedEntries[ord(docID)], ref);
} else {
return hash.get(docIdToEntry[docID], ref);
}
}
@Override
public SortedSource asSortedSource() {
if (isSorted) {
return this;
}
return null;
}
@Override
public int ord(int docID) {
assert isSorted;
try {
return adresses[docIdToEntry[docID]];
} catch (Exception e) {
return 0;
}
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
assert isSorted;
return hash.get(sortedEntries[ord], bytesRef);
}
@Override
public Reader getDocToOrd() {
return null;
}
@Override
public int getValueCount() {
return hash.size();
}
}
private static class VarIntsArraySource extends Source {
private final long[] array;
protected VarIntsArraySource(Type type, long[] array) {
super(type);
this.array = array;
}
@Override
public long getInt(int docID) {
return array[docID];
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
DocValuesArraySource.copyLong(ref, getInt(docID));
return ref;
}
}
}

View File

@ -0,0 +1,17 @@
# 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.
org.apache.lucene.codecs.diskdv.DiskDocValuesFormat
org.apache.lucene.codecs.simpletext.SimpleTextDocValuesFormat

View File

@ -0,0 +1,34 @@
package org.apache.lucene.codecs.diskdv;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseDocValuesFormatTestCase;
import org.apache.lucene.util._TestUtil;
/**
* Tests DiskDocValuesFormat
*/
public class TestDiskDocValuesFormat extends BaseDocValuesFormatTestCase {
private final Codec codec = _TestUtil.alwaysDocValuesFormat(new DiskDocValuesFormat());
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -0,0 +1,33 @@
package org.apache.lucene.codecs.simpletext;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.index.BaseDocValuesFormatTestCase;
/**
* Tests SimpleTextDocValuesFormat
*/
public class TestSimpleTextDocValuesFormat extends BaseDocValuesFormatTestCase {
private final Codec codec = new SimpleTextCodec();
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -88,6 +88,7 @@
<property name="tests.multiplier" value="1" />
<property name="tests.codec" value="random" />
<property name="tests.postingsformat" value="random" />
<property name="tests.docvaluesformat" value="random" />
<property name="tests.locale" value="random" />
<property name="tests.timezone" value="random" />
<property name="tests.directory" value="random" />
@ -864,6 +865,8 @@
<sysproperty key="tests.codec" value="${tests.codec}"/>
<!-- set the postingsformat tests should run with -->
<sysproperty key="tests.postingsformat" value="${tests.postingsformat}"/>
<!-- set the docvaluesformat tests should run with -->
<sysproperty key="tests.docvaluesformat" value="${tests.docvaluesformat}"/>
<!-- set the locale tests should run with -->
<sysproperty key="tests.locale" value="${tests.locale}"/>
<!-- set the timezone tests should run with -->

View File

@ -64,7 +64,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
/** Encodes/decodes postings */
public abstract PostingsFormat postingsFormat();
/** Encodes/decodes docvalues */
public abstract DocValuesFormat docValuesFormat();
@ -82,7 +82,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
/** Encodes/decodes document normalization values */
public abstract NormsFormat normsFormat();
/** Encodes/decodes live docs */
public abstract LiveDocsFormat liveDocsFormat();

View File

@ -1,545 +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.Collections;
import java.util.EnumMap;
import java.util.Map;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
/**
* DocValues {@link Source} implementation backed by
* simple arrays.
*
* @lucene.experimental
* @lucene.internal
*/
public abstract class DocValuesArraySource extends Source {
private static final Map<Type, DocValuesArraySource> TEMPLATES;
static {
EnumMap<Type, DocValuesArraySource> templates = new EnumMap<Type, DocValuesArraySource>(
Type.class);
templates.put(Type.FIXED_INTS_16, new ShortValues());
templates.put(Type.FIXED_INTS_32, new IntValues());
templates.put(Type.FIXED_INTS_64, new LongValues());
templates.put(Type.FIXED_INTS_8, new ByteValues());
templates.put(Type.FLOAT_32, new FloatValues());
templates.put(Type.FLOAT_64, new DoubleValues());
TEMPLATES = Collections.unmodifiableMap(templates);
}
/** Returns the {@link DocValuesArraySource} for the given
* {@link Type}. */
public static DocValuesArraySource forType(Type type) {
return TEMPLATES.get(type);
}
/** Number of bytes to encode each doc value. */
protected final int bytesPerValue;
DocValuesArraySource(int bytesPerValue, Type type) {
super(type);
this.bytesPerValue = bytesPerValue;
}
@Override
public abstract BytesRef getBytes(int docID, BytesRef ref);
/** Creates a {@link DocValuesArraySource} by loading a
* previously saved one from an {@link IndexInput}. */
public abstract DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException;
/** Creates {@link DocValuesArraySource} from a native
* array. */
public abstract DocValuesArraySource newFromArray(Object array);
@Override
public final boolean hasArray() {
return true;
}
/** Encode a long value into the provided {@link
* BytesRef}. */
public void toBytes(long value, BytesRef bytesRef) {
copyLong(bytesRef, value);
}
/** Encode a double value into the provided {@link
* BytesRef}. */
public void toBytes(double value, BytesRef bytesRef) {
copyLong(bytesRef, Double.doubleToRawLongBits(value));
}
final static class ByteValues extends DocValuesArraySource {
private final byte[] values;
ByteValues() {
super(1, Type.FIXED_INTS_8);
values = new byte[0];
}
private ByteValues(byte[] array) {
super(1, Type.FIXED_INTS_8);
values = array;
}
private ByteValues(IndexInput input, int numDocs) throws IOException {
super(1, Type.FIXED_INTS_8);
values = new byte[numDocs];
input.readBytes(values, 0, values.length, false);
}
@Override
public byte[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
return getInt(docID);
}
@Override
public long getInt(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new ByteValues(input, numDocs);
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof byte[];
return new ByteValues((byte[]) array);
}
@Override
public void toBytes(long value, BytesRef bytesRef) {
if (bytesRef.bytes.length == 0) {
bytesRef.bytes = new byte[1];
}
bytesRef.bytes[0] = (byte) (0xFFL & value);
bytesRef.offset = 0;
bytesRef.length = 1;
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getInt(docID), ref);
return ref;
}
};
final static class ShortValues extends DocValuesArraySource {
private final short[] values;
ShortValues() {
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
values = new short[0];
}
private ShortValues(short[] array) {
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
values = array;
}
private ShortValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
values = new short[numDocs];
for (int i = 0; i < values.length; i++) {
values[i] = input.readShort();
}
}
@Override
public short[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
return getInt(docID);
}
@Override
public long getInt(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new ShortValues(input, numDocs);
}
@Override
public void toBytes(long value, BytesRef bytesRef) {
copyShort(bytesRef, (short) (0xFFFFL & value));
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof short[];
return new ShortValues((short[]) array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getInt(docID), ref);
return ref;
}
};
final static class IntValues extends DocValuesArraySource {
private final int[] values;
IntValues() {
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
values = new int[0];
}
private IntValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
values = new int[numDocs];
for (int i = 0; i < values.length; i++) {
values[i] = input.readInt();
}
}
private IntValues(int[] array) {
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
values = array;
}
@Override
public int[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
return getInt(docID);
}
@Override
public long getInt(int docID) {
assert docID >= 0 && docID < values.length;
return 0xFFFFFFFF & values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new IntValues(input, numDocs);
}
@Override
public void toBytes(long value, BytesRef bytesRef) {
copyInt(bytesRef, (int) (0xFFFFFFFF & value));
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof int[];
return new IntValues((int[]) array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getInt(docID), ref);
return ref;
}
};
final static class LongValues extends DocValuesArraySource {
private final long[] values;
LongValues() {
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
values = new long[0];
}
private LongValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
values = new long[numDocs];
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
}
private LongValues(long[] array) {
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
values = array;
}
@Override
public long[] getArray() {
return values;
}
@Override
public long getInt(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new LongValues(input, numDocs);
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof long[];
return new LongValues((long[])array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getInt(docID), ref);
return ref;
}
};
final static class FloatValues extends DocValuesArraySource {
private final float[] values;
FloatValues() {
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
values = new float[0];
}
private FloatValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
values = new float[numDocs];
/*
* we always read BIG_ENDIAN here since the writer serialized plain bytes
* we can simply read the ints / longs back in using readInt / readLong
*/
for (int i = 0; i < values.length; i++) {
values[i] = Float.intBitsToFloat(input.readInt());
}
}
private FloatValues(float[] array) {
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
values = array;
}
@Override
public float[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public void toBytes(double value, BytesRef bytesRef) {
copyInt(bytesRef, Float.floatToRawIntBits((float)value));
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new FloatValues(input, numDocs);
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof float[];
return new FloatValues((float[]) array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getFloat(docID), ref);
return ref;
}
};
final static class DoubleValues extends DocValuesArraySource {
private final double[] values;
DoubleValues() {
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
values = new double[0];
}
private DoubleValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
values = new double[numDocs];
/*
* we always read BIG_ENDIAN here since the writer serialized plain bytes
* we can simply read the ints / longs back in using readInt / readLong
*/
for (int i = 0; i < values.length; i++) {
values[i] = Double.longBitsToDouble(input.readLong());
}
}
private DoubleValues(double[] array) {
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
values = array;
}
@Override
public double[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new DoubleValues(input, numDocs);
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof double[];
return new DoubleValues((double[]) array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getFloat(docID), ref);
return ref;
}
};
/**
* Copies the given long value and encodes it as 8 byte Big-Endian.
* <p>
* NOTE: this method resets the offset to 0, length to 8 and resizes the
* reference array if needed.
*/
public static void copyLong(BytesRef ref, long value) {
if (ref.bytes.length < 8) {
ref.bytes = new byte[8];
}
copyInternal(ref, (int) (value >> 32), ref.offset = 0);
copyInternal(ref, (int) value, 4);
ref.length = 8;
}
/**
* Copies the given int value and encodes it as 4 byte Big-Endian.
* <p>
* NOTE: this method resets the offset to 0, length to 4 and resizes the
* reference array if needed.
*/
public static void copyInt(BytesRef ref, int value) {
if (ref.bytes.length < 4) {
ref.bytes = new byte[4];
}
copyInternal(ref, value, ref.offset = 0);
ref.length = 4;
}
/**
* Copies the given short value and encodes it as a 2 byte Big-Endian.
* <p>
* NOTE: this method resets the offset to 0, length to 2 and resizes the
* reference array if needed.
*/
public static void copyShort(BytesRef ref, short value) {
if (ref.bytes.length < 2) {
ref.bytes = new byte[2];
}
ref.offset = 0;
ref.bytes[ref.offset] = (byte) (value >> 8);
ref.bytes[ref.offset + 1] = (byte) (value);
ref.length = 2;
}
private static void copyInternal(BytesRef ref, int value, int startOffset) {
ref.bytes[startOffset] = (byte) (value >> 24);
ref.bytes[startOffset + 1] = (byte) (value >> 16);
ref.bytes[startOffset + 2] = (byte) (value >> 8);
ref.bytes[startOffset + 3] = (byte) (value);
}
/**
* Converts 2 consecutive bytes from the current offset to a short. Bytes are
* interpreted as Big-Endian (most significant bit first)
* <p>
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
*/
public static short asShort(BytesRef b) {
return (short) (0xFFFF & ((b.bytes[b.offset] & 0xFF) << 8) | (b.bytes[b.offset + 1] & 0xFF));
}
/**
* Converts 4 consecutive bytes from the current offset to an int. Bytes are
* interpreted as Big-Endian (most significant bit first)
* <p>
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
*/
public static int asInt(BytesRef b) {
return asIntInternal(b, b.offset);
}
/**
* Converts 8 consecutive bytes from the current offset to a long. Bytes are
* interpreted as Big-Endian (most significant bit first)
* <p>
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
*/
public static long asLong(BytesRef b) {
return (((long) asIntInternal(b, b.offset) << 32) | asIntInternal(b,
b.offset + 4) & 0xFFFFFFFFL);
}
private static int asIntInternal(BytesRef b, int pos) {
return ((b.bytes[pos++] & 0xFF) << 24) | ((b.bytes[pos++] & 0xFF) << 16)
| ((b.bytes[pos++] & 0xFF) << 8) | (b.bytes[pos] & 0xFF);
}
}

View File

@ -16,235 +16,478 @@ package org.apache.lucene.codecs;
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.document.ByteDocValuesField;
import org.apache.lucene.document.DerefBytesDocValuesField;
import org.apache.lucene.document.DoubleDocValuesField;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FloatDocValuesField;
import org.apache.lucene.document.IntDocValuesField;
import org.apache.lucene.document.LongDocValuesField;
import org.apache.lucene.document.PackedLongDocValuesField;
import org.apache.lucene.document.ShortDocValuesField;
import org.apache.lucene.document.SortedBytesDocValuesField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StraightBytesDocValuesField;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.NoSuchElementException;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentWriteState;
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;
import org.apache.lucene.util.packed.AppendingLongBuffer;
/**
* Abstract API that consumes {@link StorableField}s.
* {@link DocValuesConsumer} are always associated with a specific field and
* segments. Concrete implementations of this API write the given
* {@link StorableField} into a implementation specific format depending on
* the fields meta-data.
*
/**
* Abstract API that consumes numeric, binary and
* sorted docvalues. Concrete implementations of this
* actually do "something" with the docvalues (write it into
* the index in a specific format).
* <p>
* The lifecycle is:
* <ol>
* <li>DocValuesConsumer is created by
* {@link DocValuesFormat#fieldsConsumer(SegmentWriteState)} or
* {@link NormsFormat#normsConsumer(SegmentWriteState)}.
* <li>{@link #addNumericField}, {@link #addBinaryField},
* or {@link #addSortedField} are called for each Numeric,
* Binary, or Sorted docvalues field. The API is a "pull" rather
* than "push", and the implementation is free to iterate over the
* values multiple times ({@link Iterable#iterator()}).
* <li>After all fields are added, the consumer is {@link #close}d.
* </ol>
*
* @lucene.experimental
*/
public abstract class DocValuesConsumer {
/** Spare {@link BytesRef} that subclasses can reuse. */
protected final BytesRef spare = new BytesRef();
/** Returns the {@link Type} of this consumer. */
protected abstract Type getType();
public abstract class DocValuesConsumer implements Closeable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected DocValuesConsumer() {
protected DocValuesConsumer() {}
/**
* Writes numeric docvalues for a field.
* @param field field information
* @param values Iterable of numeric values (one for each document).
* @throws IOException if an I/O error occurred.
*/
public abstract void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException;
/**
* Writes binary docvalues for a field.
* @param field field information
* @param values Iterable of binary values (one for each document).
* @throws IOException if an I/O error occurred.
*/
public abstract void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException;
/**
* Writes pre-sorted binary docvalues for a field.
* @param field field information
* @param values Iterable of binary values in sorted order (deduplicated).
* @param docToOrd Iterable of ordinals (one for each document).
* @throws IOException if an I/O error occurred.
*/
public abstract void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException;
/**
* Merges the numeric docvalues from <code>toMerge</code>.
* <p>
* The default implementation calls {@link #addNumericField}, passing
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeNumericField(FieldInfo fieldInfo, final MergeState mergeState, final List<NumericDocValues> toMerge) throws IOException {
addNumericField(fieldInfo,
new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return new Iterator<Number>() {
int readerUpto = -1;
int docIDUpto;
long nextValue;
AtomicReader currentReader;
NumericDocValues currentValues;
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;
// TODO: make a mutable number
return nextValue;
}
private boolean setNext() {
while (true) {
if (readerUpto == toMerge.size()) {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentReader = mergeState.readers.get(readerUpto);
currentValues = toMerge.get(readerUpto);
currentLiveDocs = currentReader.getLiveDocs();
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
nextValue = currentValues.get(docIDUpto);
docIDUpto++;
return true;
}
docIDUpto++;
}
}
};
}
});
}
/**
* Merges the binary docvalues from <code>toMerge</code>.
* <p>
* The default implementation calls {@link #addBinaryField}, passing
* an Iterable that merges and filters deleted documents on the fly.
*/
public void mergeBinaryField(FieldInfo fieldInfo, final MergeState mergeState, final List<BinaryDocValues> toMerge) throws IOException {
addBinaryField(fieldInfo,
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new Iterator<BytesRef>() {
int readerUpto = -1;
int docIDUpto;
BytesRef nextValue = new BytesRef();
AtomicReader currentReader;
BinaryDocValues currentValues;
Bits currentLiveDocs;
boolean nextIsSet;
@Override
public boolean hasNext() {
return nextIsSet || setNext();
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
assert nextIsSet;
nextIsSet = false;
// TODO: make a mutable number
return nextValue;
}
private boolean setNext() {
while (true) {
if (readerUpto == toMerge.size()) {
return false;
}
if (currentReader == null || docIDUpto == currentReader.maxDoc()) {
readerUpto++;
if (readerUpto < toMerge.size()) {
currentReader = mergeState.readers.get(readerUpto);
currentValues = toMerge.get(readerUpto);
currentLiveDocs = currentReader.getLiveDocs();
}
docIDUpto = 0;
continue;
}
if (currentLiveDocs == null || currentLiveDocs.get(docIDUpto)) {
nextIsSet = true;
currentValues.get(docIDUpto, nextValue);
docIDUpto++;
return true;
}
docIDUpto++;
}
}
};
}
});
}
/**
* Adds the given {@link StorableField} instance to this
* {@link DocValuesConsumer}
*
* @param docID
* the document ID to add the value for. The docID must always
* increase or be <tt>0</tt> if it is the first call to this method.
* @param value
* the value to add
* @throws IOException
* if an {@link IOException} occurs
*/
public abstract void add(int docID, StorableField value)
throws IOException;
static class SortedBytesMerger {
/**
* Called when the consumer of this API is done adding values.
*
* @param docCount
* the total number of documents in this {@link DocValuesConsumer}.
* Must be greater than or equal the last given docID to
* {@link #add(int, StorableField)}.
* @throws IOException If an I/O error occurs
*/
public abstract void finish(int docCount) throws IOException;
/**
* Returns the value size this consumer accepts or <tt>-1</tt> iff this
* consumer is value size agnostic ie. accepts variable length values.
* <p>
* NOTE: the return value is undefined until the consumer has successfully
* consumed at least one value.
*
* @return the value size this consumer accepts or <tt>-1</tt> iff this
* consumer is value size agnostic ie. accepts variable length values.
*/
public abstract int getValueSize();
/**
* Merges the given {@link org.apache.lucene.index.MergeState} into
* this {@link DocValuesConsumer}.
*
* @param mergeState
* the state to merge
* @param docValues docValues array containing one instance per reader (
* {@link org.apache.lucene.index.MergeState#readers}) or <code>null</code> if the reader has
* no {@link DocValues} instance.
* @throws IOException
* if an {@link IOException} occurs
*/
public void merge(MergeState mergeState, DocValues[] docValues) throws IOException {
assert mergeState != null;
boolean hasMerged = false;
for(int readerIDX=0;readerIDX<mergeState.readers.size();readerIDX++) {
final AtomicReader reader = mergeState.readers.get(readerIDX);
if (docValues[readerIDX] != null) {
hasMerged = true;
merge(docValues[readerIDX], mergeState.docBase[readerIDX],
reader.maxDoc(), reader.getLiveDocs());
mergeState.checkAbort.work(reader.maxDoc());
public int numMergedTerms;
final AppendingLongBuffer ordToReaderId = new AppendingLongBuffer();
final List<SegmentState> segStates = new ArrayList<SegmentState>();
private static class SegmentState {
int segmentID;
AtomicReader reader;
FixedBitSet liveTerms;
int ord = -1;
SortedDocValues values;
BytesRef scratch = new BytesRef();
AppendingLongBuffer ordDeltas = new AppendingLongBuffer();
// TODO: use another scheme?
// currently we +/- delta merged-ord from segment-ord (is this good? makes sense to me?)
// but we have a good idea "roughly" what
// the ord should be (linear projection) so we only
// need to encode the delta from that ...:
AppendingLongBuffer segOrdToMergedOrd = new AppendingLongBuffer();
public BytesRef nextTerm() {
while (ord < values.getValueCount()-1) {
ord++;
if (liveTerms == null || liveTerms.get(ord)) {
values.lookupOrd(ord, scratch);
return scratch;
}
}
return null;
}
}
// only finish if no exception is thrown!
if (hasMerged) {
finish(mergeState.segmentInfo.getDocCount());
}
}
/**
* Merges the given {@link DocValues} into this {@link DocValuesConsumer}.
*
* @throws IOException
* if an {@link IOException} occurs
*/
protected void merge(DocValues reader, int docBase, int docCount, Bits liveDocs) throws IOException {
// This enables bulk copies in subclasses per MergeState, subclasses can
// simply override this and decide if they want to merge
// segments using this generic implementation or if a bulk merge is possible
// / feasible.
final Source source = reader.getDirectSource();
assert source != null;
int docID = docBase;
final Type type = getType();
final StoredField scratchField;
switch(type) {
case VAR_INTS:
scratchField = new PackedLongDocValuesField("", (long) 0);
break;
case FIXED_INTS_8:
scratchField = new ByteDocValuesField("", (byte) 0);
break;
case FIXED_INTS_16:
scratchField = new ShortDocValuesField("", (short) 0);
break;
case FIXED_INTS_32:
scratchField = new IntDocValuesField("", 0);
break;
case FIXED_INTS_64:
scratchField = new LongDocValuesField("", (long) 0);
break;
case FLOAT_32:
scratchField = new FloatDocValuesField("", 0f);
break;
case FLOAT_64:
scratchField = new DoubleDocValuesField("", 0d);
break;
case BYTES_FIXED_STRAIGHT:
scratchField = new StraightBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_STRAIGHT:
scratchField = new StraightBytesDocValuesField("", new BytesRef(), false);
break;
case BYTES_FIXED_DEREF:
scratchField = new DerefBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_DEREF:
scratchField = new DerefBytesDocValuesField("", new BytesRef(), false);
break;
case BYTES_FIXED_SORTED:
scratchField = new SortedBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_SORTED:
scratchField = new SortedBytesDocValuesField("", new BytesRef(), false);
break;
default:
throw new IllegalStateException("unknown Type: " + type);
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;
}
}
for (int i = 0; i < docCount; i++) {
if (liveDocs == null || liveDocs.get(i)) {
mergeDoc(scratchField, source, docID++, i);
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);
int maxDoc = reader.maxDoc();
SegmentState state = new SegmentState();
state.segmentID = readerIDX;
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();
assert liveDocs != null;
for(int docID=0;docID<maxDoc;docID++) {
if (liveDocs.get(docID)) {
state.liveTerms.set(state.values.getOrd(docID));
}
}
}
// TODO: we can unload the bits/packed ints to disk to reduce
// transient ram spike... most of these just require iterators
}
// Second pass: merge only the live terms
TermMergeQueue q = new TermMergeQueue(segStates.size());
for(SegmentState segState : segStates) {
if (segState.nextTerm() != null) {
q.add(segState);
}
}
int lastOrds[] = new int[segStates.size()];
BytesRef lastTerm = null;
int ord = 0;
while (q.size() != 0) {
SegmentState top = q.top();
if (lastTerm == null || !lastTerm.equals(top.scratch)) {
// a new unique term: record its segment ID / sourceOrd pair
int readerId = top.segmentID;
ordToReaderId.add(readerId);
int sourceOrd = top.ord;
int delta = sourceOrd - lastOrds[readerId];
lastOrds[readerId] = sourceOrd;
top.ordDeltas.add(delta);
if (lastTerm == null) {
lastTerm = BytesRef.deepCopyOf(top.scratch);
} else {
lastTerm.copyBytes(top.scratch);
}
ord++;
}
long signedDelta = (ord-1) - top.ord; // global ord space - segment ord space
// fill in any holes for unused ords, then finally the value we want (segOrdToMergedOrd[top.ord])
// TODO: is there a better way...
while (top.segOrdToMergedOrd.size() <= top.ord) {
top.segOrdToMergedOrd.add(signedDelta);
}
if (top.nextTerm() == null) {
q.pop();
} else {
q.updateTop();
}
}
numMergedTerms = ord;
// clear our bitsets for GC: we dont need them anymore (e.g. while flushing merged stuff to codec)
for (SegmentState state : segStates) {
state.liveTerms = null;
}
}
}
/**
* Merges a document with the given <code>docID</code>. The methods
* implementation obtains the value for the <i>sourceDoc</i> id from the
* current {@link Source}.
* Merges the sorted docvalues from <code>toMerge</code>.
* <p>
* This method is used during merging to provide implementation agnostic
* default merge implementation.
* </p>
* <p>
* All documents IDs between the given ID and the previously given ID or
* <tt>0</tt> if the method is call the first time are filled with default
* values depending on the implementation. The given document
* ID must always be greater than the previous ID or <tt>0</tt> if called the
* first time.
* The default implementation calls {@link #addSortedField}, passing
* an Iterable that merges ordinals and values and filters deleted documents .
*/
protected void mergeDoc(StoredField scratchField, Source source, int docID, int sourceDoc)
throws IOException {
switch(getType()) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
scratchField.setBytesValue(source.getBytes(sourceDoc, spare));
break;
case FIXED_INTS_8:
scratchField.setByteValue((byte) source.getInt(sourceDoc));
break;
case FIXED_INTS_16:
scratchField.setShortValue((short) source.getInt(sourceDoc));
break;
case FIXED_INTS_32:
scratchField.setIntValue((int) source.getInt(sourceDoc));
break;
case FIXED_INTS_64:
scratchField.setLongValue(source.getInt(sourceDoc));
break;
case VAR_INTS:
scratchField.setLongValue(source.getInt(sourceDoc));
break;
case FLOAT_32:
scratchField.setFloatValue((float) source.getFloat(sourceDoc));
break;
case FLOAT_64:
scratchField.setDoubleValue(source.getFloat(sourceDoc));
break;
}
add(docID, scratchField);
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);
addSortedField(fieldInfo,
// ord -> value
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
// for each next(), tells us what reader to go to
final AppendingLongBuffer.Iterator readerIDs = merger.ordToReaderId.iterator();
// for each next(), gives us the original ord
final AppendingLongBuffer.Iterator ordDeltas[] = new AppendingLongBuffer.Iterator[merger.segStates.size()];
final int lastOrds[] = new int[ordDeltas.length];
for (int i = 0; i < ordDeltas.length; i++) {
ordDeltas[i] = merger.segStates.get(i).ordDeltas.iterator();
}
final BytesRef scratch = new BytesRef();
return new Iterator<BytesRef>() {
int ordUpto;
@Override
public boolean hasNext() {
return ordUpto < merger.numMergedTerms;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
@Override
public BytesRef next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
int readerID = (int) readerIDs.next();
int ord = lastOrds[readerID] + (int) ordDeltas[readerID].next();
merger.segStates.get(readerID).values.lookupOrd(ord, scratch);
lastOrds[readerID] = ord;
ordUpto++;
return scratch;
}
};
}
},
// 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;
// TODO 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 = (int) (segOrd + currentReader.segOrdToMergedOrd.get(segOrd));
docIDUpto++;
return true;
}
docIDUpto++;
}
}
};
}
});
}
}

View File

@ -18,24 +18,106 @@ package org.apache.lucene.codecs;
*/
import java.io.IOException;
import java.util.ServiceLoader;
import java.util.Set;
import org.apache.lucene.index.DocValues; // javadocs
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.NamedSPILoader;
/**
* Encodes/decodes {@link DocValues}
* @lucene.experimental
*/
public abstract class DocValuesFormat {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected DocValuesFormat() {
/**
* Encodes/decodes per-document values.
* <p>
* Note, when extending this class, the name ({@link #getName}) may
* written into the index in certain configurations. In order for the segment
* to be read, the name must resolve to your implementation via {@link #forName(String)}.
* This method uses Java's
* {@link ServiceLoader Service Provider Interface} (SPI) to resolve format names.
* <p>
* If you implement your own format, make sure that it has a no-arg constructor
* so SPI can load it.
* @see ServiceLoader
* @lucene.experimental */
public abstract class DocValuesFormat implements NamedSPILoader.NamedSPI {
private static final NamedSPILoader<DocValuesFormat> loader =
new NamedSPILoader<DocValuesFormat>(DocValuesFormat.class);
/** Unique name that's used to retrieve this format when
* reading the index.
*/
private final String name;
/**
* Creates a new docvalues format.
* <p>
* The provided name will be written into the index segment in some configurations
* (such as when using {@code PerFieldDocValuesFormat}): in such configurations,
* for the segment to be read this class should be registered with Java's
* SPI mechanism (registered in META-INF/ of your jar file, etc).
* @param name must be all ascii alphanumeric, and less than 128 characters in length.
*/
protected DocValuesFormat(String name) {
NamedSPILoader.checkServiceName(name);
this.name = name;
}
/** Consumes (writes) doc values during indexing. */
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
/** Returns a {@link DocValuesConsumer} to write docvalues to the
* index. */
public abstract DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException;
/** Produces (reads) doc values during reading/searching. */
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
/**
* Returns a {@link DocValuesProducer} to read docvalues from the index.
* <p>
* NOTE: by the time this call returns, it must hold open any files it will
* need to use; else, those files may be deleted. Additionally, required files
* may be deleted during the execution of this call before there is a chance
* to open them. Under these circumstances an IOException should be thrown by
* the implementation. IOExceptions are expected and will automatically cause
* a retry of the segment opening logic with the newly revised segments.
*/
public abstract DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException;
@Override
public final String getName() {
return name;
}
@Override
public String toString() {
return "DocValuesFormat(name=" + name + ")";
}
/** looks up a format by name */
public static DocValuesFormat forName(String name) {
if (loader == null) {
throw new IllegalStateException("You called DocValuesFormat.forName() before all formats could be initialized. "+
"This likely happens if you call it from a DocValuesFormat's ctor.");
}
return loader.lookup(name);
}
/** returns a list of all available format names */
public static Set<String> availableDocValuesFormats() {
if (loader == null) {
throw new IllegalStateException("You called DocValuesFormat.availableDocValuesFormats() before all formats could be initialized. "+
"This likely happens if you call it from a DocValuesFormat's ctor.");
}
return loader.availableServices();
}
/**
* Reloads the DocValues format list from the given {@link ClassLoader}.
* Changes to the docvalues formats are visible after the method ends, all
* iterators ({@link #availableDocValuesFormats()},...) stay consistent.
*
* <p><b>NOTE:</b> Only new docvalues formats are added, existing ones are
* never removed or replaced.
*
* <p><em>This method is expensive and should only be called for discovery
* of new docvalues formats on the given classpath/classloader!</em>
*/
public static void reloadDocValuesFormats(ClassLoader classloader) {
loader.reload(classloader);
}
}

View File

@ -0,0 +1,53 @@
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.Closeable;
import java.io.IOException;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SortedDocValues;
/** Abstract API that produces numeric, binary and
* sorted docvalues.
*
* @lucene.experimental
*/
public abstract class DocValuesProducer implements Closeable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected DocValuesProducer() {}
/** Returns {@link NumericDocValues} for this field.
* The returned instance need not be thread-safe: it will only be
* used by a single thread. */
public abstract NumericDocValues getNumeric(FieldInfo field) throws IOException;
/** Returns {@link BinaryDocValues} for this field.
* The returned instance need not be thread-safe: it will only be
* used by a single thread. */
public abstract BinaryDocValues getBinary(FieldInfo field) throws IOException;
/** Returns {@link SortedDocValues} for this field.
* The returned instance need not be thread-safe: it will only be
* used by a single thread. */
public abstract SortedDocValues getSorted(FieldInfo field) throws IOException;
}

View File

@ -64,12 +64,12 @@ public abstract class FieldsConsumer implements Closeable {
* implementation to do its own merging. */
public void merge(MergeState mergeState, Fields fields) throws IOException {
for (String field : fields) {
mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
assert mergeState.fieldInfo != null : "FieldInfo for field is null: "+ field;
FieldInfo info = mergeState.fieldInfos.fieldInfo(field);
assert info != null : "FieldInfo for field is null: "+ field;
Terms terms = fields.terms(field);
if (terms != null) {
final TermsConsumer termsConsumer = addField(mergeState.fieldInfo);
termsConsumer.merge(mergeState, terms.iterator(null));
final TermsConsumer termsConsumer = addField(info);
termsConsumer.merge(mergeState, info.getIndexOptions(), terms.iterator(null));
}
}
}

View File

@ -19,11 +19,11 @@ package org.apache.lucene.codecs;
import java.io.IOException;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
/**
* format for normalization factors
* Encodes/decodes per-document score normalization values.
*/
public abstract class NormsFormat {
/** Sole constructor. (For invocation by subclass
@ -31,11 +31,19 @@ public abstract class NormsFormat {
protected NormsFormat() {
}
/** Returns a {@link PerDocConsumer} to write norms to the
/** Returns a {@link DocValuesConsumer} to write norms to the
* index. */
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
public abstract DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException;
/** Returns a {@link PerDocProducer} to read norms from the
* index. */
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
/**
* Returns a {@link DocValuesProducer} to read norms from the index.
* <p>
* NOTE: by the time this call returns, it must hold open any files it will
* need to use; else, those files may be deleted. Additionally, required files
* may be deleted during the execution of this call before there is a chance
* to open them. Under these circumstances an IOException should be thrown by
* the implementation. IOExceptions are expected and will automatically cause
* a retry of the segment opening logic with the newly revised segments.
*/
public abstract DocValuesProducer normsProducer(SegmentReadState state) throws IOException;
}

View File

@ -1,115 +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.Closeable;
import java.io.IOException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.DocValues.Type;
/**
* Abstract API that consumes per document values. Concrete implementations of
* this convert field values into a Codec specific format during indexing.
* <p>
* The {@link PerDocConsumer} API is accessible through the
* {@link PostingsFormat} - API providing per field consumers and producers for inverted
* data (terms, postings) as well as per-document data.
*
* @lucene.experimental
*/
public abstract class PerDocConsumer implements Closeable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected PerDocConsumer() {
}
/** Adds a new DocValuesField */
public abstract DocValuesConsumer addValuesField(DocValues.Type type, FieldInfo field)
throws IOException;
/**
* Consumes and merges the given {@link PerDocProducer} producer
* into this consumers format.
*/
public void merge(MergeState mergeState) throws IOException {
final DocValues[] docValues = new DocValues[mergeState.readers.size()];
for (FieldInfo fieldInfo : mergeState.fieldInfos) {
mergeState.fieldInfo = fieldInfo; // set the field we are merging
if (canMerge(fieldInfo)) {
for (int i = 0; i < docValues.length; i++) {
docValues[i] = getDocValuesForMerge(mergeState.readers.get(i), fieldInfo);
}
Type docValuesType = getDocValuesType(fieldInfo);
assert docValuesType != null;
final DocValuesConsumer docValuesConsumer = addValuesField(docValuesType, fieldInfo);
assert docValuesConsumer != null;
docValuesConsumer.merge(mergeState, docValues);
}
}
}
/**
* Returns a {@link DocValues} instance for merging from the given reader for the given
* {@link FieldInfo}. This method is used for merging and uses
* {@link AtomicReader#docValues(String)} by default.
* <p>
* To enable {@link DocValues} merging for different {@link DocValues} than
* the default override this method accordingly.
* <p>
*/
protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException {
return reader.docValues(info.name);
}
/**
* Returns <code>true</code> iff the given field can be merged ie. has {@link DocValues}.
* By default this method uses {@link FieldInfo#hasDocValues()}.
* <p>
* To enable {@link DocValues} merging for different {@link DocValues} than
* the default override this method accordingly.
* <p>
*/
protected boolean canMerge(FieldInfo info) {
return info.hasDocValues();
}
/**
* Returns the {@link DocValues} {@link Type} for the given {@link FieldInfo}.
* By default this method uses {@link FieldInfo#getDocValuesType()}.
* <p>
* To enable {@link DocValues} merging for different {@link DocValues} than
* the default override this method accordingly.
* <p>
*/
protected Type getDocValuesType(FieldInfo info) {
return info.getDocValuesType();
}
/**
* Called during indexing if the indexing session is aborted due to a unrecoverable exception.
* This method should cleanup all resources.
*/
public abstract void abort();
@Override
public abstract void close() throws IOException;
}

View File

@ -1,54 +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.Closeable;
import java.io.IOException;
import org.apache.lucene.index.DocValues;
/**
* Abstract API that provides access to one or more per-document storage
* features. The concrete implementations provide access to the underlying
* storage on a per-document basis corresponding to their actual
* {@link PerDocConsumer} counterpart.
* <p>
* The {@link PerDocProducer} API is accessible through the
* {@link PostingsFormat} - API providing per field consumers and producers for inverted
* data (terms, postings) as well as per-document data.
*
* @lucene.experimental
*/
public abstract class PerDocProducer implements Closeable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected PerDocProducer() {
}
/**
* Returns {@link DocValues} for the current field.
*
* @param field
* the field name
* @return the {@link DocValues} for this field or <code>null</code> if not
* applicable.
* @throws IOException If an I/O error occurs
*/
public abstract DocValues docValues(String field) throws IOException;
@Override
public abstract void close() throws IOException;
}

View File

@ -1,138 +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.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Comparator;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type; // javadocs
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
/**
* Abstract base class for PerDocProducer implementations
* @lucene.experimental
*/
public abstract class PerDocProducerBase extends PerDocProducer {
/** Closes provided Closables. */
protected abstract void closeInternal(Collection<? extends Closeable> closeables) throws IOException;
/** Returns a map, mapping field names to doc values. */
protected abstract Map<String, DocValues> docValues();
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected PerDocProducerBase() {
}
@Override
public void close() throws IOException {
closeInternal(docValues().values());
}
@Override
public DocValues docValues(String field) throws IOException {
return docValues().get(field);
}
/** Returns the comparator used to sort {@link BytesRef} values. */
public Comparator<BytesRef> getComparator() throws IOException {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
/** Only opens files... doesn't actually load any values. */
protected TreeMap<String, DocValues> load(FieldInfos fieldInfos,
String segment, int docCount, Directory dir, IOContext context)
throws IOException {
TreeMap<String, DocValues> values = new TreeMap<String, DocValues>();
boolean success = false;
try {
for (FieldInfo fieldInfo : fieldInfos) {
if (canLoad(fieldInfo)) {
final String field = fieldInfo.name;
final String id = docValuesId(segment,
fieldInfo.number);
values.put(field,
loadDocValues(docCount, dir, id, getDocValuesType(fieldInfo), context));
}
}
success = true;
} finally {
if (!success) {
// if we fail we must close all opened resources if there are any
try {
closeInternal(values.values());
} catch (Throwable t) {} // keep our original exception
}
}
return values;
}
/** Returns true if this field indexed doc values. */
protected boolean canLoad(FieldInfo info) {
return info.hasDocValues();
}
/** Returns the doc values type for this field. */
protected Type getDocValuesType(FieldInfo info) {
return info.getDocValuesType();
}
/** Returns true if any fields indexed doc values. */
protected boolean anyDocValuesFields(FieldInfos infos) {
return infos.hasDocValues();
}
/** Returns the unique segment and field id for any
* per-field files this implementation needs to write. */
public static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId;
}
/**
* Loads a {@link DocValues} instance depending on the given {@link Type}.
* Codecs that use different implementations for a certain {@link Type} can
* simply override this method and return their custom implementations.
*
* @param docCount
* number of documents in the segment
* @param dir
* the {@link Directory} to load the {@link DocValues} from
* @param id
* the unique file ID within the segment
* @param type
* the type to load
* @return a {@link DocValues} instance for the given type
* @throws IOException
* if an {@link IOException} occurs
* @throws IllegalArgumentException
* if the given {@link Type} is not supported
*/
protected abstract DocValues loadDocValues(int docCount, Directory dir, String id,
DocValues.Type type, IOContext context) throws IOException;
}

View File

@ -73,12 +73,11 @@ public abstract class PostingsConsumer {
/** Default merge impl: append documents, mapping around
* deletes */
public TermStats merge(final MergeState mergeState, final DocsEnum postings, final FixedBitSet visitedDocs) throws IOException {
public TermStats merge(final MergeState mergeState, IndexOptions indexOptions, final DocsEnum postings, final FixedBitSet visitedDocs) throws IOException {
int df = 0;
long totTF = 0;
IndexOptions indexOptions = mergeState.fieldInfo.getIndexOptions();
if (indexOptions == IndexOptions.DOCS_ONLY) {
while(true) {
final int doc = postings.nextDoc();

View File

@ -83,7 +83,7 @@ public abstract class TermsConsumer {
private MappingMultiDocsAndPositionsEnum postingsEnum;
/** Default merge impl */
public void merge(MergeState mergeState, TermsEnum termsEnum) throws IOException {
public void merge(MergeState mergeState, IndexOptions indexOptions, TermsEnum termsEnum) throws IOException {
BytesRef term;
assert termsEnum != null;
@ -92,7 +92,6 @@ public abstract class TermsConsumer {
long sumDFsinceLastAbortCheck = 0;
FixedBitSet visitedDocs = new FixedBitSet(mergeState.segmentInfo.getDocCount());
IndexOptions indexOptions = mergeState.fieldInfo.getIndexOptions();
if (indexOptions == IndexOptions.DOCS_ONLY) {
if (docsEnum == null) {
docsEnum = new MappingMultiDocsEnum();
@ -108,7 +107,7 @@ public abstract class TermsConsumer {
if (docsEnumIn != null) {
docsEnum.reset(docsEnumIn);
final PostingsConsumer postingsConsumer = startTerm(term);
final TermStats stats = postingsConsumer.merge(mergeState, docsEnum, visitedDocs);
final TermStats stats = postingsConsumer.merge(mergeState, indexOptions, docsEnum, visitedDocs);
if (stats.docFreq > 0) {
finishTerm(term, stats);
sumTotalTermFreq += stats.docFreq;
@ -136,7 +135,7 @@ public abstract class TermsConsumer {
assert docsAndFreqsEnumIn != null;
docsAndFreqsEnum.reset(docsAndFreqsEnumIn);
final PostingsConsumer postingsConsumer = startTerm(term);
final TermStats stats = postingsConsumer.merge(mergeState, docsAndFreqsEnum, visitedDocs);
final TermStats stats = postingsConsumer.merge(mergeState, indexOptions, docsAndFreqsEnum, visitedDocs);
if (stats.docFreq > 0) {
finishTerm(term, stats);
sumTotalTermFreq += stats.totalTermFreq;
@ -162,7 +161,7 @@ public abstract class TermsConsumer {
postingsEnum.reset(postingsEnumIn);
final PostingsConsumer postingsConsumer = startTerm(term);
final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs);
final TermStats stats = postingsConsumer.merge(mergeState, indexOptions, postingsEnum, visitedDocs);
if (stats.docFreq > 0) {
finishTerm(term, stats);
sumTotalTermFreq += stats.totalTermFreq;
@ -189,7 +188,7 @@ public abstract class TermsConsumer {
postingsEnum.reset(postingsEnumIn);
final PostingsConsumer postingsConsumer = startTerm(term);
final TermStats stats = postingsConsumer.merge(mergeState, postingsEnum, visitedDocs);
final TermStats stats = postingsConsumer.merge(mergeState, indexOptions, postingsEnum, visitedDocs);
if (stats.docFreq > 0) {
finishTerm(term, stats);
sumTotalTermFreq += stats.totalTermFreq;

View File

@ -57,7 +57,7 @@ import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LongsRef;
import org.apache.lucene.util.packed.BlockPackedReader;
import org.apache.lucene.util.packed.BlockPackedReaderIterator;
import org.apache.lucene.util.packed.PackedInts;
@ -76,7 +76,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
private final int chunkSize;
private final int numDocs;
private boolean closed;
private final BlockPackedReader reader;
private final BlockPackedReaderIterator reader;
// used by clone
private CompressingTermVectorsReader(CompressingTermVectorsReader reader) {
@ -88,7 +88,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
this.decompressor = reader.decompressor.clone();
this.chunkSize = reader.chunkSize;
this.numDocs = reader.numDocs;
this.reader = new BlockPackedReader(vectorsStream, packedIntsVersion, BLOCK_SIZE, 0);
this.reader = new BlockPackedReaderIterator(vectorsStream, packedIntsVersion, BLOCK_SIZE, 0);
this.closed = false;
}
@ -119,7 +119,7 @@ public final class CompressingTermVectorsReader extends TermVectorsReader implem
packedIntsVersion = vectorsStream.readVInt();
chunkSize = vectorsStream.readVInt();
decompressor = compressionMode.newDecompressor();
this.reader = new BlockPackedReader(vectorsStream, packedIntsVersion, BLOCK_SIZE, 0);
this.reader = new BlockPackedReaderIterator(vectorsStream, packedIntsVersion, BLOCK_SIZE, 0);
success = true;
} finally {

View File

@ -18,15 +18,16 @@ package org.apache.lucene.codecs.lucene40;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
@ -42,13 +43,11 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
// if they are backwards compatible or smallish we can probably do the backwards in the postingsreader
// (it writes a minor version, etc).
@Deprecated
public final class Lucene40Codec extends Codec {
public class Lucene40Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@ -73,18 +72,13 @@ public final class Lucene40Codec extends Codec {
return vectorsFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
public FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@ -92,12 +86,21 @@ public final class Lucene40Codec extends Codec {
public final SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
private final DocValuesFormat defaultDVFormat = new Lucene40DocValuesFormat();
@Override
public final NormsFormat normsFormat() {
public DocValuesFormat docValuesFormat() {
return defaultDVFormat;
}
private final NormsFormat normsFormat = new Lucene40NormsFormat();
@Override
public NormsFormat normsFormat() {
return normsFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;

View File

@ -1,86 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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.lucene40.values.DocValuesWriterBase;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.0 PerDocConsumer implementation that uses compound file.
*
* @see Lucene40DocValuesFormat
* @lucene.experimental
*/
public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
private final Directory mainDirectory;
private Directory directory;
private final String segmentSuffix;
/** Segment suffix used when writing doc values index files. */
public final static String DOC_VALUES_SEGMENT_SUFFIX = "dv";
/** Sole constructor. */
public Lucene40DocValuesConsumer(PerDocWriteState state, String segmentSuffix) {
super(state);
this.segmentSuffix = segmentSuffix;
mainDirectory = state.directory;
//TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
}
@Override
protected Directory getDirectory() throws IOException {
// lazy init
if (directory == null) {
directory = new CompoundFileDirectory(mainDirectory,
IndexFileNames.segmentFileName(segmentName, segmentSuffix,
IndexFileNames.COMPOUND_FILE_EXTENSION), context, true);
}
return directory;
}
@Override
public void close() throws IOException {
if (directory != null) {
directory.close();
}
}
@Override
public void abort() {
try {
close();
} catch (Throwable t) {
// ignore
} finally {
// TODO: why the inconsistency here? we do this, but not SimpleText (which says IFD
// will do it).
// TODO: check that IFD really does this always, even if codec abort() throws a
// RuntimeException (e.g. ThreadInterruptedException)
IOUtils.deleteFilesIgnoringExceptions(mainDirectory, IndexFileNames.segmentFileName(
segmentName, segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
IndexFileNames.segmentFileName(segmentName, segmentSuffix,
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
}
}
}

View File

@ -20,16 +20,15 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.DocValues; // javadocs
import org.apache.lucene.index.DocValues.Type; // javadocs
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.CompoundFileDirectory; // javadocs
import org.apache.lucene.store.DataOutput; // javadocs
import org.apache.lucene.util.packed.PackedInts; // javadocs
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.packed.PackedInts;
/**
* Lucene 4.0 DocValues format.
@ -45,7 +44,7 @@ import org.apache.lucene.util.packed.PackedInts; // javadocs
* <li><tt>&lt;segment&gt;_&lt;fieldNumber&gt;.idx</tt>: index into the .dat for DEREF types</li>
* </ul>
* <p>
* There are several many types of {@link DocValues} with different encodings.
* There are several many types of {@code DocValues} with different encodings.
* From the perspective of filenames, all types store their values in <tt>.dat</tt>
* entries within the compound file. In the case of dereferenced/sorted types, the <tt>.dat</tt>
* actually contains only the unique values, and an additional <tt>.idx</tt> file contains
@ -53,42 +52,34 @@ import org.apache.lucene.util.packed.PackedInts; // javadocs
* </p>
* Formats:
* <ul>
* <li>{@link Type#VAR_INTS VAR_INTS} .dat --&gt; Header, PackedType, MinValue,
* <li>{@code VAR_INTS} .dat --&gt; Header, PackedType, MinValue,
* DefaultValue, PackedStream</li>
* <li>{@link Type#FIXED_INTS_8 FIXED_INTS_8} .dat --&gt; Header, ValueSize,
* <li>{@code FIXED_INTS_8} .dat --&gt; Header, ValueSize,
* {@link DataOutput#writeByte Byte}<sup>maxdoc</sup></li>
* <li>{@link Type#FIXED_INTS_16 FIXED_INTS_16} .dat --&gt; Header, ValueSize,
* <li>{@code FIXED_INTS_16} .dat --&gt; Header, ValueSize,
* {@link DataOutput#writeShort Short}<sup>maxdoc</sup></li>
* <li>{@link Type#FIXED_INTS_32 FIXED_INTS_32} .dat --&gt; Header, ValueSize,
* <li>{@code FIXED_INTS_32} .dat --&gt; Header, ValueSize,
* {@link DataOutput#writeInt Int32}<sup>maxdoc</sup></li>
* <li>{@link Type#FIXED_INTS_64 FIXED_INTS_64} .dat --&gt; Header, ValueSize,
* <li>{@code FIXED_INTS_64} .dat --&gt; Header, ValueSize,
* {@link DataOutput#writeLong Int64}<sup>maxdoc</sup></li>
* <li>{@link Type#FLOAT_32 FLOAT_32} .dat --&gt; Header, ValueSize,
* Float32<sup>maxdoc</sup></li>
* <li>{@link Type#FLOAT_64 FLOAT_64} .dat --&gt; Header, ValueSize,
* Float64<sup>maxdoc</sup></li>
* <li>{@link Type#BYTES_FIXED_STRAIGHT BYTES_FIXED_STRAIGHT} .dat --&gt; Header, ValueSize,
* <li>{@code FLOAT_32} .dat --&gt; Header, ValueSize, Float32<sup>maxdoc</sup></li>
* <li>{@code FLOAT_64} .dat --&gt; Header, ValueSize, Float64<sup>maxdoc</sup></li>
* <li>{@code BYTES_FIXED_STRAIGHT} .dat --&gt; Header, ValueSize,
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>maxdoc</sup></li>
* <li>{@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} .idx --&gt; Header, MaxAddress,
* Addresses</li>
* <li>{@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} .dat --&gt; Header, TotalBytes,
* Addresses, ({@link DataOutput#writeByte Byte} *
* <i>variable ValueSize</i>)<sup>maxdoc</sup></li>
* <li>{@link Type#BYTES_FIXED_DEREF BYTES_FIXED_DEREF} .idx --&gt; Header, NumValues,
* Addresses</li>
* <li>{@link Type#BYTES_FIXED_DEREF BYTES_FIXED_DEREF} .dat --&gt; Header, ValueSize,
* <li>{@code BYTES_VAR_STRAIGHT} .idx --&gt; Header, TotalBytes, Addresses</li>
* <li>{@code BYTES_VAR_STRAIGHT} .dat --&gt; Header,
({@link DataOutput#writeByte Byte} * <i>variable ValueSize</i>)<sup>maxdoc</sup></li>
* <li>{@code BYTES_FIXED_DEREF} .idx --&gt; Header, NumValues, Addresses</li>
* <li>{@code BYTES_FIXED_DEREF} .dat --&gt; Header, ValueSize,
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>NumValues</sup></li>
* <li>{@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF} .idx --&gt; Header, TotalVarBytes,
* Addresses</li>
* <li>{@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF} .dat --&gt; Header,
* <li>{@code BYTES_VAR_DEREF} .idx --&gt; Header, TotalVarBytes, Addresses</li>
* <li>{@code BYTES_VAR_DEREF} .dat --&gt; Header,
* (LengthPrefix + {@link DataOutput#writeByte Byte} * <i>variable ValueSize</i>)<sup>NumValues</sup></li>
* <li>{@link Type#BYTES_FIXED_SORTED BYTES_FIXED_SORTED} .idx --&gt; Header, NumValues,
* Ordinals</li>
* <li>{@link Type#BYTES_FIXED_SORTED BYTES_FIXED_SORTED} .dat --&gt; Header, ValueSize,
* <li>{@code BYTES_FIXED_SORTED} .idx --&gt; Header, NumValues, Ordinals</li>
* <li>{@code BYTES_FIXED_SORTED} .dat --&gt; Header, ValueSize,
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>NumValues</sup></li>
* <li>{@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED} .idx --&gt; Header, TotalVarBytes,
* Addresses, Ordinals</li>
* <li>{@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED} .dat --&gt; Header,
* <li>{@code BYTES_VAR_SORTED} .idx --&gt; Header, TotalVarBytes, Addresses, Ordinals</li>
* <li>{@code BYTES_VAR_SORTED} .dat --&gt; Header,
* ({@link DataOutput#writeByte Byte} * <i>variable ValueSize</i>)<sup>NumValues</sup></li>
* </ul>
* Data Types:
@ -122,25 +113,86 @@ import org.apache.lucene.util.packed.PackedInts; // javadocs
* In the VAR_SORTED case, there is double indirection (docid -> ordinal -> address), but
* an additional sentinel ordinal+address is always written (so there are NumValues+1 ordinals). To
* determine the length, ord+1's address is looked up as well.</li>
* <li>{@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} in contrast to other straight
* <li>{@code BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} in contrast to other straight
* variants uses a <tt>.idx</tt> file to improve lookup perfromance. In contrast to
* {@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF} it doesn't apply deduplication of the document values.
* {@code BYTES_VAR_DEREF BYTES_VAR_DEREF} it doesn't apply deduplication of the document values.
* </li>
* </ul>
* @deprecated Only for reading old 4.0 and 4.1 segments
*/
@Deprecated
// NOTE: not registered in SPI, doesnt respect segment suffix, etc
// for back compat only!
public class Lucene40DocValuesFormat extends DocValuesFormat {
/** Sole constructor. */
public Lucene40DocValuesFormat() {
super("Lucene40");
}
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new Lucene40DocValuesConsumer(state, Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX);
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new Lucene40DocValuesProducer(state, Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX);
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
String filename = IndexFileNames.segmentFileName(state.segmentInfo.name,
"dv",
IndexFileNames.COMPOUND_FILE_EXTENSION);
return new Lucene40DocValuesReader(state, filename, Lucene40FieldInfosReader.LEGACY_DV_TYPE_KEY);
}
// constants for VAR_INTS
static final String VAR_INTS_CODEC_NAME = "PackedInts";
static final int VAR_INTS_VERSION_START = 0;
static final int VAR_INTS_VERSION_CURRENT = VAR_INTS_VERSION_START;
static final byte VAR_INTS_PACKED = 0x00;
static final byte VAR_INTS_FIXED_64 = 0x01;
// constants for FIXED_INTS_8, FIXED_INTS_16, FIXED_INTS_32, FIXED_INTS_64
static final String INTS_CODEC_NAME = "Ints";
static final int INTS_VERSION_START = 0;
static final int INTS_VERSION_CURRENT = INTS_VERSION_START;
// constants for FLOAT_32, FLOAT_64
static final String FLOATS_CODEC_NAME = "Floats";
static final int FLOATS_VERSION_START = 0;
static final int FLOATS_VERSION_CURRENT = FLOATS_VERSION_START;
// constants for BYTES_FIXED_STRAIGHT
static final String BYTES_FIXED_STRAIGHT_CODEC_NAME = "FixedStraightBytes";
static final int BYTES_FIXED_STRAIGHT_VERSION_START = 0;
static final int BYTES_FIXED_STRAIGHT_VERSION_CURRENT = BYTES_FIXED_STRAIGHT_VERSION_START;
// constants for BYTES_VAR_STRAIGHT
static final String BYTES_VAR_STRAIGHT_CODEC_NAME_IDX = "VarStraightBytesIdx";
static final String BYTES_VAR_STRAIGHT_CODEC_NAME_DAT = "VarStraightBytesDat";
static final int BYTES_VAR_STRAIGHT_VERSION_START = 0;
static final int BYTES_VAR_STRAIGHT_VERSION_CURRENT = BYTES_VAR_STRAIGHT_VERSION_START;
// constants for BYTES_FIXED_DEREF
static final String BYTES_FIXED_DEREF_CODEC_NAME_IDX = "FixedDerefBytesIdx";
static final String BYTES_FIXED_DEREF_CODEC_NAME_DAT = "FixedDerefBytesDat";
static final int BYTES_FIXED_DEREF_VERSION_START = 0;
static final int BYTES_FIXED_DEREF_VERSION_CURRENT = BYTES_FIXED_DEREF_VERSION_START;
// constants for BYTES_VAR_DEREF
static final String BYTES_VAR_DEREF_CODEC_NAME_IDX = "VarDerefBytesIdx";
static final String BYTES_VAR_DEREF_CODEC_NAME_DAT = "VarDerefBytesDat";
static final int BYTES_VAR_DEREF_VERSION_START = 0;
static final int BYTES_VAR_DEREF_VERSION_CURRENT = BYTES_VAR_DEREF_VERSION_START;
// constants for BYTES_FIXED_SORTED
static final String BYTES_FIXED_SORTED_CODEC_NAME_IDX = "FixedSortedBytesIdx";
static final String BYTES_FIXED_SORTED_CODEC_NAME_DAT = "FixedSortedBytesDat";
static final int BYTES_FIXED_SORTED_VERSION_START = 0;
static final int BYTES_FIXED_SORTED_VERSION_CURRENT = BYTES_FIXED_SORTED_VERSION_START;
// constants for BYTES_VAR_SORTED
// NOTE THIS IS NOT A BUG! 4.0 actually screwed this up (VAR_SORTED and VAR_DEREF have same codec header)
static final String BYTES_VAR_SORTED_CODEC_NAME_IDX = "VarDerefBytesIdx";
static final String BYTES_VAR_SORTED_CODEC_NAME_DAT = "VarDerefBytesDat";
static final int BYTES_VAR_SORTED_VERSION_START = 0;
static final int BYTES_VAR_SORTED_VERSION_CURRENT = BYTES_VAR_SORTED_VERSION_START;
}

View File

@ -1,113 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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 java.util.ArrayList;
import java.util.Collection;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.codecs.lucene40.values.Bytes;
import org.apache.lucene.codecs.lucene40.values.Floats;
import org.apache.lucene.codecs.lucene40.values.Ints;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.0 PerDocProducer implementation that uses compound file.
*
* @see Lucene40DocValuesFormat
* @lucene.experimental
*/
public class Lucene40DocValuesProducer extends PerDocProducerBase {
/** Maps field name to {@link DocValues} instance. */
protected final TreeMap<String,DocValues> docValues;
private final Directory cfs;
/**
* Creates a new {@link Lucene40DocValuesProducer} instance and loads all
* {@link DocValues} instances for this segment and codec.
*/
public Lucene40DocValuesProducer(SegmentReadState state, String segmentSuffix) throws IOException {
if (anyDocValuesFields(state.fieldInfos)) {
cfs = new CompoundFileDirectory(state.dir,
IndexFileNames.segmentFileName(state.segmentInfo.name,
segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
state.context, false);
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.getDocCount(), cfs, state.context);
} else {
cfs = null;
docValues = new TreeMap<String,DocValues>();
}
}
@Override
protected Map<String,DocValues> docValues() {
return docValues;
}
@Override
protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
if (cfs != null) {
final ArrayList<Closeable> list = new ArrayList<Closeable>(closeables);
list.add(cfs);
IOUtils.close(list);
} else {
IOUtils.close(closeables);
}
}
@Override
protected DocValues loadDocValues(int docCount, Directory dir, String id,
Type type, IOContext context) throws IOException {
switch (type) {
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
return Ints.getValues(dir, id, docCount, type, context);
case FLOAT_32:
return Floats.getValues(dir, id, docCount, context, type);
case FLOAT_64:
return Floats.getValues(dir, id, docCount, context, type);
case BYTES_FIXED_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
case BYTES_FIXED_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
case BYTES_FIXED_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
case BYTES_VAR_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
case BYTES_VAR_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
case BYTES_VAR_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
default:
throw new IllegalStateException("unrecognized index values mode " + type);
}
}
}

View File

@ -0,0 +1,621 @@
package org.apache.lucene.codecs.lucene40;
/*
* 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.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosReader.LegacyDocValuesType;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
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;
/**
* Reads the 4.0 format of norms/docvalues
* @lucene.experimental
* @deprecated Only for reading old 4.0 and 4.1 segments
*/
@Deprecated
final class Lucene40DocValuesReader extends DocValuesProducer {
private final Directory dir;
private final SegmentReadState state;
private final String legacyKey;
private static final String segmentSuffix = "dv";
// ram instances we have already loaded
private final Map<Integer,NumericDocValues> numericInstances =
new HashMap<Integer,NumericDocValues>();
private final Map<Integer,BinaryDocValues> binaryInstances =
new HashMap<Integer,BinaryDocValues>();
private final Map<Integer,SortedDocValues> sortedInstances =
new HashMap<Integer,SortedDocValues>();
Lucene40DocValuesReader(SegmentReadState state, String filename, String legacyKey) throws IOException {
this.state = state;
this.legacyKey = legacyKey;
this.dir = new CompoundFileDirectory(state.directory, filename, state.context, false);
}
@Override
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericDocValues instance = numericInstances.get(field.number);
if (instance == null) {
String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
IndexInput input = dir.openInput(fileName, state.context);
boolean success = false;
try {
switch(LegacyDocValuesType.valueOf(field.getAttribute(legacyKey))) {
case VAR_INTS:
instance = loadVarIntsField(field, input);
break;
case FIXED_INTS_8:
instance = loadByteField(field, input);
break;
case FIXED_INTS_16:
instance = loadShortField(field, input);
break;
case FIXED_INTS_32:
instance = loadIntField(field, input);
break;
case FIXED_INTS_64:
instance = loadLongField(field, input);
break;
case FLOAT_32:
instance = loadFloatField(field, input);
break;
case FLOAT_64:
instance = loadDoubleField(field, input);
break;
default:
throw new AssertionError();
}
if (input.getFilePointer() != input.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
}
success = true;
} finally {
if (success) {
IOUtils.close(input);
} else {
IOUtils.closeWhileHandlingException(input);
}
}
numericInstances.put(field.number, instance);
}
return instance;
}
private NumericDocValues loadVarIntsField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.VAR_INTS_CODEC_NAME,
Lucene40DocValuesFormat.VAR_INTS_VERSION_START,
Lucene40DocValuesFormat.VAR_INTS_VERSION_CURRENT);
byte header = input.readByte();
if (header == Lucene40DocValuesFormat.VAR_INTS_FIXED_64) {
int maxDoc = state.segmentInfo.getDocCount();
final long values[] = new long[maxDoc];
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
return new NumericDocValues() {
@Override
public long get(int docID) {
return values[docID];
}
};
} else if (header == Lucene40DocValuesFormat.VAR_INTS_PACKED) {
final long minValue = input.readLong();
final long defaultValue = input.readLong();
final PackedInts.Reader reader = PackedInts.getReader(input);
return new NumericDocValues() {
@Override
public long get(int docID) {
final long value = reader.get(docID);
if (value == defaultValue) {
return 0;
} else {
return minValue + value;
}
}
};
} else {
throw new CorruptIndexException("invalid VAR_INTS header byte: " + header + " (resource=" + input + ")");
}
}
private NumericDocValues loadByteField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 1) {
throw new CorruptIndexException("invalid valueSize: " + valueSize);
}
int maxDoc = state.segmentInfo.getDocCount();
final byte values[] = new byte[maxDoc];
input.readBytes(values, 0, values.length);
return new NumericDocValues() {
@Override
public long get(int docID) {
return values[docID];
}
};
}
private NumericDocValues loadShortField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 2) {
throw new CorruptIndexException("invalid valueSize: " + valueSize);
}
int maxDoc = state.segmentInfo.getDocCount();
final short values[] = new short[maxDoc];
for (int i = 0; i < values.length; i++) {
values[i] = input.readShort();
}
return new NumericDocValues() {
@Override
public long get(int docID) {
return values[docID];
}
};
}
private NumericDocValues loadIntField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 4) {
throw new CorruptIndexException("invalid valueSize: " + valueSize);
}
int maxDoc = state.segmentInfo.getDocCount();
final int values[] = new int[maxDoc];
for (int i = 0; i < values.length; i++) {
values[i] = input.readInt();
}
return new NumericDocValues() {
@Override
public long get(int docID) {
return values[docID];
}
};
}
private NumericDocValues loadLongField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.INTS_CODEC_NAME,
Lucene40DocValuesFormat.INTS_VERSION_START,
Lucene40DocValuesFormat.INTS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 8) {
throw new CorruptIndexException("invalid valueSize: " + valueSize);
}
int maxDoc = state.segmentInfo.getDocCount();
final long values[] = new long[maxDoc];
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
return new NumericDocValues() {
@Override
public long get(int docID) {
return values[docID];
}
};
}
private NumericDocValues loadFloatField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.FLOATS_CODEC_NAME,
Lucene40DocValuesFormat.FLOATS_VERSION_START,
Lucene40DocValuesFormat.FLOATS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 4) {
throw new CorruptIndexException("invalid valueSize: " + valueSize);
}
int maxDoc = state.segmentInfo.getDocCount();
final int values[] = new int[maxDoc];
for (int i = 0; i < values.length; i++) {
values[i] = input.readInt();
}
return new NumericDocValues() {
@Override
public long get(int docID) {
return values[docID];
}
};
}
private NumericDocValues loadDoubleField(FieldInfo field, IndexInput input) throws IOException {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.FLOATS_CODEC_NAME,
Lucene40DocValuesFormat.FLOATS_VERSION_START,
Lucene40DocValuesFormat.FLOATS_VERSION_CURRENT);
int valueSize = input.readInt();
if (valueSize != 8) {
throw new CorruptIndexException("invalid valueSize: " + valueSize);
}
int maxDoc = state.segmentInfo.getDocCount();
final long values[] = new long[maxDoc];
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
return new NumericDocValues() {
@Override
public long get(int docID) {
return values[docID];
}
};
}
@Override
public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryDocValues instance = binaryInstances.get(field.number);
if (instance == null) {
switch(LegacyDocValuesType.valueOf(field.getAttribute(legacyKey))) {
case BYTES_FIXED_STRAIGHT:
instance = loadBytesFixedStraight(field);
break;
case BYTES_VAR_STRAIGHT:
instance = loadBytesVarStraight(field);
break;
case BYTES_FIXED_DEREF:
instance = loadBytesFixedDeref(field);
break;
case BYTES_VAR_DEREF:
instance = loadBytesVarDeref(field);
break;
default:
throw new AssertionError();
}
binaryInstances.put(field.number, instance);
}
return instance;
}
private BinaryDocValues loadBytesFixedStraight(FieldInfo field) throws IOException {
String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
IndexInput input = dir.openInput(fileName, state.context);
boolean success = false;
try {
CodecUtil.checkHeader(input, Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_CODEC_NAME,
Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_STRAIGHT_VERSION_CURRENT);
final int fixedLength = input.readInt();
PagedBytes bytes = new PagedBytes(16);
bytes.copy(input, fixedLength * (long)state.segmentInfo.getDocCount());
final PagedBytes.Reader bytesReader = bytes.freeze(true);
if (input.getFilePointer() != input.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
}
success = true;
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
bytesReader.fillSlice(result, fixedLength * (long)docID, fixedLength);
}
};
} finally {
if (success) {
IOUtils.close(input);
} else {
IOUtils.closeWhileHandlingException(input);
}
}
}
private BinaryDocValues loadBytesVarStraight(FieldInfo field) throws IOException {
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "idx");
IndexInput data = null;
IndexInput index = null;
boolean success = false;
try {
data = dir.openInput(dataName, state.context);
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_CURRENT);
index = dir.openInput(indexName, state.context);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_STRAIGHT_VERSION_CURRENT);
long totalBytes = index.readVLong();
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, totalBytes);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader reader = PackedInts.getReader(index);
if (data.getFilePointer() != data.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + dataName + "\": read " + data.getFilePointer() + " vs size " + data.length() + " (resource: " + data + ")");
}
if (index.getFilePointer() != index.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + indexName + "\": read " + index.getFilePointer() + " vs size " + index.length() + " (resource: " + index + ")");
}
success = true;
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
long startAddress = reader.get(docID);
long endAddress = reader.get(docID+1);
bytesReader.fillSlice(result, startAddress, (int)(endAddress - startAddress));
}
};
} finally {
if (success) {
IOUtils.close(data, index);
} else {
IOUtils.closeWhileHandlingException(data, index);
}
}
}
private BinaryDocValues loadBytesFixedDeref(FieldInfo field) throws IOException {
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "idx");
IndexInput data = null;
IndexInput index = null;
boolean success = false;
try {
data = dir.openInput(dataName, state.context);
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_FIXED_DEREF_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_CURRENT);
index = dir.openInput(indexName, state.context);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_FIXED_DEREF_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_DEREF_VERSION_CURRENT);
final int fixedLength = data.readInt();
final int valueCount = index.readInt();
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, fixedLength * (long) valueCount);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader reader = PackedInts.getReader(index);
if (data.getFilePointer() != data.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + dataName + "\": read " + data.getFilePointer() + " vs size " + data.length() + " (resource: " + data + ")");
}
if (index.getFilePointer() != index.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + indexName + "\": read " + index.getFilePointer() + " vs size " + index.length() + " (resource: " + index + ")");
}
success = true;
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
final long offset = fixedLength * reader.get(docID);
bytesReader.fillSlice(result, offset, fixedLength);
}
};
} finally {
if (success) {
IOUtils.close(data, index);
} else {
IOUtils.closeWhileHandlingException(data, index);
}
}
}
private BinaryDocValues loadBytesVarDeref(FieldInfo field) throws IOException {
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "idx");
IndexInput data = null;
IndexInput index = null;
boolean success = false;
try {
data = dir.openInput(dataName, state.context);
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_DEREF_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_CURRENT);
index = dir.openInput(indexName, state.context);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_DEREF_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_DEREF_VERSION_CURRENT);
final long totalBytes = index.readLong();
final PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, totalBytes);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader reader = PackedInts.getReader(index);
if (data.getFilePointer() != data.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + dataName + "\": read " + data.getFilePointer() + " vs size " + data.length() + " (resource: " + data + ")");
}
if (index.getFilePointer() != index.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + indexName + "\": read " + index.getFilePointer() + " vs size " + index.length() + " (resource: " + index + ")");
}
success = true;
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
long startAddress = reader.get(docID);
BytesRef lengthBytes = new BytesRef();
bytesReader.fillSlice(lengthBytes, startAddress, 1);
byte code = lengthBytes.bytes[lengthBytes.offset];
if ((code & 128) == 0) {
// length is 1 byte
bytesReader.fillSlice(result, startAddress + 1, (int) code);
} else {
bytesReader.fillSlice(lengthBytes, startAddress + 1, 1);
int length = ((code & 0x7f) << 8) | (lengthBytes.bytes[lengthBytes.offset] & 0xff);
bytesReader.fillSlice(result, startAddress + 2, length);
}
}
};
} finally {
if (success) {
IOUtils.close(data, index);
} else {
IOUtils.closeWhileHandlingException(data, index);
}
}
}
@Override
public synchronized SortedDocValues getSorted(FieldInfo field) throws IOException {
SortedDocValues instance = sortedInstances.get(field.number);
if (instance == null) {
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "dat");
String indexName = IndexFileNames.segmentFileName(state.segmentInfo.name + "_" + Integer.toString(field.number), segmentSuffix, "idx");
IndexInput data = null;
IndexInput index = null;
boolean success = false;
try {
data = dir.openInput(dataName, state.context);
index = dir.openInput(indexName, state.context);
switch(LegacyDocValuesType.valueOf(field.getAttribute(legacyKey))) {
case BYTES_FIXED_SORTED:
instance = loadBytesFixedSorted(field, data, index);
break;
case BYTES_VAR_SORTED:
instance = loadBytesVarSorted(field, data, index);
break;
default:
throw new AssertionError();
}
if (data.getFilePointer() != data.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + dataName + "\": read " + data.getFilePointer() + " vs size " + data.length() + " (resource: " + data + ")");
}
if (index.getFilePointer() != index.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + indexName + "\": read " + index.getFilePointer() + " vs size " + index.length() + " (resource: " + index + ")");
}
success = true;
} finally {
if (success) {
IOUtils.close(data, index);
} else {
IOUtils.closeWhileHandlingException(data, index);
}
}
sortedInstances.put(field.number, instance);
}
return instance;
}
private SortedDocValues loadBytesFixedSorted(FieldInfo field, IndexInput data, IndexInput index) throws IOException {
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_FIXED_SORTED_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_CURRENT);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_FIXED_SORTED_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_START,
Lucene40DocValuesFormat.BYTES_FIXED_SORTED_VERSION_CURRENT);
final int fixedLength = data.readInt();
final int valueCount = index.readInt();
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, fixedLength * (long) valueCount);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader reader = PackedInts.getReader(index);
return correctBuggyOrds(new SortedDocValues() {
@Override
public int getOrd(int docID) {
return (int) reader.get(docID);
}
@Override
public void lookupOrd(int ord, BytesRef result) {
bytesReader.fillSlice(result, fixedLength * (long) ord, fixedLength);
}
@Override
public int getValueCount() {
return valueCount;
}
});
}
private SortedDocValues loadBytesVarSorted(FieldInfo field, IndexInput data, IndexInput index) throws IOException {
CodecUtil.checkHeader(data, Lucene40DocValuesFormat.BYTES_VAR_SORTED_CODEC_NAME_DAT,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_CURRENT);
CodecUtil.checkHeader(index, Lucene40DocValuesFormat.BYTES_VAR_SORTED_CODEC_NAME_IDX,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_START,
Lucene40DocValuesFormat.BYTES_VAR_SORTED_VERSION_CURRENT);
long maxAddress = index.readLong();
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, maxAddress);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
final PackedInts.Reader addressReader = PackedInts.getReader(index);
final PackedInts.Reader ordsReader = PackedInts.getReader(index);
final int valueCount = addressReader.size() - 1;
return correctBuggyOrds(new SortedDocValues() {
@Override
public int getOrd(int docID) {
return (int)ordsReader.get(docID);
}
@Override
public void lookupOrd(int ord, BytesRef result) {
long startAddress = addressReader.get(ord);
long endAddress = addressReader.get(ord+1);
bytesReader.fillSlice(result, startAddress, (int)(endAddress - startAddress));
}
@Override
public int getValueCount() {
return valueCount;
}
});
}
// detects and corrects LUCENE-4717 in old indexes
private SortedDocValues correctBuggyOrds(final SortedDocValues in) {
final int maxDoc = state.segmentInfo.getDocCount();
for (int i = 0; i < maxDoc; i++) {
if (in.getOrd(i) == 0) {
return in; // ok
}
}
// we had ord holes, return an ord-shifting-impl that corrects the bug
return new SortedDocValues() {
@Override
public int getOrd(int docID) {
return in.getOrd(docID) - 1;
}
@Override
public void lookupOrd(int ord, BytesRef result) {
in.lookupOrd(ord+1, result);
}
@Override
public int getValueCount() {
return in.getValueCount() - 1;
}
};
}
@Override
public void close() throws IOException {
dir.close();
}
}

View File

@ -23,8 +23,6 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.DocValues; // javadoc
import org.apache.lucene.index.DocValues.Type; // javadoc
import org.apache.lucene.store.DataOutput; // javadoc
/**
@ -72,32 +70,33 @@ import org.apache.lucene.store.DataOutput; // javadoc
* <li>DocValuesBits: a byte containing per-document value types. The type
* recorded as two four-bit integers, with the high-order bits representing
* <code>norms</code> options, and the low-order bits representing
* {@link DocValues} options. Each four-bit integer can be decoded as such:
* {@code DocValues} options. Each four-bit integer can be decoded as such:
* <ul>
* <li>0: no DocValues for this field.</li>
* <li>1: variable-width signed integers. ({@link Type#VAR_INTS VAR_INTS})</li>
* <li>2: 32-bit floating point values. ({@link Type#FLOAT_32 FLOAT_32})</li>
* <li>3: 64-bit floating point values. ({@link Type#FLOAT_64 FLOAT_64})</li>
* <li>4: fixed-length byte array values. ({@link Type#BYTES_FIXED_STRAIGHT BYTES_FIXED_STRAIGHT})</li>
* <li>5: fixed-length dereferenced byte array values. ({@link Type#BYTES_FIXED_DEREF BYTES_FIXED_DEREF})</li>
* <li>6: variable-length byte array values. ({@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT})</li>
* <li>7: variable-length dereferenced byte array values. ({@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF})</li>
* <li>8: 16-bit signed integers. ({@link Type#FIXED_INTS_16 FIXED_INTS_16})</li>
* <li>9: 32-bit signed integers. ({@link Type#FIXED_INTS_32 FIXED_INTS_32})</li>
* <li>10: 64-bit signed integers. ({@link Type#FIXED_INTS_64 FIXED_INTS_64})</li>
* <li>11: 8-bit signed integers. ({@link Type#FIXED_INTS_8 FIXED_INTS_8})</li>
* <li>12: fixed-length sorted byte array values. ({@link Type#BYTES_FIXED_SORTED BYTES_FIXED_SORTED})</li>
* <li>13: variable-length sorted byte array values. ({@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED})</li>
* <li>1: variable-width signed integers. ({@code Type#VAR_INTS VAR_INTS})</li>
* <li>2: 32-bit floating point values. ({@code Type#FLOAT_32 FLOAT_32})</li>
* <li>3: 64-bit floating point values. ({@code Type#FLOAT_64 FLOAT_64})</li>
* <li>4: fixed-length byte array values. ({@code Type#BYTES_FIXED_STRAIGHT BYTES_FIXED_STRAIGHT})</li>
* <li>5: fixed-length dereferenced byte array values. ({@code Type#BYTES_FIXED_DEREF BYTES_FIXED_DEREF})</li>
* <li>6: variable-length byte array values. ({@code Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT})</li>
* <li>7: variable-length dereferenced byte array values. ({@code Type#BYTES_VAR_DEREF BYTES_VAR_DEREF})</li>
* <li>8: 16-bit signed integers. ({@code Type#FIXED_INTS_16 FIXED_INTS_16})</li>
* <li>9: 32-bit signed integers. ({@code Type#FIXED_INTS_32 FIXED_INTS_32})</li>
* <li>10: 64-bit signed integers. ({@code Type#FIXED_INTS_64 FIXED_INTS_64})</li>
* <li>11: 8-bit signed integers. ({@code Type#FIXED_INTS_8 FIXED_INTS_8})</li>
* <li>12: fixed-length sorted byte array values. ({@code Type#BYTES_FIXED_SORTED BYTES_FIXED_SORTED})</li>
* <li>13: variable-length sorted byte array values. ({@code Type#BYTES_VAR_SORTED BYTES_VAR_SORTED})</li>
* </ul>
* </li>
* <li>Attributes: a key-value map of codec-private attributes.</li>
* </ul>
*
* @lucene.experimental
* @deprecated Only for reading old 4.0 and 4.1 segments
*/
@Deprecated
public class Lucene40FieldInfosFormat extends FieldInfosFormat {
private final FieldInfosReader reader = new Lucene40FieldInfosReader();
private final FieldInfosWriter writer = new Lucene40FieldInfosWriter();
/** Sole constructor. */
public Lucene40FieldInfosFormat() {
@ -110,6 +109,21 @@ public class Lucene40FieldInfosFormat extends FieldInfosFormat {
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
return writer;
throw new UnsupportedOperationException("this codec can only be used for reading");
}
/** Extension of field infos */
static final String FIELD_INFOS_EXTENSION = "fnm";
static final String CODEC_NAME = "Lucene40FieldInfos";
static final int FORMAT_START = 0;
static final int FORMAT_CURRENT = FORMAT_START;
static final byte IS_INDEXED = 0x1;
static final byte STORE_TERMVECTOR = 0x2;
static final byte STORE_OFFSETS_IN_POSTINGS = 0x4;
static final byte OMIT_NORMS = 0x10;
static final byte STORE_PAYLOADS = 0x20;
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
static final byte OMIT_POSITIONS = -128;
}

View File

@ -27,8 +27,8 @@ import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@ -39,8 +39,10 @@ import org.apache.lucene.util.IOUtils;
*
* @lucene.experimental
* @see Lucene40FieldInfosFormat
* @deprecated Only for reading old 4.0 and 4.1 segments
*/
public class Lucene40FieldInfosReader extends FieldInfosReader {
@Deprecated
class Lucene40FieldInfosReader extends FieldInfosReader {
/** Sole constructor. */
public Lucene40FieldInfosReader() {
@ -48,14 +50,14 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
@Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosWriter.FIELD_INFOS_EXTENSION);
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene40FieldInfosFormat.FIELD_INFOS_EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);
boolean success = false;
try {
CodecUtil.checkHeader(input, Lucene40FieldInfosWriter.CODEC_NAME,
Lucene40FieldInfosWriter.FORMAT_START,
Lucene40FieldInfosWriter.FORMAT_CURRENT);
CodecUtil.checkHeader(input, Lucene40FieldInfosFormat.CODEC_NAME,
Lucene40FieldInfosFormat.FORMAT_START,
Lucene40FieldInfosFormat.FORMAT_CURRENT);
final int size = input.readVInt(); //read in the size
FieldInfo infos[] = new FieldInfo[size];
@ -64,18 +66,18 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
String name = input.readString();
final int fieldNumber = input.readVInt();
byte bits = input.readByte();
boolean isIndexed = (bits & Lucene40FieldInfosWriter.IS_INDEXED) != 0;
boolean storeTermVector = (bits & Lucene40FieldInfosWriter.STORE_TERMVECTOR) != 0;
boolean omitNorms = (bits & Lucene40FieldInfosWriter.OMIT_NORMS) != 0;
boolean storePayloads = (bits & Lucene40FieldInfosWriter.STORE_PAYLOADS) != 0;
boolean isIndexed = (bits & Lucene40FieldInfosFormat.IS_INDEXED) != 0;
boolean storeTermVector = (bits & Lucene40FieldInfosFormat.STORE_TERMVECTOR) != 0;
boolean omitNorms = (bits & Lucene40FieldInfosFormat.OMIT_NORMS) != 0;
boolean storePayloads = (bits & Lucene40FieldInfosFormat.STORE_PAYLOADS) != 0;
final IndexOptions indexOptions;
if (!isIndexed) {
indexOptions = null;
} else if ((bits & Lucene40FieldInfosWriter.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
} else if ((bits & Lucene40FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_ONLY;
} else if ((bits & Lucene40FieldInfosWriter.OMIT_POSITIONS) != 0) {
} else if ((bits & Lucene40FieldInfosFormat.OMIT_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS;
} else if ((bits & Lucene40FieldInfosWriter.STORE_OFFSETS_IN_POSTINGS) != 0) {
} else if ((bits & Lucene40FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
} else {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
@ -89,11 +91,20 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
}
// DV Types are packed in one byte
byte val = input.readByte();
final DocValues.Type docValuesType = getDocValuesType((byte) (val & 0x0F));
final DocValues.Type normsType = getDocValuesType((byte) ((val >>> 4) & 0x0F));
final Map<String,String> attributes = input.readStringStringMap();
final LegacyDocValuesType oldValuesType = getDocValuesType((byte) (val & 0x0F));
final LegacyDocValuesType oldNormsType = getDocValuesType((byte) ((val >>> 4) & 0x0F));
final Map<String,String> attributes = input.readStringStringMap();;
if (oldValuesType.mapping != null) {
attributes.put(LEGACY_DV_TYPE_KEY, oldValuesType.name());
}
if (oldNormsType.mapping != null) {
if (oldNormsType.mapping != DocValuesType.NUMERIC) {
throw new CorruptIndexException("invalid norm type: " + oldNormsType);
}
attributes.put(LEGACY_NORM_TYPE_KEY, oldNormsType.name());
}
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes));
omitNorms, storePayloads, indexOptions, oldValuesType.mapping, oldNormsType.mapping, Collections.unmodifiableMap(attributes));
}
if (input.getFilePointer() != input.length()) {
@ -110,39 +121,35 @@ public class Lucene40FieldInfosReader extends FieldInfosReader {
}
}
}
private static DocValues.Type getDocValuesType(final byte b) {
switch(b) {
case 0:
return null;
case 1:
return DocValues.Type.VAR_INTS;
case 2:
return DocValues.Type.FLOAT_32;
case 3:
return DocValues.Type.FLOAT_64;
case 4:
return DocValues.Type.BYTES_FIXED_STRAIGHT;
case 5:
return DocValues.Type.BYTES_FIXED_DEREF;
case 6:
return DocValues.Type.BYTES_VAR_STRAIGHT;
case 7:
return DocValues.Type.BYTES_VAR_DEREF;
case 8:
return DocValues.Type.FIXED_INTS_16;
case 9:
return DocValues.Type.FIXED_INTS_32;
case 10:
return DocValues.Type.FIXED_INTS_64;
case 11:
return DocValues.Type.FIXED_INTS_8;
case 12:
return DocValues.Type.BYTES_FIXED_SORTED;
case 13:
return DocValues.Type.BYTES_VAR_SORTED;
default:
throw new IllegalStateException("unhandled indexValues type " + b);
static final String LEGACY_DV_TYPE_KEY = Lucene40FieldInfosReader.class.getSimpleName() + ".dvtype";
static final String LEGACY_NORM_TYPE_KEY = Lucene40FieldInfosReader.class.getSimpleName() + ".normtype";
// mapping of 4.0 types -> 4.2 types
static enum LegacyDocValuesType {
NONE(null),
VAR_INTS(DocValuesType.NUMERIC),
FLOAT_32(DocValuesType.NUMERIC),
FLOAT_64(DocValuesType.NUMERIC),
BYTES_FIXED_STRAIGHT(DocValuesType.BINARY),
BYTES_FIXED_DEREF(DocValuesType.BINARY),
BYTES_VAR_STRAIGHT(DocValuesType.BINARY),
BYTES_VAR_DEREF(DocValuesType.BINARY),
FIXED_INTS_16(DocValuesType.NUMERIC),
FIXED_INTS_32(DocValuesType.NUMERIC),
FIXED_INTS_64(DocValuesType.NUMERIC),
FIXED_INTS_8(DocValuesType.NUMERIC),
BYTES_FIXED_SORTED(DocValuesType.SORTED),
BYTES_VAR_SORTED(DocValuesType.SORTED);
final DocValuesType mapping;
LegacyDocValuesType(DocValuesType mapping) {
this.mapping = mapping;
}
}
// decodes a 4.0 type
private static LegacyDocValuesType getDocValuesType(byte b) {
return LegacyDocValuesType.values()[b];
}
}

View File

@ -19,17 +19,13 @@ package org.apache.lucene.codecs.lucene40;
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.CompoundFileDirectory; // javadocs
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.CompoundFileDirectory;
/**
* Lucene 4.0 Norms Format.
@ -44,82 +40,24 @@ import org.apache.lucene.store.CompoundFileDirectory; // javadocs
*
* @see Lucene40DocValuesFormat
* @lucene.experimental
* @deprecated Only for reading old 4.0 and 4.1 segments
*/
@Deprecated
public class Lucene40NormsFormat extends NormsFormat {
private final static String NORMS_SEGMENT_SUFFIX = "nrm";
/** Sole constructor. */
public Lucene40NormsFormat() {
}
public Lucene40NormsFormat() {}
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new Lucene40NormsDocValuesConsumer(state, NORMS_SEGMENT_SUFFIX);
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
throw new UnsupportedOperationException("this codec can only be used for reading");
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new Lucene40NormsDocValuesProducer(state, NORMS_SEGMENT_SUFFIX);
}
/**
* Lucene 4.0 PerDocProducer implementation that uses compound file.
*
* @see Lucene40DocValuesFormat
*/
public static class Lucene40NormsDocValuesProducer extends Lucene40DocValuesProducer {
/** Sole constructor. */
public Lucene40NormsDocValuesProducer(SegmentReadState state,
String segmentSuffix) throws IOException {
super(state, segmentSuffix);
}
@Override
protected boolean canLoad(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
@Override
protected boolean anyDocValuesFields(FieldInfos infos) {
return infos.hasNorms();
}
}
/**
* Lucene 4.0 PerDocConsumer implementation that uses compound file.
*
* @see Lucene40DocValuesFormat
* @lucene.experimental
*/
public static class Lucene40NormsDocValuesConsumer extends Lucene40DocValuesConsumer {
/** Sole constructor. */
public Lucene40NormsDocValuesConsumer(PerDocWriteState state,
String segmentSuffix) {
super(state, segmentSuffix);
}
@Override
protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
throws IOException {
return reader.normValues(info.name);
}
@Override
protected boolean canMerge(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
String filename = IndexFileNames.segmentFileName(state.segmentInfo.name,
"nrm",
IndexFileNames.COMPOUND_FILE_EXTENSION);
return new Lucene40DocValuesReader(state, filename, Lucene40FieldInfosReader.LEGACY_NORM_TYPE_KEY);
}
}

View File

@ -42,7 +42,7 @@ public final class Lucene40PostingsBaseFormat extends PostingsBaseFormat {
@Override
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
return new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
return new Lucene40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
}
@Override

View File

@ -248,12 +248,12 @@ public class Lucene40PostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postings = new Lucene40PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
PostingsReaderBase postings = new Lucene40PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(
state.dir,
state.directory,
state.fieldInfos,
state.segmentInfo,
postings,

View File

@ -363,11 +363,11 @@ file, previously they were stored in text format only.</li>
frequencies.</li>
<li>In version 4.0, the format of the inverted index became extensible via
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
factors need no longer be a single byte, they can be any DocValues
{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
strings, they can be any byte sequence. Term offsets can optionally be indexed
into the postings lists. Payloads can be stored in the term vectors.</li>
({@code DocValues}) was introduced. Normalization factors need no longer be a
single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
Terms need not be unicode strings, they can be any byte sequence. Term offsets
can optionally be indexed into the postings lists. Payloads can be stored in the
term vectors.</li>
</ul>
<a name="Limitations" id="Limitations"></a>
<h2>Limitations</h2>

View File

@ -1,611 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.
*/
/** Base class for specific Bytes Reader/Writer implementations */
import java.io.IOException;
import java.util.Comparator;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash.TrackingDirectBytesStartArray;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts;
/**
* Provides concrete Writer/Reader implementations for <tt>byte[]</tt> value per
* document. There are 6 package-private default implementations of this, for
* all combinations of {@link Mode#DEREF}/{@link Mode#STRAIGHT} x fixed-length/variable-length.
*
* <p>
* NOTE: Currently the total amount of byte[] data stored (across a single
* segment) cannot exceed 2GB.
* </p>
* <p>
* NOTE: Each byte[] must be <= 32768 bytes in length
* </p>
*
* @lucene.experimental
*/
public final class Bytes {
static final String DV_SEGMENT_SUFFIX = "dv";
// TODO - add bulk copy where possible
private Bytes() { /* don't instantiate! */
}
/**
* Defines the {@link Writer}s store mode. The writer will either store the
* bytes sequentially ({@link #STRAIGHT}, dereferenced ({@link #DEREF}) or
* sorted ({@link #SORTED})
*
* @lucene.experimental
*/
public static enum Mode {
/**
* Mode for sequentially stored bytes
*/
STRAIGHT,
/**
* Mode for dereferenced stored bytes
*/
DEREF,
/**
* Mode for sorted stored bytes
*/
SORTED
};
/**
* Creates a new <tt>byte[]</tt> {@link Writer} instances for the given
* directory.
*
* @param dir
* the directory to write the values to
* @param id
* the id used to create a unique file name. Usually composed out of
* the segment name and a unique id per segment.
* @param mode
* the writers store mode
* @param fixedSize
* <code>true</code> if all bytes subsequently passed to the
* {@link Writer} will have the same length
* @param sortComparator {@link BytesRef} comparator used by sorted variants.
* If <code>null</code> {@link BytesRef#getUTF8SortedAsUnicodeComparator()}
* is used instead
* @param bytesUsed
* an {@link AtomicLong} instance to track the used bytes within the
* {@link Writer}. A call to {@link Writer#finish(int)} will release
* all internally used resources and frees the memory tracking
* reference.
* @param acceptableOverheadRatio
* how to trade space for speed. This option is only applicable for
* docvalues of type {@link Type#BYTES_FIXED_SORTED} and
* {@link Type#BYTES_VAR_SORTED}.
* @param context I/O Context
* @return a new {@link Writer} instance
* @see PackedInts#getReader(org.apache.lucene.store.DataInput)
*/
public static DocValuesConsumer getWriter(Directory dir, String id, Mode mode,
boolean fixedSize, Comparator<BytesRef> sortComparator,
Counter bytesUsed, IOContext context, float acceptableOverheadRatio) {
// TODO -- i shouldn't have to specify fixed? can
// track itself & do the write thing at write time?
if (sortComparator == null) {
sortComparator = BytesRef.getUTF8SortedAsUnicodeComparator();
}
if (fixedSize) {
if (mode == Mode.STRAIGHT) {
return new FixedStraightBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.DEREF) {
return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.SORTED) {
return new FixedSortedBytesImpl.Writer(dir, id, sortComparator, bytesUsed, context, acceptableOverheadRatio);
}
} else {
if (mode == Mode.STRAIGHT) {
return new VarStraightBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.DEREF) {
return new VarDerefBytesImpl.Writer(dir, id, bytesUsed, context);
} else if (mode == Mode.SORTED) {
return new VarSortedBytesImpl.Writer(dir, id, sortComparator, bytesUsed, context, acceptableOverheadRatio);
}
}
throw new IllegalArgumentException("");
}
/**
* Creates a new {@link DocValues} instance that provides either memory
* resident or iterative access to a per-document stored <tt>byte[]</tt>
* value. The returned {@link DocValues} instance will be initialized without
* consuming a significant amount of memory.
*
* @param dir
* the directory to load the {@link DocValues} from.
* @param id
* the file ID in the {@link Directory} to load the values from.
* @param mode
* the mode used to store the values
* @param fixedSize
* <code>true</code> iff the values are stored with fixed-size,
* otherwise <code>false</code>
* @param maxDoc
* the number of document values stored for the given ID
* @param sortComparator {@link BytesRef} comparator used by sorted variants.
* If <code>null</code> {@link BytesRef#getUTF8SortedAsUnicodeComparator()}
* is used instead
* @return an initialized {@link DocValues} instance.
* @throws IOException
* if an {@link IOException} occurs
*/
public static DocValues getValues(Directory dir, String id, Mode mode,
boolean fixedSize, int maxDoc, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
if (sortComparator == null) {
sortComparator = BytesRef.getUTF8SortedAsUnicodeComparator();
}
// TODO -- I can peek @ header to determing fixed/mode?
if (fixedSize) {
if (mode == Mode.STRAIGHT) {
return new FixedStraightBytesImpl.FixedStraightReader(dir, id, maxDoc, context);
} else if (mode == Mode.DEREF) {
return new FixedDerefBytesImpl.FixedDerefReader(dir, id, maxDoc, context);
} else if (mode == Mode.SORTED) {
return new FixedSortedBytesImpl.Reader(dir, id, maxDoc, context, Type.BYTES_FIXED_SORTED, sortComparator);
}
} else {
if (mode == Mode.STRAIGHT) {
return new VarStraightBytesImpl.VarStraightReader(dir, id, maxDoc, context);
} else if (mode == Mode.DEREF) {
return new VarDerefBytesImpl.VarDerefReader(dir, id, maxDoc, context);
} else if (mode == Mode.SORTED) {
return new VarSortedBytesImpl.Reader(dir, id, maxDoc,context, Type.BYTES_VAR_SORTED, sortComparator);
}
}
throw new IllegalArgumentException("Illegal Mode: " + mode);
}
// TODO open up this API?
static abstract class BytesSourceBase extends Source {
private final PagedBytes pagedBytes;
protected final IndexInput datIn;
protected final IndexInput idxIn;
protected final static int PAGED_BYTES_BITS = 15;
protected final PagedBytes.Reader data;
protected final long totalLengthInBytes;
protected BytesSourceBase(IndexInput datIn, IndexInput idxIn,
PagedBytes pagedBytes, long bytesToRead, Type type) throws IOException {
super(type);
assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
+ (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
this.datIn = datIn;
this.totalLengthInBytes = bytesToRead;
this.pagedBytes = pagedBytes;
this.pagedBytes.copy(datIn, bytesToRead);
data = pagedBytes.freeze(true);
this.idxIn = idxIn;
}
}
// TODO: open up this API?!
static abstract class BytesWriterBase extends Writer {
private final String id;
private IndexOutput idxOut;
private IndexOutput datOut;
protected BytesRef bytesRef = new BytesRef();
private final Directory dir;
private final String codecNameIdx;
private final String codecNameDat;
private final int version;
private final IOContext context;
protected BytesWriterBase(Directory dir, String id, String codecNameIdx, String codecNameDat,
int version, Counter bytesUsed, IOContext context, Type type) {
super(bytesUsed, type);
this.id = id;
this.dir = dir;
this.codecNameIdx = codecNameIdx;
this.codecNameDat = codecNameDat;
this.version = version;
this.context = context;
assert codecNameDat != null || codecNameIdx != null: "both codec names are null";
assert (codecNameDat != null && !codecNameDat.equals(codecNameIdx))
|| (codecNameIdx != null && !codecNameIdx.equals(codecNameDat)):
"index and data codec names must not be equal";
}
protected IndexOutput getOrCreateDataOut() throws IOException {
if (datOut == null) {
boolean success = false;
assert codecNameDat != null;
try {
datOut = dir.createOutput(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
DocValuesWriterBase.DATA_EXTENSION), context);
CodecUtil.writeHeader(datOut, codecNameDat, version);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(datOut);
}
}
}
return datOut;
}
protected IndexOutput getIndexOut() {
return idxOut;
}
protected IndexOutput getDataOut() {
return datOut;
}
protected IndexOutput getOrCreateIndexOut() throws IOException {
boolean success = false;
try {
if (idxOut == null) {
assert codecNameIdx != null;
idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
DocValuesWriterBase.INDEX_EXTENSION), context);
CodecUtil.writeHeader(idxOut, codecNameIdx, version);
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(idxOut);
}
}
return idxOut;
}
@Override
public abstract void finish(int docCount) throws IOException;
}
/**
* Opens all necessary files, but does not read any data in until you call
* {@link #loadSource}.
*/
static abstract class BytesReaderBase extends DocValues {
protected final IndexInput idxIn;
protected final IndexInput datIn;
protected final int version;
protected final String id;
protected final Type type;
protected BytesReaderBase(Directory dir, String id, String codecNameIdx, String codecNameDat,
int maxVersion, boolean doIndex, IOContext context, Type type) throws IOException {
IndexInput dataIn = null;
IndexInput indexIn = null;
boolean success = false;
try {
dataIn = dir.openInput(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
DocValuesWriterBase.DATA_EXTENSION), context);
version = CodecUtil.checkHeader(dataIn, codecNameDat, maxVersion, maxVersion);
if (doIndex) {
indexIn = dir.openInput(IndexFileNames.segmentFileName(id, DV_SEGMENT_SUFFIX,
DocValuesWriterBase.INDEX_EXTENSION), context);
final int version2 = CodecUtil.checkHeader(indexIn, codecNameIdx,
maxVersion, maxVersion);
assert version == version2;
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(dataIn, indexIn);
}
}
datIn = dataIn;
idxIn = indexIn;
this.type = type;
this.id = id;
}
/**
* clones and returns the data {@link IndexInput}
*/
protected final IndexInput cloneData() {
assert datIn != null;
return datIn.clone();
}
/**
* clones and returns the indexing {@link IndexInput}
*/
protected final IndexInput cloneIndex() {
assert idxIn != null;
return idxIn.clone();
}
@Override
public void close() throws IOException {
try {
super.close();
} finally {
IOUtils.close(datIn, idxIn);
}
}
@Override
public Type getType() {
return type;
}
}
static abstract class DerefBytesWriterBase extends BytesWriterBase {
protected int size = -1;
protected int lastDocId = -1;
protected int[] docToEntry;
protected final BytesRefHash hash;
protected final float acceptableOverheadRatio;
protected long maxBytes = 0;
protected DerefBytesWriterBase(Directory dir, String id, String codecNameIdx, String codecNameDat,
int codecVersion, Counter bytesUsed, IOContext context, Type type) {
this(dir, id, codecNameIdx, codecNameDat, codecVersion, new DirectTrackingAllocator(
ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, PackedInts.DEFAULT, type);
}
protected DerefBytesWriterBase(Directory dir, String id, String codecNameIdx, String codecNameDat,
int codecVersion, Counter bytesUsed, IOContext context, float acceptableOverheadRatio, Type type) {
this(dir, id, codecNameIdx, codecNameDat, codecVersion, new DirectTrackingAllocator(
ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed), bytesUsed, context, acceptableOverheadRatio, type);
}
protected DerefBytesWriterBase(Directory dir, String id, String codecNameIdx, String codecNameDat, int codecVersion, Allocator allocator,
Counter bytesUsed, IOContext context, float acceptableOverheadRatio, Type type) {
super(dir, id, codecNameIdx, codecNameDat, codecVersion, bytesUsed, context, type);
hash = new BytesRefHash(new ByteBlockPool(allocator),
BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
docToEntry = new int[1];
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
this.acceptableOverheadRatio = acceptableOverheadRatio;
}
protected static int writePrefixLength(DataOutput datOut, BytesRef bytes)
throws IOException {
if (bytes.length < 128) {
datOut.writeByte((byte) bytes.length);
return 1;
} else {
datOut.writeByte((byte) (0x80 | (bytes.length >> 8)));
datOut.writeByte((byte) (bytes.length & 0xff));
return 2;
}
}
@Override
public void add(int docID, StorableField value) throws IOException {
BytesRef bytes = value.binaryValue();
assert bytes != null;
if (bytes.length == 0) { // default value - skip it
return;
}
checkSize(bytes);
fillDefault(docID);
int ord = hash.add(bytes);
if (ord < 0) {
ord = (-ord) - 1;
} else {
maxBytes += bytes.length;
}
docToEntry[docID] = ord;
lastDocId = docID;
}
protected void fillDefault(int docID) {
if (docID >= docToEntry.length) {
final int size = docToEntry.length;
docToEntry = ArrayUtil.grow(docToEntry, 1 + docID);
bytesUsed.addAndGet((docToEntry.length - size)
* RamUsageEstimator.NUM_BYTES_INT);
}
assert size >= 0;
BytesRef ref = new BytesRef(size);
ref.length = size;
int ord = hash.add(ref);
if (ord < 0) {
ord = (-ord) - 1;
}
for (int i = lastDocId+1; i < docID; i++) {
docToEntry[i] = ord;
}
}
protected void checkSize(BytesRef bytes) {
if (size == -1) {
size = bytes.length;
} else if (bytes.length != size) {
throw new IllegalArgumentException("expected bytes size=" + size
+ " but got " + bytes.length);
}
}
@Override
public int getValueSize() {
return size;
}
// Important that we get docCount, in case there were
// some last docs that we didn't see
@Override
public void finish(int docCount) throws IOException {
boolean success = false;
try {
finishInternal(docCount);
success = true;
} finally {
releaseResources();
if (success) {
IOUtils.close(getIndexOut(), getDataOut());
} else {
IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
}
}
}
protected abstract void finishInternal(int docCount) throws IOException;
protected void releaseResources() {
hash.close();
bytesUsed.addAndGet((-docToEntry.length) * RamUsageEstimator.NUM_BYTES_INT);
docToEntry = null;
}
protected void writeIndex(IndexOutput idxOut, int docCount,
long maxValue, int[] toEntry) throws IOException {
writeIndex(idxOut, docCount, maxValue, (int[])null, toEntry);
}
protected void writeIndex(IndexOutput idxOut, int docCount,
long maxValue, int[] addresses, int[] toEntry) throws IOException {
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
PackedInts.bitsRequired(maxValue), acceptableOverheadRatio);
final int limit = docCount > docToEntry.length ? docToEntry.length
: docCount;
assert toEntry.length >= limit -1;
if (addresses != null) {
for (int i = 0; i < limit; i++) {
assert addresses[toEntry[i]] >= 0;
w.add(addresses[toEntry[i]]);
}
} else {
for (int i = 0; i < limit; i++) {
assert toEntry[i] >= 0;
w.add(toEntry[i]);
}
}
for (int i = limit; i < docCount; i++) {
w.add(0);
}
w.finish();
}
protected void writeIndex(IndexOutput idxOut, int docCount,
long maxValue, long[] addresses, int[] toEntry) throws IOException {
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
PackedInts.bitsRequired(maxValue), acceptableOverheadRatio);
final int limit = docCount > docToEntry.length ? docToEntry.length
: docCount;
assert toEntry.length >= limit -1;
if (addresses != null) {
for (int i = 0; i < limit; i++) {
assert addresses[toEntry[i]] >= 0;
w.add(addresses[toEntry[i]]);
}
} else {
for (int i = 0; i < limit; i++) {
assert toEntry[i] >= 0;
w.add(toEntry[i]);
}
}
for (int i = limit; i < docCount; i++) {
w.add(0);
}
w.finish();
}
}
static abstract class BytesSortedSourceBase extends SortedSource {
private final PagedBytes pagedBytes;
protected final PackedInts.Reader docToOrdIndex;
protected final PackedInts.Reader ordToOffsetIndex;
protected final IndexInput datIn;
protected final IndexInput idxIn;
protected final BytesRef defaultValue = new BytesRef();
protected final static int PAGED_BYTES_BITS = 15;
protected final PagedBytes.Reader data;
protected BytesSortedSourceBase(IndexInput datIn, IndexInput idxIn,
Comparator<BytesRef> comp, long bytesToRead, Type type, boolean hasOffsets) throws IOException {
this(datIn, idxIn, comp, new PagedBytes(PAGED_BYTES_BITS), bytesToRead, type, hasOffsets);
}
protected BytesSortedSourceBase(IndexInput datIn, IndexInput idxIn,
Comparator<BytesRef> comp, PagedBytes pagedBytes, long bytesToRead, Type type, boolean hasOffsets)
throws IOException {
super(type, comp);
assert bytesToRead <= datIn.length() : " file size is less than the expected size diff: "
+ (bytesToRead - datIn.length()) + " pos: " + datIn.getFilePointer();
this.datIn = datIn;
this.pagedBytes = pagedBytes;
this.pagedBytes.copy(datIn, bytesToRead);
data = pagedBytes.freeze(true);
this.idxIn = idxIn;
ordToOffsetIndex = hasOffsets ? PackedInts.getReader(idxIn) : null;
docToOrdIndex = PackedInts.getReader(idxIn);
}
@Override
public boolean hasPackedDocToOrd() {
return true;
}
@Override
public PackedInts.Reader getDocToOrd() {
return docToOrdIndex;
}
@Override
public int ord(int docID) {
assert docToOrdIndex.get(docID) < getValueCount();
return (int) docToOrdIndex.get(docID);
}
protected void closeIndexInput() throws IOException {
IOUtils.close(datIn, idxIn);
}
}
}

View File

@ -1,164 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
/**
* Base class for disk resident source implementations
* @lucene.internal
*/
abstract class DirectSource extends Source {
protected final IndexInput data;
private final ToNumeric toNumeric;
protected final long baseOffset;
public DirectSource(IndexInput input, Type type) {
super(type);
this.data = input;
baseOffset = input.getFilePointer();
switch (type) {
case FIXED_INTS_16:
toNumeric = new ShortToLong();
break;
case FLOAT_32:
toNumeric = new BytesToFloat();
break;
case FLOAT_64:
toNumeric = new BytesToDouble();
break;
case FIXED_INTS_32:
toNumeric = new IntToLong();
break;
case FIXED_INTS_8:
toNumeric = new ByteToLong();
break;
default:
toNumeric = new LongToLong();
}
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
try {
final int sizeToRead = position(docID);
ref.offset = 0;
ref.grow(sizeToRead);
data.readBytes(ref.bytes, 0, sizeToRead);
ref.length = sizeToRead;
return ref;
} catch (IOException ex) {
throw new IllegalStateException("failed to get value for docID: " + docID, ex);
}
}
@Override
public long getInt(int docID) {
try {
position(docID);
return toNumeric.toLong(data);
} catch (IOException ex) {
throw new IllegalStateException("failed to get value for docID: " + docID, ex);
}
}
@Override
public double getFloat(int docID) {
try {
position(docID);
return toNumeric.toDouble(data);
} catch (IOException ex) {
throw new IllegalStateException("failed to get value for docID: " + docID, ex);
}
}
protected abstract int position(int docID) throws IOException;
private abstract static class ToNumeric {
abstract long toLong(IndexInput input) throws IOException;
double toDouble(IndexInput input) throws IOException {
return toLong(input);
}
}
private static final class ByteToLong extends ToNumeric {
@Override
long toLong(IndexInput input) throws IOException {
return input.readByte();
}
}
private static final class ShortToLong extends ToNumeric {
@Override
long toLong(IndexInput input) throws IOException {
return input.readShort();
}
}
private static final class IntToLong extends ToNumeric {
@Override
long toLong(IndexInput input) throws IOException {
return input.readInt();
}
}
private static final class BytesToFloat extends ToNumeric {
@Override
long toLong(IndexInput input) {
throw new UnsupportedOperationException("ints are not supported");
}
@Override
double toDouble(IndexInput input) throws IOException {
return Float.intBitsToFloat(input.readInt());
}
}
private static final class BytesToDouble extends ToNumeric {
@Override
long toLong(IndexInput input) {
throw new UnsupportedOperationException("ints are not supported");
}
@Override
double toDouble(IndexInput input) throws IOException {
return Double.longBitsToDouble(input.readLong());
}
}
private static final class LongToLong extends ToNumeric {
@Override
long toLong(IndexInput input) throws IOException {
return input.readLong();
}
@Override
double toDouble(IndexInput input) {
throw new UnsupportedOperationException("doubles are not supported");
}
}
}

View File

@ -1,107 +0,0 @@
package org.apache.lucene.codecs.lucene40.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 java.util.Comparator;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.lucene40.values.Writer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.DocValues.Type; // javadoc
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.packed.PackedInts;
/**
* Abstract base class for PerDocConsumer implementations
*
* @lucene.experimental
*/
public abstract class DocValuesWriterBase extends PerDocConsumer {
/** Segment name to use when writing files. */
protected final String segmentName;
private final Counter bytesUsed;
/** {@link IOContext} to use when writing files. */
protected final IOContext context;
private final float acceptableOverheadRatio;
/**
* Filename extension for index files
*/
public static final String INDEX_EXTENSION = "idx";
/**
* Filename extension for data files.
*/
public static final String DATA_EXTENSION = "dat";
/**
* Creates {@code DocValuesWriterBase}, using {@link
* PackedInts#FAST}.
* @param state The state to initiate a {@link PerDocConsumer} instance
*/
protected DocValuesWriterBase(PerDocWriteState state) {
this(state, PackedInts.FAST);
}
/**
* Creates {@code DocValuesWriterBase}.
* @param state The state to initiate a {@link PerDocConsumer} instance
* @param acceptableOverheadRatio
* how to trade space for speed. This option is only applicable for
* docvalues of type {@link Type#BYTES_FIXED_SORTED} and
* {@link Type#BYTES_VAR_SORTED}.
* @see PackedInts#getReader(org.apache.lucene.store.DataInput)
*/
protected DocValuesWriterBase(PerDocWriteState state, float acceptableOverheadRatio) {
this.segmentName = state.segmentInfo.name;
this.bytesUsed = state.bytesUsed;
this.context = state.context;
this.acceptableOverheadRatio = acceptableOverheadRatio;
}
/** Returns the {@link Directory} that files should be
* written to. */
protected abstract Directory getDirectory() throws IOException;
@Override
public void close() throws IOException {
}
@Override
public DocValuesConsumer addValuesField(Type valueType, FieldInfo field) throws IOException {
return Writer.create(valueType,
PerDocProducerBase.docValuesId(segmentName, field.number),
getDirectory(), getComparator(), bytesUsed, context, acceptableOverheadRatio);
}
/** Returns the comparator used to sort {@link BytesRef}
* values. */
public Comparator<BytesRef> getComparator() throws IOException {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
}

View File

@ -1,135 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts;
// Stores fixed-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[]
/**
* @lucene.experimental
*/
class FixedDerefBytesImpl {
static final String CODEC_NAME_IDX = "FixedDerefBytesIdx";
static final String CODEC_NAME_DAT = "FixedDerefBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
public static class Writer extends DerefBytesWriterBase {
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, Type.BYTES_FIXED_DEREF);
}
@Override
protected void finishInternal(int docCount) throws IOException {
final int numValues = hash.size();
final IndexOutput datOut = getOrCreateDataOut();
datOut.writeInt(size);
if (size != -1) {
final BytesRef bytesRef = new BytesRef(size);
for (int i = 0; i < numValues; i++) {
hash.get(i, bytesRef);
datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
}
}
final IndexOutput idxOut = getOrCreateIndexOut();
idxOut.writeInt(numValues);
writeIndex(idxOut, docCount, numValues, docToEntry);
}
}
public static class FixedDerefReader extends BytesReaderBase {
private final int size;
private final int numValuesStored;
FixedDerefReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, Type.BYTES_FIXED_DEREF);
size = datIn.readInt();
numValuesStored = idxIn.readInt();
}
@Override
protected Source loadSource() throws IOException {
return new FixedDerefSource(cloneData(), cloneIndex(), size, numValuesStored);
}
@Override
protected Source loadDirectSource()
throws IOException {
return new DirectFixedDerefSource(cloneData(), cloneIndex(), size, getType());
}
@Override
public int getValueSize() {
return size;
}
}
static final class FixedDerefSource extends BytesSourceBase {
private final int size;
private final PackedInts.Reader addresses;
protected FixedDerefSource(IndexInput datIn, IndexInput idxIn, int size, long numValues) throws IOException {
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * numValues,
Type.BYTES_FIXED_DEREF);
this.size = size;
addresses = PackedInts.getReader(idxIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
return data.fillSlice(bytesRef, addresses.get(docID) * size, size);
}
}
final static class DirectFixedDerefSource extends DirectSource {
private final PackedInts.Reader index;
private final int size;
DirectFixedDerefSource(IndexInput data, IndexInput index, int size, Type type)
throws IOException {
super(data, type);
this.size = size;
this.index = PackedInts.getDirectReader(index);
}
@Override
protected int position(int docID) throws IOException {
data.seek(baseOffset + index.get(docID) * size);
return size;
}
}
}

View File

@ -1,232 +0,0 @@
package org.apache.lucene.codecs.lucene40.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 java.util.Comparator;
import java.util.List;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSortedSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.SortedBytesMergeUtils;
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.SortedBytesMergeUtils.IndexOutputBytesRefConsumer;
import org.apache.lucene.index.SortedBytesMergeUtils.MergeContext;
import org.apache.lucene.index.SortedBytesMergeUtils.SortedSourceSlice;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
// Stores fixed-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[]
/**
* @lucene.experimental
*/
class FixedSortedBytesImpl {
static final String CODEC_NAME_IDX = "FixedSortedBytesIdx";
static final String CODEC_NAME_DAT = "FixedSortedBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static final class Writer extends DerefBytesWriterBase {
private final Comparator<BytesRef> comp;
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Counter bytesUsed, IOContext context, float acceptableOverheadRatio) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, acceptableOverheadRatio, Type.BYTES_FIXED_SORTED);
this.comp = comp;
}
@Override
public void merge(MergeState mergeState, DocValues[] docValues)
throws IOException {
boolean success = false;
try {
final MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_FIXED_SORTED, docValues, comp, mergeState.segmentInfo.getDocCount());
List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState.docBase, mergeState.docMaps, docValues, ctx);
final IndexOutput datOut = getOrCreateDataOut();
datOut.writeInt(ctx.sizePerValues);
final int maxOrd = SortedBytesMergeUtils.mergeRecords(ctx, new IndexOutputBytesRefConsumer(datOut), slices);
final IndexOutput idxOut = getOrCreateIndexOut();
idxOut.writeInt(maxOrd);
final PackedInts.Writer ordsWriter = PackedInts.getWriter(idxOut, ctx.docToEntry.length,
PackedInts.bitsRequired(maxOrd), PackedInts.DEFAULT);
for (SortedSourceSlice slice : slices) {
slice.writeOrds(ordsWriter);
}
ordsWriter.finish();
success = true;
} finally {
releaseResources();
if (success) {
IOUtils.close(getIndexOut(), getDataOut());
} else {
IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
}
}
}
// Important that we get docCount, in case there were
// some last docs that we didn't see
@Override
public void finishInternal(int docCount) throws IOException {
fillDefault(docCount);
final IndexOutput datOut = getOrCreateDataOut();
final int count = hash.size();
final int[] address = new int[count];
datOut.writeInt(size);
if (size != -1) {
final int[] sortedEntries = hash.sort(comp);
// first dump bytes data, recording address as we go
final BytesRef spare = new BytesRef(size);
for (int i = 0; i < count; i++) {
final int e = sortedEntries[i];
final BytesRef bytes = hash.get(e, spare);
assert bytes.length == size;
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
address[e] = i;
}
}
final IndexOutput idxOut = getOrCreateIndexOut();
idxOut.writeInt(count);
writeIndex(idxOut, docCount, count, address, docToEntry);
}
}
static final class Reader extends BytesReaderBase {
private final int size;
private final int valueCount;
private final Comparator<BytesRef> comparator;
public Reader(Directory dir, String id, int maxDoc, IOContext context,
Type type, Comparator<BytesRef> comparator) throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, type);
size = datIn.readInt();
valueCount = idxIn.readInt();
this.comparator = comparator;
}
@Override
protected Source loadSource() throws IOException {
return new FixedSortedSource(cloneData(), cloneIndex(), size, valueCount,
comparator);
}
@Override
protected Source loadDirectSource() throws IOException {
return new DirectFixedSortedSource(cloneData(), cloneIndex(), size,
valueCount, comparator, type);
}
@Override
public int getValueSize() {
return size;
}
}
static final class FixedSortedSource extends BytesSortedSourceBase {
private final int valueCount;
private final int size;
FixedSortedSource(IndexInput datIn, IndexInput idxIn, int size,
int numValues, Comparator<BytesRef> comp) throws IOException {
super(datIn, idxIn, comp, size * numValues, Type.BYTES_FIXED_SORTED,
false);
this.size = size;
this.valueCount = numValues;
closeIndexInput();
}
@Override
public int getValueCount() {
return valueCount;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
return data.fillSlice(bytesRef, (ord * size), size);
}
}
static final class DirectFixedSortedSource extends SortedSource {
final PackedInts.Reader docToOrdIndex;
private final IndexInput datIn;
private final long basePointer;
private final int size;
private final int valueCount;
DirectFixedSortedSource(IndexInput datIn, IndexInput idxIn, int size,
int valueCount, Comparator<BytesRef> comp, Type type)
throws IOException {
super(type, comp);
docToOrdIndex = PackedInts.getDirectReader(idxIn);
basePointer = datIn.getFilePointer();
this.datIn = datIn;
this.size = size;
this.valueCount = valueCount;
}
@Override
public int ord(int docID) {
return (int) docToOrdIndex.get(docID);
}
@Override
public boolean hasPackedDocToOrd() {
return true;
}
@Override
public PackedInts.Reader getDocToOrd() {
return docToOrdIndex;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
try {
datIn.seek(basePointer + size * ord);
bytesRef.offset = 0;
bytesRef.grow(size);
datIn.readBytes(bytesRef.bytes, 0, size);
bytesRef.length = size;
return bytesRef;
} catch (IOException ex) {
throw new IllegalStateException("failed to getByOrd", ex);
}
}
@Override
public int getValueCount() {
return valueCount;
}
}
}

View File

@ -1,369 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesWriterBase;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StraightBytesDocValuesField;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
// Simplest storage: stores fixed length byte[] per
// document, with no dedup and no sorting.
/**
* @lucene.experimental
*/
class FixedStraightBytesImpl {
static final String CODEC_NAME = "FixedStraightBytes";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static abstract class FixedBytesWriterBase extends BytesWriterBase {
protected final StraightBytesDocValuesField bytesSpareField = new StraightBytesDocValuesField("", new BytesRef(), true);
protected int lastDocID = -1;
// start at -1 if the first added value is > 0
protected int size = -1;
private final int byteBlockSize = BYTE_BLOCK_SIZE;
private final ByteBlockPool pool;
protected FixedBytesWriterBase(Directory dir, String id, String codecNameDat,
int version, Counter bytesUsed, IOContext context) {
this(dir, id, codecNameDat, version, bytesUsed, context, Type.BYTES_FIXED_STRAIGHT);
}
protected FixedBytesWriterBase(Directory dir, String id, String codecNameDat,
int version, Counter bytesUsed, IOContext context, Type type) {
super(dir, id, null, codecNameDat, version, bytesUsed, context, type);
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
pool.nextBuffer();
}
@Override
public void add(int docID, StorableField value) throws IOException {
final BytesRef bytes = value.binaryValue();
assert bytes != null;
assert lastDocID < docID;
if (size == -1) {
if (bytes.length > BYTE_BLOCK_SIZE) {
throw new IllegalArgumentException("bytes arrays > " + BYTE_BLOCK_SIZE + " are not supported");
}
size = bytes.length;
} else if (bytes.length != size) {
throw new IllegalArgumentException("byte[] length changed for BYTES_FIXED_STRAIGHT type (before=" + size + " now=" + bytes.length);
}
if (lastDocID+1 < docID) {
advancePool(docID);
}
pool.copy(bytes);
lastDocID = docID;
}
private final void advancePool(int docID) {
long numBytes = (docID - (lastDocID+1))*size;
while(numBytes > 0) {
if (numBytes + pool.byteUpto < byteBlockSize) {
pool.byteUpto += numBytes;
numBytes = 0;
} else {
numBytes -= byteBlockSize - pool.byteUpto;
pool.nextBuffer();
}
}
assert numBytes == 0;
}
protected void set(BytesRef ref, int docId) {
assert BYTE_BLOCK_SIZE % size == 0 : "BYTE_BLOCK_SIZE ("+ BYTE_BLOCK_SIZE + ") must be a multiple of the size: " + size;
ref.offset = docId*size;
ref.length = size;
pool.deref(ref);
}
protected void resetPool() {
pool.reset(false, false);
}
protected void writeData(IndexOutput out) throws IOException {
pool.writePool(out);
}
protected void writeZeros(int num, IndexOutput out) throws IOException {
final byte[] zeros = new byte[size];
for (int i = 0; i < num; i++) {
out.writeBytes(zeros, zeros.length);
}
}
@Override
public int getValueSize() {
return size;
}
}
static class Writer extends FixedBytesWriterBase {
private boolean hasMerged;
private IndexOutput datOut;
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
}
public Writer(Directory dir, String id, String codecNameDat, int version, Counter bytesUsed, IOContext context) {
super(dir, id, codecNameDat, version, bytesUsed, context);
}
@Override
protected void merge(DocValues readerIn, int docBase, int docCount, Bits liveDocs) throws IOException {
datOut = getOrCreateDataOut();
boolean success = false;
try {
if (!hasMerged && size != -1) {
datOut.writeInt(size);
}
if (liveDocs == null && tryBulkMerge(readerIn)) {
FixedStraightReader reader = (FixedStraightReader) readerIn;
final int maxDocs = reader.maxDoc;
if (maxDocs == 0) {
return;
}
if (size == -1) {
size = reader.size;
datOut.writeInt(size);
} else if (size != reader.size) {
throw new IllegalArgumentException("expected bytes size=" + size
+ " but got " + reader.size);
}
if (lastDocID+1 < docBase) {
fill(datOut, docBase);
lastDocID = docBase-1;
}
// TODO should we add a transfer to API to each reader?
final IndexInput cloneData = reader.cloneData();
try {
datOut.copyBytes(cloneData, size * maxDocs);
} finally {
IOUtils.close(cloneData);
}
lastDocID += maxDocs;
} else {
super.merge(readerIn, docBase, docCount, liveDocs);
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(datOut);
}
hasMerged = true;
}
}
protected boolean tryBulkMerge(DocValues docValues) {
return docValues instanceof FixedStraightReader;
}
@Override
protected void mergeDoc(StoredField scratchField, Source source, int docID, int sourceDoc) throws IOException {
assert lastDocID < docID;
setMergeBytes(source, sourceDoc);
if (size == -1) {
size = bytesRef.length;
datOut.writeInt(size);
}
assert size == bytesRef.length : "size: " + size + " ref: " + bytesRef.length;
if (lastDocID+1 < docID) {
fill(datOut, docID);
}
datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
lastDocID = docID;
}
protected void setMergeBytes(Source source, int sourceDoc) {
source.getBytes(sourceDoc, bytesRef);
}
// Fills up to but not including this docID
private void fill(IndexOutput datOut, int docID) throws IOException {
assert size >= 0;
writeZeros((docID - (lastDocID+1)), datOut);
}
@Override
public void finish(int docCount) throws IOException {
boolean success = false;
try {
if (!hasMerged) {
// indexing path - no disk IO until here
assert datOut == null;
datOut = getOrCreateDataOut();
if (size == -1) {
datOut.writeInt(0);
} else {
datOut.writeInt(size);
writeData(datOut);
}
if (lastDocID + 1 < docCount) {
fill(datOut, docCount);
}
} else {
// merge path - datOut should be initialized
assert datOut != null;
if (size == -1) {// no data added
datOut.writeInt(0);
} else {
fill(datOut, docCount);
}
}
success = true;
} finally {
resetPool();
if (success) {
IOUtils.close(datOut);
} else {
IOUtils.closeWhileHandlingException(datOut);
}
}
}
}
public static class FixedStraightReader extends BytesReaderBase {
protected final int size;
protected final int maxDoc;
FixedStraightReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
this(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, Type.BYTES_FIXED_STRAIGHT);
}
protected FixedStraightReader(Directory dir, String id, String codecNameDat, int version, int maxDoc, IOContext context, Type type) throws IOException {
super(dir, id, null, codecNameDat, version, false, context, type);
size = datIn.readInt();
this.maxDoc = maxDoc;
}
@Override
protected Source loadSource() throws IOException {
return size == 1 ? new SingleByteSource(cloneData(), maxDoc) :
new FixedStraightSource(cloneData(), size, maxDoc, type);
}
@Override
public void close() throws IOException {
datIn.close();
}
@Override
protected Source loadDirectSource() throws IOException {
return new DirectFixedStraightSource(cloneData(), size, getType());
}
@Override
public int getValueSize() {
return size;
}
}
// specialized version for single bytes
private static final class SingleByteSource extends Source {
private final byte[] data;
public SingleByteSource(IndexInput datIn, int maxDoc) throws IOException {
super(Type.BYTES_FIXED_STRAIGHT);
try {
data = new byte[maxDoc];
datIn.readBytes(data, 0, data.length, false);
} finally {
IOUtils.close(datIn);
}
}
@Override
public boolean hasArray() {
return true;
}
@Override
public Object getArray() {
return data;
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
bytesRef.length = 1;
bytesRef.bytes = data;
bytesRef.offset = docID;
return bytesRef;
}
}
private final static class FixedStraightSource extends BytesSourceBase {
private final int size;
public FixedStraightSource(IndexInput datIn, int size, int maxDoc, Type type)
throws IOException {
super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc,
type);
this.size = size;
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
return data.fillSlice(bytesRef, size * ((long) docID), size);
}
}
public final static class DirectFixedStraightSource extends DirectSource {
private final int size;
DirectFixedStraightSource(IndexInput input, int size, Type type) {
super(input, type);
this.size = size;
}
@Override
protected int position(int docID) throws IOException {
data.seek(baseOffset + size * ((long) docID));
return size;
}
}
}

View File

@ -1,139 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.DocValuesArraySource;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
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.Counter;
import org.apache.lucene.util.IOUtils;
/**
* Exposes {@link Writer} and reader ({@link Source}) for 32 bit and 64 bit
* floating point values.
* <p>
* Current implementations store either 4 byte or 8 byte floating points with
* full precision without any compression.
*
* @lucene.experimental
*/
public class Floats {
/** Codec name, written in the header. */
protected static final String CODEC_NAME = "Floats";
/** Initial version. */
protected static final int VERSION_START = 0;
/** Current version. */
protected static final int VERSION_CURRENT = VERSION_START;
/** Sole constructor. */
private Floats() {
}
/** Creates and returns a {@link DocValuesConsumer} to
* write float values. */
public static DocValuesConsumer getWriter(Directory dir, String id, Counter bytesUsed,
IOContext context, Type type) {
return new FloatsWriter(dir, id, bytesUsed, context, type);
}
/** Creates and returns a {@link DocValues} to
* read previously written float values. */
public static DocValues getValues(Directory dir, String id, int maxDoc, IOContext context, Type type)
throws IOException {
return new FloatsReader(dir, id, maxDoc, context, type);
}
private static int typeToSize(Type type) {
switch (type) {
case FLOAT_32:
return 4;
case FLOAT_64:
return 8;
default:
throw new IllegalStateException("illegal type " + type);
}
}
final static class FloatsWriter extends FixedStraightBytesImpl.Writer {
private final int size;
private final DocValuesArraySource template;
public FloatsWriter(Directory dir, String id, Counter bytesUsed,
IOContext context, Type type) {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
size = typeToSize(type);
this.bytesRef = new BytesRef(size);
bytesRef.length = size;
template = DocValuesArraySource.forType(type);
assert template != null;
}
@Override
protected boolean tryBulkMerge(DocValues docValues) {
// only bulk merge if value type is the same otherwise size differs
return super.tryBulkMerge(docValues) && docValues.getType() == template.getType();
}
@Override
public void add(int docID, StorableField value) throws IOException {
template.toBytes(value.numericValue().doubleValue(), bytesRef);
bytesSpareField.setBytesValue(bytesRef);
super.add(docID, bytesSpareField);
}
@Override
protected void setMergeBytes(Source source, int sourceDoc) {
final double value = source.getFloat(sourceDoc);
template.toBytes(value, bytesRef);
}
}
final static class FloatsReader extends FixedStraightBytesImpl.FixedStraightReader {
final DocValuesArraySource arrayTemplate;
FloatsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, type);
arrayTemplate = DocValuesArraySource.forType(type);
assert size == 4 || size == 8: "wrong size=" + size + " type=" + type + " id=" + id;
}
@Override
protected Source loadSource() throws IOException {
final IndexInput indexInput = cloneData();
try {
return arrayTemplate.newFromInput(indexInput, maxDoc);
} finally {
IOUtils.close(indexInput);
}
}
}
}

View File

@ -1,161 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.DocValuesArraySource;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
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.Counter;
import org.apache.lucene.util.IOUtils;
/**
* Stores ints packed and fixed with fixed-bit precision.
*
* @lucene.experimental
*/
public final class Ints {
/** Codec name, written in the header. */
protected static final String CODEC_NAME = "Ints";
/** Initial version. */
protected static final int VERSION_START = 0;
/** Current version. */
protected static final int VERSION_CURRENT = VERSION_START;
/** Sole constructor. */
private Ints() {
}
/** Creates and returns a {@link DocValuesConsumer} to
* write int values. */
public static DocValuesConsumer getWriter(Directory dir, String id, Counter bytesUsed,
Type type, IOContext context) {
return type == Type.VAR_INTS ? new PackedIntValues.PackedIntsWriter(dir, id,
bytesUsed, context) : new IntsWriter(dir, id, bytesUsed, context, type);
}
/** Creates and returns a {@link DocValues} to
* read previously written int values. */
public static DocValues getValues(Directory dir, String id, int numDocs,
Type type, IOContext context) throws IOException {
return type == Type.VAR_INTS ? new PackedIntValues.PackedIntsReader(dir, id,
numDocs, context) : new IntsReader(dir, id, numDocs, context, type);
}
private static Type sizeToType(int size) {
switch (size) {
case 1:
return Type.FIXED_INTS_8;
case 2:
return Type.FIXED_INTS_16;
case 4:
return Type.FIXED_INTS_32;
case 8:
return Type.FIXED_INTS_64;
default:
throw new IllegalStateException("illegal size " + size);
}
}
private static int typeToSize(Type type) {
switch (type) {
case FIXED_INTS_16:
return 2;
case FIXED_INTS_32:
return 4;
case FIXED_INTS_64:
return 8;
case FIXED_INTS_8:
return 1;
default:
throw new IllegalStateException("illegal type " + type);
}
}
static class IntsWriter extends FixedStraightBytesImpl.Writer {
private final DocValuesArraySource template;
public IntsWriter(Directory dir, String id, Counter bytesUsed,
IOContext context, Type valueType) {
this(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, valueType);
}
protected IntsWriter(Directory dir, String id, String codecName,
int version, Counter bytesUsed, IOContext context, Type valueType) {
super(dir, id, codecName, version, bytesUsed, context);
size = typeToSize(valueType);
this.bytesRef = new BytesRef(size);
bytesRef.length = size;
template = DocValuesArraySource.forType(valueType);
}
@Override
protected void setMergeBytes(Source source, int sourceDoc) {
final long value = source.getInt(sourceDoc);
template.toBytes(value, bytesRef);
}
@Override
public void add(int docID, StorableField value) throws IOException {
template.toBytes(value.numericValue().longValue(), bytesRef);
bytesSpareField.setBytesValue(bytesRef);
super.add(docID, bytesSpareField);
}
@Override
protected boolean tryBulkMerge(DocValues docValues) {
// only bulk merge if value type is the same otherwise size differs
return super.tryBulkMerge(docValues) && docValues.getType() == template.getType();
}
}
final static class IntsReader extends FixedStraightBytesImpl.FixedStraightReader {
private final DocValuesArraySource arrayTemplate;
IntsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc,
context, type);
arrayTemplate = DocValuesArraySource.forType(type);
assert arrayTemplate != null;
assert type == sizeToType(size);
}
@Override
protected Source loadSource() throws IOException {
final IndexInput indexInput = cloneData();
try {
return arrayTemplate.newFromInput(indexInput, maxDoc);
} finally {
IOUtils.close(indexInput);
}
}
}
}

View File

@ -1,256 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.DocValuesArraySource;
import org.apache.lucene.codecs.lucene40.values.FixedStraightBytesImpl.FixedBytesWriterBase;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
* Stores integers using {@link PackedInts}
*
* @lucene.experimental
* */
class PackedIntValues {
private static final String CODEC_NAME = "PackedInts";
private static final byte PACKED = 0x00;
private static final byte FIXED_64 = 0x01;
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static class PackedIntsWriter extends FixedBytesWriterBase {
private long minValue;
private long maxValue;
private boolean started;
private int lastDocId = -1;
protected PackedIntsWriter(Directory dir, String id, Counter bytesUsed,
IOContext context) {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.VAR_INTS);
bytesRef = new BytesRef(8);
}
@Override
public void finish(int docCount) throws IOException {
boolean success = false;
final IndexOutput dataOut = getOrCreateDataOut();
try {
if (!started) {
minValue = maxValue = 0;
}
final long delta = maxValue - minValue;
// if we exceed the range of positive longs we must switch to fixed
// ints
if (delta <= (maxValue >= 0 && minValue <= 0 ? Long.MAX_VALUE
: Long.MAX_VALUE - 1) && delta >= 0) {
dataOut.writeByte(PACKED);
writePackedInts(dataOut, docCount);
return; // done
} else {
dataOut.writeByte(FIXED_64);
}
writeData(dataOut);
writeZeros(docCount - (lastDocID + 1), dataOut);
success = true;
} finally {
resetPool();
if (success) {
IOUtils.close(dataOut);
} else {
IOUtils.closeWhileHandlingException(dataOut);
}
}
}
private void writePackedInts(IndexOutput datOut, int docCount) throws IOException {
datOut.writeLong(minValue);
// write a default value to recognize docs without a value for that
// field
final long defaultValue = maxValue >= 0 && minValue <= 0 ? 0 - minValue
: ++maxValue - minValue;
datOut.writeLong(defaultValue);
PackedInts.Writer w = PackedInts.getWriter(datOut, docCount,
PackedInts.bitsRequired(maxValue - minValue), PackedInts.DEFAULT);
for (int i = 0; i < lastDocID + 1; i++) {
set(bytesRef, i);
byte[] bytes = bytesRef.bytes;
int offset = bytesRef.offset;
long asLong =
(((long)(bytes[offset+0] & 0xff) << 56) |
((long)(bytes[offset+1] & 0xff) << 48) |
((long)(bytes[offset+2] & 0xff) << 40) |
((long)(bytes[offset+3] & 0xff) << 32) |
((long)(bytes[offset+4] & 0xff) << 24) |
((long)(bytes[offset+5] & 0xff) << 16) |
((long)(bytes[offset+6] & 0xff) << 8) |
((long)(bytes[offset+7] & 0xff)));
w.add(asLong == 0 ? defaultValue : asLong - minValue);
}
for (int i = lastDocID + 1; i < docCount; i++) {
w.add(defaultValue);
}
w.finish();
}
@Override
public void add(int docID, StorableField docValue) throws IOException {
final long v = docValue.numericValue().longValue();
assert lastDocId < docID;
if (!started) {
started = true;
minValue = maxValue = v;
} else {
if (v < minValue) {
minValue = v;
} else if (v > maxValue) {
maxValue = v;
}
}
lastDocId = docID;
DocValuesArraySource.copyLong(bytesRef, v);
bytesSpareField.setBytesValue(bytesRef);
super.add(docID, bytesSpareField);
}
}
/**
* Opens all necessary files, but does not read any data in until you call
* {@link #loadSource}.
*/
static class PackedIntsReader extends DocValues {
private final IndexInput datIn;
private final byte type;
private final int numDocs;
private final DocValuesArraySource values;
protected PackedIntsReader(Directory dir, String id, int numDocs,
IOContext context) throws IOException {
datIn = dir.openInput(
IndexFileNames.segmentFileName(id, Bytes.DV_SEGMENT_SUFFIX, DocValuesWriterBase.DATA_EXTENSION),
context);
this.numDocs = numDocs;
boolean success = false;
try {
CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
type = datIn.readByte();
values = type == FIXED_64 ? DocValuesArraySource.forType(Type.FIXED_INTS_64) : null;
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(datIn);
}
}
}
/**
* Loads the actual values. You may call this more than once, eg if you
* already previously loaded but then discarded the Source.
*/
@Override
protected Source loadSource() throws IOException {
boolean success = false;
final Source source;
IndexInput input = null;
try {
input = datIn.clone();
if (values == null) {
source = new PackedIntsSource(input, false);
} else {
source = values.newFromInput(input, numDocs);
}
success = true;
return source;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(input, datIn);
}
}
}
@Override
public void close() throws IOException {
super.close();
datIn.close();
}
@Override
public Type getType() {
return Type.VAR_INTS;
}
@Override
protected Source loadDirectSource() throws IOException {
return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource(datIn.clone(), 8, Type.FIXED_INTS_64) : new PackedIntsSource(datIn.clone(), true);
}
}
static class PackedIntsSource extends Source {
private final long minValue;
private final long defaultValue;
private final PackedInts.Reader values;
public PackedIntsSource(IndexInput dataIn, boolean direct) throws IOException {
super(Type.VAR_INTS);
minValue = dataIn.readLong();
defaultValue = dataIn.readLong();
values = direct ? PackedInts.getDirectReader(dataIn) : PackedInts.getReader(dataIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
ref.grow(8);
DocValuesArraySource.copyLong(ref, getInt(docID));
return ref;
}
@Override
public long getInt(int docID) {
// TODO -- can we somehow avoid 2X method calls
// on each get? must push minValue down, and make
// PackedInts implement Ints.Source
assert docID >= 0;
final long value = values.get(docID);
return value == defaultValue ? 0 : minValue + value;
}
}
}

View File

@ -1,152 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts;
// Stores variable-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[] and both
// docs reference that single source
/**
* @lucene.experimental
*/
class VarDerefBytesImpl {
static final String CODEC_NAME_IDX = "VarDerefBytesIdx";
static final String CODEC_NAME_DAT = "VarDerefBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
/*
* TODO: if impls like this are merged we are bound to the amount of memory we
* can store into a BytesRefHash and therefore how much memory a ByteBlockPool
* can address. This is currently limited to 2GB. While we could extend that
* and use 64bit for addressing this still limits us to the existing main
* memory as all distinct bytes will be loaded up into main memory. We could
* move the byte[] writing to #finish(int) and store the bytes in sorted
* order and merge them in a streamed fashion.
*/
static class Writer extends DerefBytesWriterBase {
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, Type.BYTES_VAR_DEREF);
size = 0;
}
@Override
protected void checkSize(BytesRef bytes) {
// allow var bytes sizes
}
// Important that we get docCount, in case there were
// some last docs that we didn't see
@Override
public void finishInternal(int docCount) throws IOException {
fillDefault(docCount);
final int size = hash.size();
final long[] addresses = new long[size];
final IndexOutput datOut = getOrCreateDataOut();
int addr = 0;
final BytesRef bytesRef = new BytesRef();
for (int i = 0; i < size; i++) {
hash.get(i, bytesRef);
addresses[i] = addr;
addr += writePrefixLength(datOut, bytesRef) + bytesRef.length;
datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
}
final IndexOutput idxOut = getOrCreateIndexOut();
// write the max address to read directly on source load
idxOut.writeLong(addr);
writeIndex(idxOut, docCount, addresses[addresses.length-1], addresses, docToEntry);
}
}
public static class VarDerefReader extends BytesReaderBase {
private final long totalBytes;
VarDerefReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, Type.BYTES_VAR_DEREF);
totalBytes = idxIn.readLong();
}
@Override
protected Source loadSource() throws IOException {
return new VarDerefSource(cloneData(), cloneIndex(), totalBytes);
}
@Override
protected Source loadDirectSource()
throws IOException {
return new DirectVarDerefSource(cloneData(), cloneIndex(), getType());
}
}
final static class VarDerefSource extends BytesSourceBase {
private final PackedInts.Reader addresses;
public VarDerefSource(IndexInput datIn, IndexInput idxIn, long totalBytes)
throws IOException {
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), totalBytes,
Type.BYTES_VAR_DEREF);
addresses = PackedInts.getReader(idxIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
return data.fillSliceWithPrefix(bytesRef,
addresses.get(docID));
}
}
final static class DirectVarDerefSource extends DirectSource {
private final PackedInts.Reader index;
DirectVarDerefSource(IndexInput data, IndexInput index, Type type)
throws IOException {
super(data, type);
this.index = PackedInts.getDirectReader(index);
}
@Override
protected int position(int docID) throws IOException {
data.seek(baseOffset + index.get(docID));
final byte sizeByte = data.readByte();
if ((sizeByte & 128) == 0) {
// length is 1 byte
return sizeByte;
} else {
return ((sizeByte & 0x7f) << 8) | ((data.readByte() & 0xff));
}
}
}
}

View File

@ -1,258 +0,0 @@
package org.apache.lucene.codecs.lucene40.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 java.util.Comparator;
import java.util.List;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSortedSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.SortedBytesMergeUtils;
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.SortedBytesMergeUtils.IndexOutputBytesRefConsumer;
import org.apache.lucene.index.SortedBytesMergeUtils.MergeContext;
import org.apache.lucene.index.SortedBytesMergeUtils.SortedSourceSlice;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
// Stores variable-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[] and both
// docs reference that single source
/**
* @lucene.experimental
*/
final class VarSortedBytesImpl {
static final String CODEC_NAME_IDX = "VarDerefBytesIdx";
static final String CODEC_NAME_DAT = "VarDerefBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
final static class Writer extends DerefBytesWriterBase {
private final Comparator<BytesRef> comp;
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Counter bytesUsed, IOContext context, float acceptableOverheadRatio) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, acceptableOverheadRatio, Type.BYTES_VAR_SORTED);
this.comp = comp;
size = 0;
}
@Override
public void merge(MergeState mergeState, DocValues[] docValues)
throws IOException {
boolean success = false;
try {
MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_VAR_SORTED, docValues, comp, mergeState.segmentInfo.getDocCount());
final List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState.docBase, mergeState.docMaps, docValues, ctx);
IndexOutput datOut = getOrCreateDataOut();
ctx.offsets = new long[1];
final int maxOrd = SortedBytesMergeUtils.mergeRecords(ctx, new IndexOutputBytesRefConsumer(datOut), slices);
final long[] offsets = ctx.offsets;
maxBytes = offsets[maxOrd-1];
final IndexOutput idxOut = getOrCreateIndexOut();
idxOut.writeLong(maxBytes);
final PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, maxOrd+1,
PackedInts.bitsRequired(maxBytes), PackedInts.DEFAULT);
offsetWriter.add(0);
for (int i = 0; i < maxOrd; i++) {
offsetWriter.add(offsets[i]);
}
offsetWriter.finish();
final PackedInts.Writer ordsWriter = PackedInts.getWriter(idxOut, ctx.docToEntry.length,
PackedInts.bitsRequired(maxOrd-1), PackedInts.DEFAULT);
for (SortedSourceSlice slice : slices) {
slice.writeOrds(ordsWriter);
}
ordsWriter.finish();
success = true;
} finally {
releaseResources();
if (success) {
IOUtils.close(getIndexOut(), getDataOut());
} else {
IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
}
}
}
@Override
protected void checkSize(BytesRef bytes) {
// allow var bytes sizes
}
// Important that we get docCount, in case there were
// some last docs that we didn't see
@Override
public void finishInternal(int docCount) throws IOException {
fillDefault(docCount);
final int count = hash.size();
final IndexOutput datOut = getOrCreateDataOut();
final IndexOutput idxOut = getOrCreateIndexOut();
long offset = 0;
final int[] index = new int[count];
final int[] sortedEntries = hash.sort(comp);
// total bytes of data
idxOut.writeLong(maxBytes);
PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count+1,
PackedInts.bitsRequired(maxBytes), PackedInts.DEFAULT);
// first dump bytes data, recording index & write offset as
// we go
final BytesRef spare = new BytesRef();
for (int i = 0; i < count; i++) {
final int e = sortedEntries[i];
offsetWriter.add(offset);
index[e] = i;
final BytesRef bytes = hash.get(e, spare);
// TODO: we could prefix code...
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
offset += bytes.length;
}
// write sentinel
offsetWriter.add(offset);
offsetWriter.finish();
// write index
writeIndex(idxOut, docCount, count, index, docToEntry);
}
}
public static class Reader extends BytesReaderBase {
private final Comparator<BytesRef> comparator;
Reader(Directory dir, String id, int maxDoc,
IOContext context, Type type, Comparator<BytesRef> comparator)
throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, type);
this.comparator = comparator;
}
@Override
public org.apache.lucene.index.DocValues.Source loadSource()
throws IOException {
return new VarSortedSource(cloneData(), cloneIndex(), comparator);
}
@Override
protected Source loadDirectSource() throws IOException {
return new DirectSortedSource(cloneData(), cloneIndex(), comparator, getType());
}
}
private static final class VarSortedSource extends BytesSortedSourceBase {
private final int valueCount;
VarSortedSource(IndexInput datIn, IndexInput idxIn,
Comparator<BytesRef> comp) throws IOException {
super(datIn, idxIn, comp, idxIn.readLong(), Type.BYTES_VAR_SORTED, true);
valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
closeIndexInput();
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
final long offset = ordToOffsetIndex.get(ord);
final long nextOffset = ordToOffsetIndex.get(1 + ord);
data.fillSlice(bytesRef, offset, (int) (nextOffset - offset));
return bytesRef;
}
@Override
public int getValueCount() {
return valueCount;
}
}
private static final class DirectSortedSource extends SortedSource {
private final PackedInts.Reader docToOrdIndex;
private final PackedInts.Reader ordToOffsetIndex;
private final IndexInput datIn;
private final long basePointer;
private final int valueCount;
DirectSortedSource(IndexInput datIn, IndexInput idxIn,
Comparator<BytesRef> comparator, Type type) throws IOException {
super(type, comparator);
idxIn.readLong();
ordToOffsetIndex = PackedInts.getDirectReader(idxIn);
valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
// advance this iterator to the end and clone the stream once it points to the docToOrdIndex header
ordToOffsetIndex.get(valueCount);
docToOrdIndex = PackedInts.getDirectReader(idxIn.clone()); // read the ords in to prevent too many random disk seeks
basePointer = datIn.getFilePointer();
this.datIn = datIn;
}
@Override
public int ord(int docID) {
return (int) docToOrdIndex.get(docID);
}
@Override
public boolean hasPackedDocToOrd() {
return true;
}
@Override
public PackedInts.Reader getDocToOrd() {
return docToOrdIndex;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
try {
final long offset = ordToOffsetIndex.get(ord);
// 1+ord is safe because we write a sentinel at the end
final long nextOffset = ordToOffsetIndex.get(1+ord);
datIn.seek(basePointer + offset);
final int length = (int) (nextOffset - offset);
bytesRef.offset = 0;
bytesRef.grow(length);
datIn.readBytes(bytesRef.bytes, 0, length);
bytesRef.length = length;
return bytesRef;
} catch (IOException ex) {
throw new IllegalStateException("failed", ex);
}
}
@Override
public int getValueCount() {
return valueCount;
}
}
}

View File

@ -1,297 +0,0 @@
package org.apache.lucene.codecs.lucene40.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.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesWriterBase;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts.ReaderIterator;
import org.apache.lucene.util.packed.PackedInts;
// Variable length byte[] per document, no sharing
/**
* @lucene.experimental
*/
class VarStraightBytesImpl {
static final String CODEC_NAME_IDX = "VarStraightBytesIdx";
static final String CODEC_NAME_DAT = "VarStraightBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static class Writer extends BytesWriterBase {
private long address;
// start at -1 if the first added value is > 0
private int lastDocID = -1;
private long[] docToAddress;
private final ByteBlockPool pool;
private IndexOutput datOut;
private boolean merge = false;
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, Type.BYTES_VAR_STRAIGHT);
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
docToAddress = new long[1];
pool.nextBuffer(); // init
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
}
// Fills up to but not including this docID
private void fill(final int docID, final long nextAddress) {
if (docID >= docToAddress.length) {
int oldSize = docToAddress.length;
docToAddress = ArrayUtil.grow(docToAddress, 1 + docID);
bytesUsed.addAndGet((docToAddress.length - oldSize)
* RamUsageEstimator.NUM_BYTES_INT);
}
for (int i = lastDocID + 1; i < docID; i++) {
docToAddress[i] = nextAddress;
}
}
@Override
public void add(int docID, StorableField value) throws IOException {
final BytesRef bytes = value.binaryValue();
assert bytes != null;
assert !merge;
if (bytes.length == 0) {
return; // default
}
fill(docID, address);
docToAddress[docID] = address;
pool.copy(bytes);
address += bytes.length;
lastDocID = docID;
}
@Override
protected void merge(DocValues readerIn, int docBase, int docCount, Bits liveDocs) throws IOException {
merge = true;
datOut = getOrCreateDataOut();
boolean success = false;
try {
if (liveDocs == null && readerIn instanceof VarStraightReader) {
// bulk merge since we don't have any deletes
VarStraightReader reader = (VarStraightReader) readerIn;
final int maxDocs = reader.maxDoc;
if (maxDocs == 0) {
return;
}
if (lastDocID+1 < docBase) {
fill(docBase, address);
lastDocID = docBase-1;
}
final long numDataBytes;
final IndexInput cloneIdx = reader.cloneIndex();
try {
numDataBytes = cloneIdx.readVLong();
final ReaderIterator iter = PackedInts.getReaderIterator(cloneIdx, PackedInts.DEFAULT_BUFFER_SIZE);
for (int i = 0; i < maxDocs; i++) {
long offset = iter.next();
++lastDocID;
if (lastDocID >= docToAddress.length) {
int oldSize = docToAddress.length;
docToAddress = ArrayUtil.grow(docToAddress, 1 + lastDocID);
bytesUsed.addAndGet((docToAddress.length - oldSize)
* RamUsageEstimator.NUM_BYTES_INT);
}
docToAddress[lastDocID] = address + offset;
}
address += numDataBytes; // this is the address after all addr pointers are updated
} finally {
IOUtils.close(cloneIdx);
}
final IndexInput cloneData = reader.cloneData();
try {
datOut.copyBytes(cloneData, numDataBytes);
} finally {
IOUtils.close(cloneData);
}
} else {
super.merge(readerIn, docBase, docCount, liveDocs);
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(datOut);
}
}
}
@Override
protected void mergeDoc(StoredField scratchField, Source source, int docID, int sourceDoc) throws IOException {
assert merge;
assert lastDocID < docID;
source.getBytes(sourceDoc, bytesRef);
if (bytesRef.length == 0) {
return; // default
}
fill(docID, address);
datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
docToAddress[docID] = address;
address += bytesRef.length;
lastDocID = docID;
}
@Override
public void finish(int docCount) throws IOException {
boolean success = false;
assert (!merge && datOut == null) || (merge && datOut != null);
final IndexOutput datOut = getOrCreateDataOut();
try {
if (!merge) {
// header is already written in getDataOut()
pool.writePool(datOut);
}
success = true;
} finally {
if (success) {
IOUtils.close(datOut);
} else {
IOUtils.closeWhileHandlingException(datOut);
}
pool.reset(false, false);
}
success = false;
final IndexOutput idxOut = getOrCreateIndexOut();
try {
if (lastDocID == -1) {
idxOut.writeVLong(0);
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount+1,
PackedInts.bitsRequired(0), PackedInts.DEFAULT);
// docCount+1 so we write sentinel
for (int i = 0; i < docCount+1; i++) {
w.add(0);
}
w.finish();
} else {
fill(docCount, address);
idxOut.writeVLong(address);
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount+1,
PackedInts.bitsRequired(address), PackedInts.DEFAULT);
for (int i = 0; i < docCount; i++) {
w.add(docToAddress[i]);
}
// write sentinel
w.add(address);
w.finish();
}
success = true;
} finally {
bytesUsed.addAndGet(-(docToAddress.length)
* RamUsageEstimator.NUM_BYTES_INT);
docToAddress = null;
if (success) {
IOUtils.close(idxOut);
} else {
IOUtils.closeWhileHandlingException(idxOut);
}
}
}
public long ramBytesUsed() {
return bytesUsed.get();
}
@Override
public int getValueSize() {
return -1;
}
}
public static class VarStraightReader extends BytesReaderBase {
final int maxDoc;
VarStraightReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, Type.BYTES_VAR_STRAIGHT);
this.maxDoc = maxDoc;
}
@Override
protected Source loadSource() throws IOException {
return new VarStraightSource(cloneData(), cloneIndex());
}
@Override
protected Source loadDirectSource()
throws IOException {
return new DirectVarStraightSource(cloneData(), cloneIndex(), getType());
}
}
private static final class VarStraightSource extends BytesSourceBase {
private final PackedInts.Reader addresses;
public VarStraightSource(IndexInput datIn, IndexInput idxIn) throws IOException {
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), idxIn.readVLong(),
Type.BYTES_VAR_STRAIGHT);
addresses = PackedInts.getReader(idxIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
final long address = addresses.get(docID);
return data.fillSlice(bytesRef, address,
(int) (addresses.get(docID + 1) - address));
}
}
public final static class DirectVarStraightSource extends DirectSource {
private final PackedInts.Reader index;
DirectVarStraightSource(IndexInput data, IndexInput index, Type type)
throws IOException {
super(data, type);
index.readVLong();
this.index = PackedInts.getDirectReader(index);
}
@Override
protected int position(int docID) throws IOException {
final long offset = index.get(docID);
data.seek(baseOffset + offset);
// Safe to do 1+docID because we write sentinel at the end:
final long nextOffset = index.get(1+docID);
return (int) (nextOffset - offset);
}
}
}

View File

@ -1,126 +0,0 @@
package org.apache.lucene.codecs.lucene40.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 java.util.Comparator;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.packed.PackedInts;
/**
* Abstract API for per-document stored primitive values of type <tt>byte[]</tt>
* , <tt>long</tt> or <tt>double</tt>. The API accepts a single value for each
* document. The underlying storage mechanism, file formats, data-structures and
* representations depend on the actual implementation.
* <p>
* Document IDs passed to this API must always be increasing unless stated
* otherwise.
* </p>
*
* @lucene.experimental
*/
abstract class Writer extends DocValuesConsumer {
protected final Counter bytesUsed;
protected Type type;
/**
* Creates a new {@link Writer}.
*
* @param bytesUsed
* bytes-usage tracking reference used by implementation to track
* internally allocated memory. All tracked bytes must be released
* once {@link #finish(int)} has been called.
*/
protected Writer(Counter bytesUsed, Type type) {
this.bytesUsed = bytesUsed;
this.type = type;
}
@Override
protected Type getType() {
return type;
}
/**
* Factory method to create a {@link Writer} instance for a given type. This
* method returns default implementations for each of the different types
* defined in the {@link Type} enumeration.
*
* @param type
* the {@link Type} to create the {@link Writer} for
* @param id
* the file name id used to create files within the writer.
* @param directory
* the {@link Directory} to create the files from.
* @param bytesUsed
* a byte-usage tracking reference
* @param acceptableOverheadRatio
* how to trade space for speed. This option is only applicable for
* docvalues of type {@link Type#BYTES_FIXED_SORTED} and
* {@link Type#BYTES_VAR_SORTED}.
* @return a new {@link Writer} instance for the given {@link Type}
* @see PackedInts#getReader(org.apache.lucene.store.DataInput)
*/
public static DocValuesConsumer create(Type type, String id, Directory directory,
Comparator<BytesRef> comp, Counter bytesUsed, IOContext context, float acceptableOverheadRatio) {
if (comp == null) {
comp = BytesRef.getUTF8SortedAsUnicodeComparator();
}
switch (type) {
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
return Ints.getWriter(directory, id, bytesUsed, type, context);
case FLOAT_32:
return Floats.getWriter(directory, id, bytesUsed, context, type);
case FLOAT_64:
return Floats.getWriter(directory, id, bytesUsed, context, type);
case BYTES_FIXED_STRAIGHT:
return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, true, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_FIXED_DEREF:
return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, true, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_FIXED_SORTED:
return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, true, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_VAR_STRAIGHT:
return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, false, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_VAR_DEREF:
return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, false, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_VAR_SORTED:
return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, false, comp,
bytesUsed, context, acceptableOverheadRatio);
default:
throw new IllegalArgumentException("Unknown Values: " + type);
}
}
}

View File

@ -20,19 +20,19 @@ package org.apache.lucene.codecs.lucene41;
import java.io.IOException;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.compressing.CompressingStoredFieldsFormat;
import org.apache.lucene.codecs.compressing.CompressionMode;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
@ -64,9 +64,7 @@ public class Lucene41Codec extends Codec {
};
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@ -92,18 +90,13 @@ public class Lucene41Codec extends Codec {
return vectorsFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
public FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@ -111,11 +104,6 @@ public class Lucene41Codec extends Codec {
public final SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
@ -131,5 +119,17 @@ public class Lucene41Codec extends Codec {
return defaultFormat;
}
@Override
public DocValuesFormat docValuesFormat() {
return dvFormat;
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final DocValuesFormat dvFormat = new Lucene40DocValuesFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat();
@Override
public NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -41,7 +41,7 @@ public final class Lucene41PostingsBaseFormat extends PostingsBaseFormat {
@Override
public PostingsReaderBase postingsReaderBase(SegmentReadState state) throws IOException {
return new Lucene41PostingsReader(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
return new Lucene41PostingsReader(state.directory, state.fieldInfos, state.segmentInfo, state.context, state.segmentSuffix);
}
@Override

View File

@ -427,14 +427,14 @@ public final class Lucene41PostingsFormat extends PostingsFormat {
@Override
public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.dir,
PostingsReaderBase postingsReader = new Lucene41PostingsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
state.context,
state.segmentSuffix);
boolean success = false;
try {
FieldsProducer ret = new BlockTreeTermsReader(state.dir,
FieldsProducer ret = new BlockTreeTermsReader(state.directory,
state.fieldInfos,
state.segmentInfo,
postingsReader,

View File

@ -368,11 +368,11 @@ file, previously they were stored in text format only.</li>
frequencies.</li>
<li>In version 4.0, the format of the inverted index became extensible via
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
factors need no longer be a single byte, they can be any DocValues
{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
strings, they can be any byte sequence. Term offsets can optionally be indexed
into the postings lists. Payloads can be stored in the term vectors.</li>
({@code DocValues}) was introduced. Normalization factors need no longer be a
single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
Terms need not be unicode strings, they can be any byte sequence. Term offsets
can optionally be indexed into the postings lists. Payloads can be stored in the
term vectors.</li>
<li>In version 4.1, the format of the postings list changed to use either
of FOR compression or variable-byte encoding, depending upon the frequency
of the term.</li>

View File

@ -27,16 +27,15 @@ import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
import org.apache.lucene.codecs.lucene41.Lucene41StoredFieldsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 4.2 index format, with configurable per-field postings formats.
* Implements the Lucene 4.2 index format, with configurable per-field postings
* and docvalues formats.
* <p>
* If you want to reuse functionality of this codec in another codec, extend
* {@link FilterCodec}.
@ -50,7 +49,7 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
public class Lucene42Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene42TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene42FieldInfosFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
@ -60,6 +59,14 @@ public class Lucene42Codec extends Codec {
return Lucene42Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Lucene42Codec.this.getDocValuesFormatForField(field);
}
};
/** Sole constructor. */
public Lucene42Codec() {
@ -105,15 +112,24 @@ public class Lucene42Codec extends Codec {
return defaultFormat;
}
/** Returns the docvalues format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene42"
*/
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene41");
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene42");
private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final NormsFormat normsFormat = new Lucene42NormsFormat();
@Override
public final NormsFormat normsFormat() {

View File

@ -0,0 +1,220 @@
package org.apache.lucene.codecs.lucene42;
/*
* 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.HashMap;
import java.util.HashSet;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.FST.INPUT_TYPE;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
import org.apache.lucene.util.packed.BlockPackedWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.PackedInts.FormatAndBits;
/**
* Writer for {@link Lucene42DocValuesFormat}
*/
class Lucene42DocValuesConsumer extends DocValuesConsumer {
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static final byte NUMBER = 0;
static final byte BYTES = 1;
static final byte FST = 2;
static final int BLOCK_SIZE = 4096;
static final byte DELTA_COMPRESSED = 0;
static final byte TABLE_COMPRESSED = 1;
static final byte UNCOMPRESSED = 2;
final IndexOutput data, meta;
final int maxDoc;
final float acceptableOverheadRatio;
Lucene42DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, float acceptableOverheadRatio) throws IOException {
this.acceptableOverheadRatio = acceptableOverheadRatio;
maxDoc = state.segmentInfo.getDocCount();
boolean success = false;
try {
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = state.directory.createOutput(dataName, state.context);
CodecUtil.writeHeader(data, dataCodec, VERSION_CURRENT);
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
meta = state.directory.createOutput(metaName, state.context);
CodecUtil.writeHeader(meta, metaCodec, VERSION_CURRENT);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this);
}
}
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(NUMBER);
meta.writeLong(data.getFilePointer());
long minValue = Long.MAX_VALUE;
long maxValue = Long.MIN_VALUE;
// TODO: more efficient?
HashSet<Long> uniqueValues = new HashSet<Long>();
for(Number nv : values) {
long v = nv.longValue();
minValue = Math.min(minValue, v);
maxValue = Math.max(maxValue, v);
if (uniqueValues != null) {
if (uniqueValues.add(v)) {
if (uniqueValues.size() > 256) {
uniqueValues = null;
}
}
}
}
if (uniqueValues != null) {
// small number of unique values
final int bitsPerValue = PackedInts.bitsRequired(uniqueValues.size()-1);
FormatAndBits formatAndBits = PackedInts.fastestFormatAndBits(maxDoc, bitsPerValue, acceptableOverheadRatio);
if (formatAndBits.bitsPerValue == 8 && minValue >= Byte.MIN_VALUE && maxValue <= Byte.MAX_VALUE) {
meta.writeByte(UNCOMPRESSED); // uncompressed
for (Number nv : values) {
data.writeByte((byte) nv.longValue());
}
} else {
meta.writeByte(TABLE_COMPRESSED); // table-compressed
Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
final HashMap<Long,Integer> encode = new HashMap<Long,Integer>();
data.writeVInt(decode.length);
for (int i = 0; i < decode.length; i++) {
data.writeLong(decode[i]);
encode.put(decode[i], i);
}
meta.writeVInt(PackedInts.VERSION_CURRENT);
data.writeVInt(formatAndBits.format.getId());
data.writeVInt(formatAndBits.bitsPerValue);
final PackedInts.Writer writer = PackedInts.getWriterNoHeader(data, formatAndBits.format, maxDoc, formatAndBits.bitsPerValue, PackedInts.DEFAULT_BUFFER_SIZE);
for(Number nv : values) {
writer.add(encode.get(nv.longValue()));
}
writer.finish();
}
} else {
meta.writeByte(DELTA_COMPRESSED); // delta-compressed
meta.writeVInt(PackedInts.VERSION_CURRENT);
data.writeVInt(BLOCK_SIZE);
final BlockPackedWriter writer = new BlockPackedWriter(data, BLOCK_SIZE);
for (Number nv : values) {
writer.add(nv.longValue());
}
writer.finish();
}
}
@Override
public void close() throws IOException {
boolean success = false;
try {
if (meta != null) {
meta.writeVInt(-1); // write EOF marker
}
success = true;
} finally {
if (success) {
IOUtils.close(data, meta);
} else {
IOUtils.closeWhileHandlingException(data, meta);
}
}
}
@Override
public void addBinaryField(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
// write the byte[] data
meta.writeVInt(field.number);
meta.writeByte(BYTES);
int minLength = Integer.MAX_VALUE;
int maxLength = Integer.MIN_VALUE;
final long startFP = data.getFilePointer();
for(BytesRef v : values) {
minLength = Math.min(minLength, v.length);
maxLength = Math.max(maxLength, v.length);
data.writeBytes(v.bytes, v.offset, v.length);
}
meta.writeLong(startFP);
meta.writeLong(data.getFilePointer() - startFP);
meta.writeVInt(minLength);
meta.writeVInt(maxLength);
// if minLength == maxLength, its a fixed-length byte[], we are done (the addresses are implicit)
// otherwise, we need to record the length fields...
if (minLength != maxLength) {
meta.writeVInt(PackedInts.VERSION_CURRENT);
meta.writeVInt(BLOCK_SIZE);
final MonotonicBlockPackedWriter writer = new MonotonicBlockPackedWriter(data, BLOCK_SIZE);
long addr = 0;
for (BytesRef v : values) {
addr += v.length;
writer.add(addr);
}
writer.finish();
}
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
// write the ordinals as numerics
addNumericField(field, docToOrd);
// write the values as FST
meta.writeVInt(field.number);
meta.writeByte(FST);
meta.writeLong(data.getFilePointer());
PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
Builder<Long> builder = new Builder<Long>(INPUT_TYPE.BYTE1, outputs);
IntsRef scratch = new IntsRef();
long ord = 0;
for (BytesRef v : values) {
builder.add(Util.toIntsRef(v, scratch), ord);
ord++;
}
FST<Long> fst = builder.finish();
fst.save(data);
meta.writeVInt((int)ord);
}
}

View File

@ -0,0 +1,134 @@
package org.apache.lucene.codecs.lucene42;
/*
* 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.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
import org.apache.lucene.util.packed.BlockPackedWriter;
/**
* Lucene 4.2 DocValues format.
* <p>
* Encodes the three per-document value types (Numeric,Binary,Sorted) with five basic strategies.
* <p>
* <ul>
* <li>Delta-compressed Numerics: per-document integers written in blocks of 4096. For each block
* the minimum value is encoded, and each entry is a delta from that minimum value.
* <li>Table-compressed Numerics: when the number of unique values is very small, a lookup table
* is written instead. Each per-document entry is instead the ordinal to this table.
* <li>Uncompressed Numerics: when all values would fit into a single byte, and the
* <code>acceptableOverheadRatio</code> would pack values into 8 bits per value anyway, they
* are written as absolute values (with no indirection or packing) for performance.
* <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
* Each document's value can be addressed by maxDoc*length.
* <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
* for each document. The addresses are written in blocks of 4096, with the current absolute
* start for the block, and the average (expected) delta per entry. For each document the
* deviation from the delta (actual - expected) is written.
* <li>Sorted: an FST mapping deduplicated terms to ordinals is written, along with the per-document
* ordinals written using one of the numeric stratgies above.
* </ul>
* <p>
* Files:
* <ol>
* <li><tt>.dvd</tt>: DocValues data</li>
* <li><tt>.dvm</tt>: DocValues metadata</li>
* </ol>
* <ol>
* <li><a name="dvm" id="dvm"></a>
* <p>The DocValues metadata or .dvm file.</p>
* <p>For DocValues field, this stores metadata, such as the offset into the
* DocValues data (.dvd)</p>
* <p>DocValues metadata (.dvm) --&gt; Header,&lt;FieldNumber,EntryType,Entry&gt;<sup>NumFields</sup></p>
* <ul>
* <li>Entry --&gt; NumericEntry | BinaryEntry | SortedEntry</li>
* <li>NumericEntry --&gt; DataOffset,CompressionType,PackedVersion</li>
* <li>BinaryEntry --&gt; DataOffset,DataLength,MinLength,MaxLength,PackedVersion?,BlockSize?</li>
* <li>SortedEntry --&gt; DataOffset,ValueCount</li>
* <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>DataOffset,DataLength --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>EntryType,CompressionType --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* </ul>
* <p>Sorted fields have two entries: a SortedEntry with the FST metadata,
* and an ordinary NumericEntry for the document-to-ord metadata.</p>
* <p>FieldNumber of -1 indicates the end of metadata.</p>
* <p>EntryType is a 0 (NumericEntry), 1 (BinaryEntry, or 2 (SortedEntry)</p>
* <p>DataOffset is the pointer to the start of the data in the DocValues data (.dvd)</p>
* <p>CompressionType indicates how Numeric values will be compressed:
* <ul>
* <li>0 --&gt; delta-compressed. For each block of 4096 integers, every integer is delta-encoded
* from the minimum value within the block.
* <li>1 --&gt; table-compressed. When the number of unique numeric values is small and it would save space,
* a lookup table of unique values is written, followed by the ordinal for each document.
* <li>2 --&gt; uncompressed. When the <code>acceptableOverheadRatio</code> parameter would upgrade the number
* of bits required to 8, and all values fit in a byte, these are written as absolute binary values
* for performance.
* </ul>
* <p>MinLength and MaxLength represent the min and max byte[] value lengths for Binary values.
* If they are equal, then all values are of a fixed size, and can be addressed as DataOffset + (docID * length).
* Otherwise, the binary values are of variable size, and packed integer metadata (PackedVersion,BlockSize)
* is written for the addresses.
* <li><a name="dvd" id="dvd"></a>
* <p>The DocValues data or .dvd file.</p>
* <p>For DocValues field, this stores the actual per-document data (the heavy-lifting)</p>
* <p>DocValues data (.dvd) --&gt; Header,&lt;NumericData | BinaryData | SortedData&gt;<sup>NumFields</sup></p>
* <ul>
* <li>NumericData --&gt; DeltaCompressedNumerics | TableCompressedNumerics | UncompressedNumerics</li>
* <li>BinaryData --&gt; {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
* <li>SortedData --&gt; {@link FST FST&lt;Int64&gt;}</li>
* <li>DeltaCompressedNumerics --&gt; {@link BlockPackedWriter BlockPackedInts(blockSize=4096)}</li>
* <li>TableCompressedNumerics --&gt; TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,{@link PackedInts PackedInts}</li>
* <li>UncompressedNumerics --&gt; {@link DataOutput#writeByte Byte}<sup>maxdoc</sup></li>
* <li>Addresses --&gt; {@link MonotonicBlockPackedWriter MonotonicBlockPackedInts(blockSize=4096)}</li>
* </ul>
* </ol>
*/
public final class Lucene42DocValuesFormat extends DocValuesFormat {
/** Sole constructor */
public Lucene42DocValuesFormat() {
super("Lucene42");
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
// note: we choose DEFAULT here (its reasonably fast, and for small bpv has tiny waste)
return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, PackedInts.DEFAULT);
}
@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene42DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
private static final String DATA_CODEC = "Lucene42DocValuesData";
private static final String DATA_EXTENSION = "dvd";
private static final String METADATA_CODEC = "Lucene42DocValuesMetadata";
private static final String METADATA_EXTENSION = "dvm";
}

View File

@ -0,0 +1,313 @@
package org.apache.lucene.codecs.lucene42;
/*
* 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.HashMap;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.IntsRef;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.BytesRefFSTEnum.InputOutput;
import org.apache.lucene.util.fst.FST;
import org.apache.lucene.util.fst.FST.Arc;
import org.apache.lucene.util.fst.FST.BytesReader;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
import org.apache.lucene.util.packed.BlockPackedReader;
import org.apache.lucene.util.packed.MonotonicBlockPackedReader;
import org.apache.lucene.util.packed.PackedInts;
/**
* Reader for {@link Lucene42DocValuesFormat}
*/
class Lucene42DocValuesProducer extends DocValuesProducer {
// metadata maps (just file pointers and minimal stuff)
private final Map<Integer,NumericEntry> numerics;
private final Map<Integer,BinaryEntry> binaries;
private final Map<Integer,FSTEntry> fsts;
private final IndexInput data;
// ram instances we have already loaded
private final Map<Integer,NumericDocValues> numericInstances =
new HashMap<Integer,NumericDocValues>();
private final Map<Integer,BinaryDocValues> binaryInstances =
new HashMap<Integer,BinaryDocValues>();
private final Map<Integer,FST<Long>> fstInstances =
new HashMap<Integer,FST<Long>>();
private final int maxDoc;
Lucene42DocValuesProducer(SegmentReadState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
maxDoc = state.segmentInfo.getDocCount();
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
// read in the entries from the metadata file.
IndexInput in = state.directory.openInput(metaName, state.context);
boolean success = false;
try {
CodecUtil.checkHeader(in, metaCodec,
Lucene42DocValuesConsumer.VERSION_START,
Lucene42DocValuesConsumer.VERSION_START);
numerics = new HashMap<Integer,NumericEntry>();
binaries = new HashMap<Integer,BinaryEntry>();
fsts = new HashMap<Integer,FSTEntry>();
readFields(in, state.fieldInfos);
success = true;
} finally {
if (success) {
IOUtils.close(in);
} else {
IOUtils.closeWhileHandlingException(in);
}
}
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = state.directory.openInput(dataName, state.context);
CodecUtil.checkHeader(data, dataCodec,
Lucene42DocValuesConsumer.VERSION_START,
Lucene42DocValuesConsumer.VERSION_START);
}
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
int fieldType = meta.readByte();
if (fieldType == Lucene42DocValuesConsumer.NUMBER) {
NumericEntry entry = new NumericEntry();
entry.offset = meta.readLong();
entry.format = meta.readByte();
if (entry.format != Lucene42DocValuesConsumer.UNCOMPRESSED) {
entry.packedIntsVersion = meta.readVInt();
}
numerics.put(fieldNumber, entry);
} else if (fieldType == Lucene42DocValuesConsumer.BYTES) {
BinaryEntry entry = new BinaryEntry();
entry.offset = meta.readLong();
entry.numBytes = meta.readLong();
entry.minLength = meta.readVInt();
entry.maxLength = meta.readVInt();
if (entry.minLength != entry.maxLength) {
entry.packedIntsVersion = meta.readVInt();
entry.blockSize = meta.readVInt();
}
binaries.put(fieldNumber, entry);
} else if (fieldType == Lucene42DocValuesConsumer.FST) {
FSTEntry entry = new FSTEntry();
entry.offset = meta.readLong();
entry.numOrds = meta.readVInt();
fsts.put(fieldNumber, entry);
} else {
throw new CorruptIndexException("invalid entry type: " + fieldType + ", input=" + meta);
}
fieldNumber = meta.readVInt();
}
}
@Override
public synchronized NumericDocValues getNumeric(FieldInfo field) throws IOException {
NumericDocValues instance = numericInstances.get(field.number);
if (instance == null) {
instance = loadNumeric(field);
numericInstances.put(field.number, instance);
}
return instance;
}
private NumericDocValues loadNumeric(FieldInfo field) throws IOException {
NumericEntry entry = numerics.get(field.number);
data.seek(entry.offset);
if (entry.format == Lucene42DocValuesConsumer.TABLE_COMPRESSED) {
int size = data.readVInt();
final long decode[] = new long[size];
for (int i = 0; i < decode.length; i++) {
decode[i] = data.readLong();
}
final int formatID = data.readVInt();
final int bitsPerValue = data.readVInt();
final PackedInts.Reader reader = PackedInts.getReaderNoHeader(data, PackedInts.Format.byId(formatID), entry.packedIntsVersion, maxDoc, bitsPerValue);
return new NumericDocValues() {
@Override
public long get(int docID) {
return decode[(int)reader.get(docID)];
}
};
} else if (entry.format == Lucene42DocValuesConsumer.DELTA_COMPRESSED) {
final int blockSize = data.readVInt();
final BlockPackedReader reader = new BlockPackedReader(data, entry.packedIntsVersion, blockSize, maxDoc, false);
return new NumericDocValues() {
@Override
public long get(int docID) {
return reader.get(docID);
}
};
} else if (entry.format == Lucene42DocValuesConsumer.UNCOMPRESSED) {
final byte bytes[] = new byte[maxDoc];
data.readBytes(bytes, 0, bytes.length);
return new NumericDocValues() {
@Override
public long get(int docID) {
return bytes[docID];
}
};
} else {
throw new IllegalStateException();
}
}
@Override
public synchronized BinaryDocValues getBinary(FieldInfo field) throws IOException {
BinaryDocValues instance = binaryInstances.get(field.number);
if (instance == null) {
instance = loadBinary(field);
binaryInstances.put(field.number, instance);
}
return instance;
}
private BinaryDocValues loadBinary(FieldInfo field) throws IOException {
BinaryEntry entry = binaries.get(field.number);
data.seek(entry.offset);
PagedBytes bytes = new PagedBytes(16);
bytes.copy(data, entry.numBytes);
final PagedBytes.Reader bytesReader = bytes.freeze(true);
if (entry.minLength == entry.maxLength) {
final int fixedLength = entry.minLength;
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
bytesReader.fillSlice(result, fixedLength * (long)docID, fixedLength);
}
};
} else {
final MonotonicBlockPackedReader addresses = new MonotonicBlockPackedReader(data, entry.packedIntsVersion, entry.blockSize, maxDoc, false);
return new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
long startAddress = docID == 0 ? 0 : addresses.get(docID-1);
long endAddress = addresses.get(docID);
bytesReader.fillSlice(result, startAddress, (int) (endAddress - startAddress));
}
};
}
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
final FSTEntry entry = fsts.get(field.number);
FST<Long> instance;
synchronized(this) {
instance = fstInstances.get(field.number);
if (instance == null) {
data.seek(entry.offset);
instance = new FST<Long>(data, PositiveIntOutputs.getSingleton(true));
fstInstances.put(field.number, instance);
}
}
final NumericDocValues docToOrd = getNumeric(field);
final FST<Long> fst = instance;
// per-thread resources
final BytesReader in = fst.getBytesReader();
final Arc<Long> firstArc = new Arc<Long>();
final Arc<Long> scratchArc = new Arc<Long>();
final IntsRef scratchInts = new IntsRef();
final BytesRefFSTEnum<Long> fstEnum = new BytesRefFSTEnum<Long>(fst);
return new SortedDocValues() {
@Override
public int getOrd(int docID) {
return (int) docToOrd.get(docID);
}
@Override
public void lookupOrd(int ord, BytesRef result) {
try {
in.setPosition(0);
fst.getFirstArc(firstArc);
IntsRef output = Util.getByOutput(fst, ord, in, firstArc, scratchArc, scratchInts);
result.bytes = new byte[output.length];
result.offset = 0;
result.length = 0;
Util.toBytesRef(output, result);
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
}
@Override
public int lookupTerm(BytesRef key) {
try {
InputOutput<Long> o = fstEnum.seekCeil(key);
if (o == null) {
return -getValueCount()-1;
} else if (o.input.equals(key)) {
return o.output.intValue();
} else {
return (int) -o.output-1;
}
} catch (IOException bogus) {
throw new RuntimeException(bogus);
}
}
@Override
public int getValueCount() {
return entry.numOrds;
}
};
}
@Override
public void close() throws IOException {
data.close();
}
static class NumericEntry {
long offset;
byte format;
int packedIntsVersion;
}
static class BinaryEntry {
long offset;
long numBytes;
int minLength;
int maxLength;
int packedIntsVersion;
int blockSize;
}
static class FSTEntry {
long offset;
int numOrds;
}
}

View File

@ -0,0 +1,121 @@
package org.apache.lucene.codecs.lucene42;
/*
* 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.FieldInfosFormat;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.index.FieldInfo.DocValuesType; // javadoc
import org.apache.lucene.store.DataOutput; // javadoc
/**
* Lucene 4.2 Field Infos format.
* <p>
* <p>Field names are stored in the field info file, with suffix <tt>.fnm</tt>.</p>
* <p>FieldInfos (.fnm) --&gt; Header,FieldsCount, &lt;FieldName,FieldNumber,
* FieldBits,DocValuesBits,Attributes&gt; <sup>FieldsCount</sup></p>
* <p>Data types:
* <ul>
* <li>Header --&gt; {@link CodecUtil#checkHeader CodecHeader}</li>
* <li>FieldsCount --&gt; {@link DataOutput#writeVInt VInt}</li>
* <li>FieldName --&gt; {@link DataOutput#writeString String}</li>
* <li>FieldBits, DocValuesBits --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>FieldNumber --&gt; {@link DataOutput#writeInt VInt}</li>
* <li>Attributes --&gt; {@link DataOutput#writeStringStringMap Map&lt;String,String&gt;}</li>
* </ul>
* </p>
* Field Descriptions:
* <ul>
* <li>FieldsCount: the number of fields in this file.</li>
* <li>FieldName: name of the field as a UTF-8 String.</li>
* <li>FieldNumber: the field's number. Note that unlike previous versions of
* Lucene, the fields are not numbered implicitly by their order in the
* file, instead explicitly.</li>
* <li>FieldBits: a byte containing field options.
* <ul>
* <li>The low-order bit is one for indexed fields, and zero for non-indexed
* fields.</li>
* <li>The second lowest-order bit is one for fields that have term vectors
* stored, and zero for fields without term vectors.</li>
* <li>If the third lowest order-bit is set (0x4), offsets are stored into
* the postings list in addition to positions.</li>
* <li>Fourth bit is unused.</li>
* <li>If the fifth lowest-order bit is set (0x10), norms are omitted for the
* indexed field.</li>
* <li>If the sixth lowest-order bit is set (0x20), payloads are stored for the
* indexed field.</li>
* <li>If the seventh lowest-order bit is set (0x40), term frequencies and
* positions omitted for the indexed field.</li>
* <li>If the eighth lowest-order bit is set (0x80), positions are omitted for the
* indexed field.</li>
* </ul>
* </li>
* <li>DocValuesBits: a byte containing per-document value types. The type
* recorded as two four-bit integers, with the high-order bits representing
* <code>norms</code> options, and the low-order bits representing
* {@code DocValues} options. Each four-bit integer can be decoded as such:
* <ul>
* <li>0: no DocValues for this field.</li>
* <li>1: NumericDocValues. ({@link DocValuesType#NUMERIC})</li>
* <li>2: BinaryDocValues. ({@code DocValuesType#BINARY})</li>
* <li>3: SortedDocValues. ({@code DocValuesType#SORTED})</li>
* </ul>
* </li>
* <li>Attributes: a key-value map of codec-private attributes.</li>
* </ul>
*
* @lucene.experimental
*/
public final class Lucene42FieldInfosFormat extends FieldInfosFormat {
private final FieldInfosReader reader = new Lucene42FieldInfosReader();
private final FieldInfosWriter writer = new Lucene42FieldInfosWriter();
/** Sole constructor. */
public Lucene42FieldInfosFormat() {
}
@Override
public FieldInfosReader getFieldInfosReader() throws IOException {
return reader;
}
@Override
public FieldInfosWriter getFieldInfosWriter() throws IOException {
return writer;
}
/** Extension of field infos */
static final String EXTENSION = "fnm";
// Codec header
static final String CODEC_NAME = "Lucene42FieldInfos";
static final int FORMAT_START = 0;
static final int FORMAT_CURRENT = FORMAT_START;
// Field flags
static final byte IS_INDEXED = 0x1;
static final byte STORE_TERMVECTOR = 0x2;
static final byte STORE_OFFSETS_IN_POSTINGS = 0x4;
static final byte OMIT_NORMS = 0x10;
static final byte STORE_PAYLOADS = 0x20;
static final byte OMIT_TERM_FREQ_AND_POSITIONS = 0x40;
static final byte OMIT_POSITIONS = -128;
}

View File

@ -0,0 +1,121 @@
package org.apache.lucene.codecs.lucene42;
/*
* 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.Collections;
import java.util.Map;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldInfosReader;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.2 FieldInfos reader.
*
* @lucene.experimental
* @see Lucene42FieldInfosFormat
*/
final class Lucene42FieldInfosReader extends FieldInfosReader {
/** Sole constructor. */
public Lucene42FieldInfosReader() {
}
@Override
public FieldInfos read(Directory directory, String segmentName, IOContext iocontext) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene42FieldInfosFormat.EXTENSION);
IndexInput input = directory.openInput(fileName, iocontext);
boolean success = false;
try {
CodecUtil.checkHeader(input, Lucene42FieldInfosFormat.CODEC_NAME,
Lucene42FieldInfosFormat.FORMAT_START,
Lucene42FieldInfosFormat.FORMAT_CURRENT);
final int size = input.readVInt(); //read in the size
FieldInfo infos[] = new FieldInfo[size];
for (int i = 0; i < size; i++) {
String name = input.readString();
final int fieldNumber = input.readVInt();
byte bits = input.readByte();
boolean isIndexed = (bits & Lucene42FieldInfosFormat.IS_INDEXED) != 0;
boolean storeTermVector = (bits & Lucene42FieldInfosFormat.STORE_TERMVECTOR) != 0;
boolean omitNorms = (bits & Lucene42FieldInfosFormat.OMIT_NORMS) != 0;
boolean storePayloads = (bits & Lucene42FieldInfosFormat.STORE_PAYLOADS) != 0;
final IndexOptions indexOptions;
if (!isIndexed) {
indexOptions = null;
} else if ((bits & Lucene42FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_ONLY;
} else if ((bits & Lucene42FieldInfosFormat.OMIT_POSITIONS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS;
} else if ((bits & Lucene42FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS) != 0) {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS;
} else {
indexOptions = IndexOptions.DOCS_AND_FREQS_AND_POSITIONS;
}
// DV Types are packed in one byte
byte val = input.readByte();
final DocValuesType docValuesType = getDocValuesType(input, (byte) (val & 0x0F));
final DocValuesType normsType = getDocValuesType(input, (byte) ((val >>> 4) & 0x0F));
final Map<String,String> attributes = input.readStringStringMap();
infos[i] = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector,
omitNorms, storePayloads, indexOptions, docValuesType, normsType, Collections.unmodifiableMap(attributes));
}
if (input.getFilePointer() != input.length()) {
throw new CorruptIndexException("did not read all bytes from file \"" + fileName + "\": read " + input.getFilePointer() + " vs size " + input.length() + " (resource: " + input + ")");
}
FieldInfos fieldInfos = new FieldInfos(infos);
success = true;
return fieldInfos;
} finally {
if (success) {
input.close();
} else {
IOUtils.closeWhileHandlingException(input);
}
}
}
private static DocValuesType getDocValuesType(IndexInput input, byte b) throws IOException {
if (b == 0) {
return null;
} else if (b == 1) {
return DocValuesType.NUMERIC;
} else if (b == 2) {
return DocValuesType.BINARY;
} else if (b == 3) {
return DocValuesType.SORTED;
} else {
throw new CorruptIndexException("invalid docvalues byte: " + b + " (resource=" + input + ")");
}
}
}

View File

@ -0,0 +1,106 @@
package org.apache.lucene.codecs.lucene42;
/*
* 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.FieldInfosWriter;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.2 FieldInfos writer.
*
* @see Lucene42FieldInfosFormat
* @lucene.experimental
*/
final class Lucene42FieldInfosWriter extends FieldInfosWriter {
/** Sole constructor. */
public Lucene42FieldInfosWriter() {
}
@Override
public void write(Directory directory, String segmentName, FieldInfos infos, IOContext context) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segmentName, "", Lucene42FieldInfosFormat.EXTENSION);
IndexOutput output = directory.createOutput(fileName, context);
boolean success = false;
try {
CodecUtil.writeHeader(output, Lucene42FieldInfosFormat.CODEC_NAME, Lucene42FieldInfosFormat.FORMAT_CURRENT);
output.writeVInt(infos.size());
for (FieldInfo fi : infos) {
IndexOptions indexOptions = fi.getIndexOptions();
byte bits = 0x0;
if (fi.hasVectors()) bits |= Lucene42FieldInfosFormat.STORE_TERMVECTOR;
if (fi.omitsNorms()) bits |= Lucene42FieldInfosFormat.OMIT_NORMS;
if (fi.hasPayloads()) bits |= Lucene42FieldInfosFormat.STORE_PAYLOADS;
if (fi.isIndexed()) {
bits |= Lucene42FieldInfosFormat.IS_INDEXED;
assert indexOptions.compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0 || !fi.hasPayloads();
if (indexOptions == IndexOptions.DOCS_ONLY) {
bits |= Lucene42FieldInfosFormat.OMIT_TERM_FREQ_AND_POSITIONS;
} else if (indexOptions == IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS) {
bits |= Lucene42FieldInfosFormat.STORE_OFFSETS_IN_POSTINGS;
} else if (indexOptions == IndexOptions.DOCS_AND_FREQS) {
bits |= Lucene42FieldInfosFormat.OMIT_POSITIONS;
}
}
output.writeString(fi.name);
output.writeVInt(fi.number);
output.writeByte(bits);
// pack the DV types in one byte
final byte dv = docValuesByte(fi.getDocValuesType());
final byte nrm = docValuesByte(fi.getNormType());
assert (dv & (~0xF)) == 0 && (nrm & (~0x0F)) == 0;
byte val = (byte) (0xff & ((nrm << 4) | dv));
output.writeByte(val);
output.writeStringStringMap(fi.attributes());
}
success = true;
} finally {
if (success) {
output.close();
} else {
IOUtils.closeWhileHandlingException(output);
}
}
}
private static byte docValuesByte(DocValuesType type) {
if (type == null) {
return 0;
} else if (type == DocValuesType.NUMERIC) {
return 1;
} else if (type == DocValuesType.BINARY) {
return 2;
} else if (type == DocValuesType.SORTED) {
return 3;
} else {
throw new AssertionError();
}
}
}

View File

@ -0,0 +1,64 @@
package org.apache.lucene.codecs.lucene42;
/*
* 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.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.packed.PackedInts;
/**
* Lucene 4.2 score normalization format.
* <p>
* NOTE: this uses the same format as {@link Lucene42DocValuesFormat}
* Numeric DocValues, but with different file extensions, and passing
* {@link PackedInts#FASTEST} for uncompressed encoding: trading off
* space for performance.
* <p>
* Files:
* <ul>
* <li><tt>.nvd</tt>: DocValues data</li>
* <li><tt>.nvm</tt>: DocValues metadata</li>
* </ul>
* @see Lucene42DocValuesFormat
*/
public final class Lucene42NormsFormat extends NormsFormat {
/** Sole constructor */
public Lucene42NormsFormat() {}
@Override
public DocValuesConsumer normsConsumer(SegmentWriteState state) throws IOException {
// note: we choose FASTEST here (otherwise our norms are half as big but 15% slower than previous lucene)
return new Lucene42DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION, PackedInts.FASTEST);
}
@Override
public DocValuesProducer normsProducer(SegmentReadState state) throws IOException {
return new Lucene42DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, METADATA_CODEC, METADATA_EXTENSION);
}
private static final String DATA_CODEC = "Lucene41NormsData";
private static final String DATA_EXTENSION = "nvd";
private static final String METADATA_CODEC = "Lucene41NormsMetadata";
private static final String METADATA_EXTENSION = "nvm";
}

View File

@ -143,7 +143,7 @@ its numbering.</p>
what files it uses,
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Field names}.
{@link org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat Field names}.
This contains the set of field names used in the index.
</li>
<li>
@ -173,7 +173,7 @@ term occurs in each document. Note that this will not exist if all fields in
all documents omit position data.
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Normalization factors}.
{@link org.apache.lucene.codecs.lucene42.Lucene42NormsFormat Normalization factors}.
For each field in each document, a value is stored
that is multiplied into the score for hits on that field.
</li>
@ -185,7 +185,7 @@ term frequency. To add Term Vectors to your index see the
{@link org.apache.lucene.document.Field Field} constructors
</li>
<li>
{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-document values}.
{@link org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat Per-document values}.
Like stored values, these are also keyed by document
number, but are generally intended to be loaded into main memory for fast
access. Whereas stored values are generally intended for summary results from
@ -249,7 +249,7 @@ file.</td>
systems that frequently run out of file handles.</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat Fields}</td>
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42FieldInfosFormat Fields}</td>
<td>.fnm</td>
<td>Stores information about the fields</td>
</tr>
@ -289,13 +289,13 @@ systems that frequently run out of file handles.</td>
<td>Stores additional per-position metadata information such as character offsets and user payloads</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40NormsFormat Norms}</td>
<td>.nrm.cfs, .nrm.cfe</td>
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42NormsFormat Norms}</td>
<td>.nvd, .nvm</td>
<td>Encodes length and boost factors for docs and fields</td>
</tr>
<tr>
<td>{@link org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat Per-Document Values}</td>
<td>.dv.cfs, .dv.cfe</td>
<td>{@link org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat Per-Document Values}</td>
<td>.dvd, .dvm</td>
<td>Encodes additional scoring factors or other per-document information.</td>
</tr>
<tr>
@ -368,11 +368,11 @@ file, previously they were stored in text format only.</li>
frequencies.</li>
<li>In version 4.0, the format of the inverted index became extensible via
the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
({@link org.apache.lucene.index.DocValues DocValues}) was introduced. Normalization
factors need no longer be a single byte, they can be any DocValues
{@link org.apache.lucene.index.DocValues.Type type}. Terms need not be unicode
strings, they can be any byte sequence. Term offsets can optionally be indexed
into the postings lists. Payloads can be stored in the term vectors.</li>
({@code DocValues}) was introduced. Normalization factors need no longer be a
single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
Terms need not be unicode strings, they can be any byte sequence. Term offsets
can optionally be indexed into the postings lists. Payloads can be stored in the
term vectors.</li>
<li>In version 4.1, the format of the postings list changed to use either
of FOR compression or variable-byte encoding, depending upon the frequency
of the term.</li>

View File

@ -0,0 +1,280 @@
package org.apache.lucene.codecs.perfield;
/*
* 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 java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.Map;
import java.util.ServiceLoader; // javadocs
import java.util.TreeMap;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
/**
* Enables per field docvalues support.
* <p>
* Note, when extending this class, the name ({@link #getName}) is
* written into the index. In order for the field to be read, the
* name must resolve to your implementation via {@link #forName(String)}.
* This method uses Java's
* {@link ServiceLoader Service Provider Interface} to resolve format names.
* <p>
* Files written by each docvalues format have an additional suffix containing the
* format name. For example, in a per-field configuration instead of <tt>_1.dat</tt>
* filenames would look like <tt>_1_Lucene40_0.dat</tt>.
* @see ServiceLoader
* @lucene.experimental
*/
public abstract class PerFieldDocValuesFormat extends DocValuesFormat {
/** Name of this {@link PostingsFormat}. */
public static final String PER_FIELD_NAME = "PerFieldDV40";
/** {@link FieldInfo} attribute name used to store the
* format name for each field. */
public static final String PER_FIELD_FORMAT_KEY = PerFieldDocValuesFormat.class.getSimpleName() + ".format";
/** {@link FieldInfo} attribute name used to store the
* segment suffix name for each field. */
public static final String PER_FIELD_SUFFIX_KEY = PerFieldDocValuesFormat.class.getSimpleName() + ".suffix";
/** Sole constructor. */
public PerFieldDocValuesFormat() {
super(PER_FIELD_NAME);
}
@Override
public final DocValuesConsumer fieldsConsumer(SegmentWriteState state)
throws IOException {
return new FieldsWriter(state);
}
static class ConsumerAndSuffix implements Closeable {
DocValuesConsumer consumer;
int suffix;
@Override
public void close() throws IOException {
consumer.close();
}
}
private class FieldsWriter extends DocValuesConsumer {
private final Map<DocValuesFormat,ConsumerAndSuffix> formats = new HashMap<DocValuesFormat,ConsumerAndSuffix>();
private final Map<String,Integer> suffixes = new HashMap<String,Integer>();
private final SegmentWriteState segmentWriteState;
public FieldsWriter(SegmentWriteState state) {
segmentWriteState = state;
}
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
getInstance(field).addNumericField(field, values);
}
@Override
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
getInstance(field).addBinaryField(field, values);
}
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
getInstance(field).addSortedField(field, values, docToOrd);
}
private DocValuesConsumer getInstance(FieldInfo field) throws IOException {
final DocValuesFormat format = getDocValuesFormatForField(field.name);
if (format == null) {
throw new IllegalStateException("invalid null DocValuesFormat for field=\"" + field.name + "\"");
}
final String formatName = format.getName();
String previousValue = field.putAttribute(PER_FIELD_FORMAT_KEY, formatName);
assert previousValue == null: "formatName=" + formatName + " prevValue=" + previousValue;
Integer suffix;
ConsumerAndSuffix consumer = formats.get(format);
if (consumer == null) {
// First time we are seeing this format; create a new instance
// bump the suffix
suffix = suffixes.get(formatName);
if (suffix == null) {
suffix = 0;
} else {
suffix = suffix + 1;
}
suffixes.put(formatName, suffix);
final String segmentSuffix = getFullSegmentSuffix(field.name,
segmentWriteState.segmentSuffix,
getSuffix(formatName, Integer.toString(suffix)));
consumer = new ConsumerAndSuffix();
consumer.consumer = format.fieldsConsumer(new SegmentWriteState(segmentWriteState, segmentSuffix));
consumer.suffix = suffix;
formats.put(format, consumer);
} else {
// we've already seen this format, so just grab its suffix
assert suffixes.containsKey(formatName);
suffix = consumer.suffix;
}
previousValue = field.putAttribute(PER_FIELD_SUFFIX_KEY, Integer.toString(suffix));
assert previousValue == null;
// TODO: we should only provide the "slice" of FIS
// that this PF actually sees ...
return consumer.consumer;
}
@Override
public void close() throws IOException {
// Close all subs
IOUtils.close(formats.values());
}
}
static String getSuffix(String formatName, String suffix) {
return formatName + "_" + suffix;
}
static String getFullSegmentSuffix(String fieldName, String outerSegmentSuffix, String segmentSuffix) {
if (outerSegmentSuffix.length() == 0) {
return segmentSuffix;
} else {
// TODO: support embedding; I think it should work but
// we need a test confirm to confirm
// return outerSegmentSuffix + "_" + segmentSuffix;
throw new IllegalStateException("cannot embed PerFieldPostingsFormat inside itself (field \"" + fieldName + "\" returned PerFieldPostingsFormat)");
}
}
private class FieldsReader extends DocValuesProducer {
private final Map<String,DocValuesProducer> fields = new TreeMap<String,DocValuesProducer>();
private final Map<String,DocValuesProducer> formats = new HashMap<String,DocValuesProducer>();
public FieldsReader(final SegmentReadState readState) throws IOException {
// Read _X.per and init each format:
boolean success = false;
try {
// Read field name -> format name
for (FieldInfo fi : readState.fieldInfos) {
if (fi.hasDocValues()) {
final String fieldName = fi.name;
final String formatName = fi.getAttribute(PER_FIELD_FORMAT_KEY);
if (formatName != null) {
// null formatName means the field is in fieldInfos, but has no docvalues!
final String suffix = fi.getAttribute(PER_FIELD_SUFFIX_KEY);
assert suffix != null;
DocValuesFormat format = DocValuesFormat.forName(formatName);
String segmentSuffix = getSuffix(formatName, suffix);
if (!formats.containsKey(segmentSuffix)) {
formats.put(segmentSuffix, format.fieldsProducer(new SegmentReadState(readState, segmentSuffix)));
}
fields.put(fieldName, formats.get(segmentSuffix));
}
}
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(formats.values());
}
}
}
private FieldsReader(FieldsReader other) {
Map<DocValuesProducer,DocValuesProducer> oldToNew = new IdentityHashMap<DocValuesProducer,DocValuesProducer>();
// First clone all formats
for(Map.Entry<String,DocValuesProducer> ent : other.formats.entrySet()) {
DocValuesProducer values = ent.getValue();
formats.put(ent.getKey(), values);
oldToNew.put(ent.getValue(), values);
}
// Then rebuild fields:
for(Map.Entry<String,DocValuesProducer> ent : other.fields.entrySet()) {
DocValuesProducer producer = oldToNew.get(ent.getValue());
assert producer != null;
fields.put(ent.getKey(), producer);
}
}
@Override
public NumericDocValues getNumeric(FieldInfo field) throws IOException {
DocValuesProducer producer = fields.get(field.name);
return producer == null ? null : producer.getNumeric(field);
}
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
DocValuesProducer producer = fields.get(field.name);
return producer == null ? null : producer.getBinary(field);
}
@Override
public SortedDocValues getSorted(FieldInfo field) throws IOException {
DocValuesProducer producer = fields.get(field.name);
return producer == null ? null : producer.getSorted(field);
}
@Override
public void close() throws IOException {
IOUtils.close(formats.values());
}
@Override
public DocValuesProducer clone() {
return new FieldsReader(this);
}
}
@Override
public final DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new FieldsReader(state);
}
/**
* Returns the doc values format that should be used for writing
* new segments of <code>field</code>.
* <p>
* The field to format mapping is written to the index, so
* this method is only invoked when writing, not when reading. */
public abstract DocValuesFormat getDocValuesFormatForField(String field);
}

View File

@ -37,7 +37,7 @@ import org.apache.lucene.index.Terms;
import org.apache.lucene.util.IOUtils;
/**
* Enables per field format support.
* Enables per field postings support.
* <p>
* Note, when extending this class, the name ({@link #getName}) is
* written into the index. In order for the field to be read, the

View File

@ -17,43 +17,47 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.BytesRef;
/**
* Field that stores a per-document {@link BytesRef} value.
* <p>
* Field that stores a per-document <code>short</code> value for scoring,
* sorting or value retrieval. Here's an example usage:
* The values are stored directly with no sharing, which is a good fit when
* the fields don't share (many) values, such as a title field. If values
* may be shared and sorted it's better to use {@link SortedDocValuesField}.
* Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new ShortDocValuesField(name, (short) 22));
* document.add(new BinaryDocValuesField(name, new BytesRef("hello")));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* @see BinaryDocValues
* */
public class ShortDocValuesField extends StoredField {
public class BinaryDocValuesField extends StoredField {
/**
* Type for 16-bit short DocValues.
* Type for straight bytes DocValues.
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FIXED_INTS_16);
TYPE.setDocValueType(FieldInfo.DocValuesType.BINARY);
TYPE.freeze();
}
/**
* Creates a new DocValues field with the specified 16-bit short value
/**
* Create a new binary DocValues field.
* @param name field name
* @param value 16-bit short value
* @param value binary content
* @throws IllegalArgumentException if the field name is null
*/
public ShortDocValuesField(String name, short value) {
public BinaryDocValuesField(String name, BytesRef value) {
super(name, TYPE);
fieldsData = Short.valueOf(value);
fieldsData = value;
}
}

View File

@ -1,59 +0,0 @@
package org.apache.lucene.document;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
/**
* <p>
* Field that stores a per-document <code>byte</code> value for scoring,
* sorting or value retrieval. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new ByteDocValuesField(name, (byte) 22));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class ByteDocValuesField extends StoredField {
/**
* Type for 8-bit byte DocValues.
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FIXED_INTS_8);
TYPE.freeze();
}
/**
* Creates a new DocValues field with the specified 8-bit byte value
* @param name field name
* @param value 8-bit byte value
* @throws IllegalArgumentException if the field name is null.
*/
public ByteDocValuesField(String name, byte value) {
super(name, TYPE);
fieldsData = Byte.valueOf(value);
}
}

View File

@ -1,94 +0,0 @@
package org.apache.lucene.document;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.util.BytesRef;
/**
* <p>
* Field that stores
* a per-document {@link BytesRef} value. The values are
* stored indirectly, such that many documents sharing the
* same value all point to a single copy of the value, which
* is a good fit when the fields share values. If values
* are (mostly) unique it's better to use {@link
* StraightBytesDocValuesField}. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new DerefBytesDocValuesField(name, new BytesRef("hello")));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class DerefBytesDocValuesField extends StoredField {
// TODO: ideally indexer figures out var vs fixed on its own!?
/**
* Type for indirect bytes DocValues: all with the same length
*/
public static final FieldType TYPE_FIXED_LEN = new FieldType();
static {
TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_DEREF);
TYPE_FIXED_LEN.freeze();
}
/**
* Type for indirect bytes DocValues: can have variable lengths
*/
public static final FieldType TYPE_VAR_LEN = new FieldType();
static {
TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_DEREF);
TYPE_VAR_LEN.freeze();
}
/**
* Create a new variable-length indirect DocValues field.
* <p>
* This calls
* {@link DerefBytesDocValuesField#DerefBytesDocValuesField(String, BytesRef, boolean)
* DerefBytesDocValuesField(name, bytes, false}, meaning by default
* it allows for values of different lengths. If your values are all
* the same length, use that constructor instead.
* @param name field name
* @param bytes binary content
* @throws IllegalArgumentException if the field name is null
*/
public DerefBytesDocValuesField(String name, BytesRef bytes) {
super(name, TYPE_VAR_LEN);
fieldsData = bytes;
}
/**
* Create a new fixed or variable length indirect DocValues field.
* <p>
* @param name field name
* @param bytes binary content
* @param isFixedLength true if all values have the same length.
* @throws IllegalArgumentException if the field name is null
*/
public DerefBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
fieldsData = bytes;
}
}

View File

@ -17,34 +17,21 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.AtomicReader; // javadocs
import org.apache.lucene.search.FieldCache; // javadocs
/**
* Syntactic sugar for encoding doubles as NumericDocValues
* via {@link Double#doubleToRawLongBits(double)}.
* <p>
* Field that stores a per-document <code>double</code> value for scoring,
* sorting or value retrieval. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new DoubleDocValuesField(name, 22.0));
* </pre>
*
* Per-document double values can be retrieved via
* {@link FieldCache#getDoubles(AtomicReader, String, boolean)}.
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class DoubleDocValuesField extends StoredField {
/**
* Type for 64-bit double DocValues.
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FLOAT_64);
TYPE.freeze();
}
* <b>NOTE</b>: In most all cases this will be rather inefficient,
* requiring eight bytes per document. Consider encoding double
* values yourself with only as much precision as you require.
*/
public class DoubleDocValuesField extends NumericDocValuesField {
/**
* Creates a new DocValues field with the specified 64-bit double value
@ -53,7 +40,16 @@ public class DoubleDocValuesField extends StoredField {
* @throws IllegalArgumentException if the field name is null
*/
public DoubleDocValuesField(String name, double value) {
super(name, TYPE);
fieldsData = Double.valueOf(value);
super(name, Double.doubleToRawLongBits(value));
}
@Override
public void setDoubleValue(double value) {
super.setLongValue(Double.doubleToRawLongBits(value));
}
@Override
public void setLongValue(long value) {
throw new IllegalArgumentException("cannot change value type from Double to Long");
}
}

View File

@ -29,7 +29,6 @@ import org.apache.lucene.document.FieldType.NumericType;
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.IndexableFieldType;
import org.apache.lucene.index.Norm; // javadocs
import org.apache.lucene.index.StorableField;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.index.FieldInvertState; // javadocs
@ -38,13 +37,8 @@ import org.apache.lucene.index.FieldInvertState; // javadocs
* Expert: directly create a field for a document. Most
* users should use one of the sugar subclasses: {@link
* IntField}, {@link LongField}, {@link FloatField}, {@link
* DoubleField}, {@link ByteDocValuesField}, {@link
* ShortDocValuesField}, {@link IntDocValuesField}, {@link
* LongDocValuesField}, {@link PackedLongDocValuesField},
* {@link FloatDocValuesField}, {@link
* DoubleDocValuesField}, {@link SortedBytesDocValuesField},
* {@link DerefBytesDocValuesField}, {@link
* StraightBytesDocValuesField}, {@link
* DoubleField}, {@link BinaryDocValuesField}, {@link
* NumericDocValuesField}, {@link SortedDocValuesField}, {@link
* StringField}, {@link TextField}, {@link StoredField}.
*
* <p/> A field is a section of a Document. Each field has three

View File

@ -18,7 +18,7 @@ package org.apache.lucene.document;
*/
import org.apache.lucene.analysis.Analyzer; // javadocs
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexableFieldType;
import org.apache.lucene.search.NumericRangeQuery; // javadocs
@ -55,7 +55,7 @@ public class FieldType implements IndexableFieldType {
private NumericType numericType;
private boolean frozen;
private int numericPrecisionStep = NumericUtils.PRECISION_STEP_DEFAULT;
private DocValues.Type docValueType;
private DocValuesType docValueType;
/**
* Create a new mutable FieldType with all of the properties from <code>ref</code>
@ -416,21 +416,21 @@ public class FieldType implements IndexableFieldType {
* {@inheritDoc}
* <p>
* The default is <code>null</code> (no docValues)
* @see #setDocValueType(DocValues.Type)
* @see #setDocValueType(org.apache.lucene.index.FieldInfo.DocValuesType)
*/
@Override
public DocValues.Type docValueType() {
public DocValuesType docValueType() {
return docValueType;
}
/**
* Set's the field's DocValues.Type
* Set's the field's DocValuesType
* @param type DocValues type, or null if no DocValues should be stored.
* @throws IllegalStateException if this FieldType is frozen against
* future modifications.
* @see #docValueType()
*/
public void setDocValueType(DocValues.Type type) {
public void setDocValueType(DocValuesType type) {
checkIfFrozen();
docValueType = type;
}

View File

@ -17,33 +17,21 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.AtomicReader; // javadocs
import org.apache.lucene.search.FieldCache; // javadocs
/**
* Syntactic sugar for encoding floats as NumericDocValues
* via {@link Float#floatToRawIntBits(float)}.
* <p>
* Field that stores a per-document <code>float</code> value for scoring,
* sorting or value retrieval. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new FloatDocValuesField(name, 22f));
* </pre>
*
* Per-document floating point values can be retrieved via
* {@link FieldCache#getFloats(AtomicReader, String, boolean)}.
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
* @see DocValues
* */
public class FloatDocValuesField extends StoredField {
/**
* Type for 32-bit float DocValues.
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FLOAT_32);
TYPE.freeze();
}
* <b>NOTE</b>: In most all cases this will be rather inefficient,
* requiring four bytes per document. Consider encoding floating
* point values yourself with only as much precision as you require.
*/
public class FloatDocValuesField extends NumericDocValuesField {
/**
* Creates a new DocValues field with the specified 32-bit float value
@ -52,7 +40,16 @@ public class FloatDocValuesField extends StoredField {
* @throws IllegalArgumentException if the field name is null
*/
public FloatDocValuesField(String name, float value) {
super(name, TYPE);
fieldsData = Float.valueOf(value);
super(name, Float.floatToRawIntBits(value));
}
@Override
public void setFloatValue(float value) {
super.setLongValue(Float.floatToRawIntBits(value));
}
@Override
public void setLongValue(long value) {
throw new IllegalArgumentException("cannot change value type from Float to Long");
}
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
/**
* <p>
@ -25,23 +25,22 @@ import org.apache.lucene.index.DocValues;
* sorting or value retrieval. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new LongDocValuesField(name, 22L));
* document.add(new NumericDocValuesField(name, 22L));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
* @see DocValues
* */
public class LongDocValuesField extends StoredField {
public class NumericDocValuesField extends StoredField {
/**
* Type for 64-bit long DocValues.
* Type for numeric DocValues.
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FIXED_INTS_64);
TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
TYPE.freeze();
}
@ -51,7 +50,7 @@ public class LongDocValuesField extends StoredField {
* @param value 64-bit long value
* @throws IllegalArgumentException if the field name is null
*/
public LongDocValuesField(String name, long value) {
public NumericDocValuesField(String name, long value) {
super(name, TYPE);
fieldsData = Long.valueOf(value);
}

View File

@ -1,63 +0,0 @@
package org.apache.lucene.document;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.AtomicReader; // javadocs
/**
* <p>
* Field that stores a per-document <code>long</code> value
* for scoring, sorting or value retrieval. The values are
* encoded in the index an in RAM (when loaded via
* {@link AtomicReader#docValues})
* using packed ints. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new PackedLongDocValuesField(name, 22L));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class PackedLongDocValuesField extends StoredField {
/**
* Type for packed long DocValues.
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.VAR_INTS);
TYPE.freeze();
}
/**
* Creates a new DocValues field with the specified long value
* @param name field name
* @param value 64-bit long value
* @throws IllegalArgumentException if the field name is null
*/
public PackedLongDocValuesField(String name, long value) {
super(name, TYPE);
fieldsData = Long.valueOf(value);
}
}

View File

@ -1,88 +0,0 @@
package org.apache.lucene.document;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.util.BytesRef;
/**
* <p>
* Field that stores
* a per-document {@link BytesRef} value, indexed for
* sorting. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new SortedBytesDocValuesField(name, new BytesRef("hello")));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class SortedBytesDocValuesField extends StoredField {
// TODO: ideally indexer figures out var vs fixed on its own!?
/**
* Type for sorted bytes DocValues: all with the same length
*/
public static final FieldType TYPE_FIXED_LEN = new FieldType();
static {
TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_SORTED);
TYPE_FIXED_LEN.freeze();
}
/**
* Type for sorted bytes DocValues: can have variable lengths
*/
public static final FieldType TYPE_VAR_LEN = new FieldType();
static {
TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_SORTED);
TYPE_VAR_LEN.freeze();
}
/**
* Create a new variable-length sorted DocValues field.
* <p>
* This calls
* {@link SortedBytesDocValuesField#SortedBytesDocValuesField(String, BytesRef, boolean)
* SortedBytesDocValuesField(name, bytes, false}, meaning by default
* it allows for values of different lengths. If your values are all
* the same length, use that constructor instead.
* @param name field name
* @param bytes binary content
* @throws IllegalArgumentException if the field name is null
*/
public SortedBytesDocValuesField(String name, BytesRef bytes) {
this(name, bytes, false);
}
/**
* Create a new fixed or variable length sorted DocValues field.
* @param name field name
* @param bytes binary content
* @param isFixedLength true if all values have the same length.
* @throws IllegalArgumentException if the field name is null
*/
public SortedBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
fieldsData = bytes;
}
}

View File

@ -17,42 +17,44 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.BytesRef;
/**
* <p>
* Field that stores a per-document <code>int</code> value for scoring,
* sorting or value retrieval. Here's an example usage:
* Field that stores
* a per-document {@link BytesRef} value, indexed for
* sorting. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new IntDocValuesField(name, 22));
* document.add(new SortedDocValuesField(name, new BytesRef("hello")));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
* @see DocValues
*
* */
public class IntDocValuesField extends StoredField {
public class SortedDocValuesField extends StoredField {
/**
* Type for 32-bit integer DocValues.
* Type for sorted bytes DocValues
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FIXED_INTS_32);
TYPE.setDocValueType(FieldInfo.DocValuesType.SORTED);
TYPE.freeze();
}
/**
* Creates a new DocValues field with the specified 32-bit integer value
/**
* Create a new sorted DocValues field.
* @param name field name
* @param value 32-bit integer value
* @param bytes binary content
* @throws IllegalArgumentException if the field name is null
*/
public IntDocValuesField(String name, int value) {
public SortedDocValuesField(String name, BytesRef bytes) {
super(name, TYPE);
fieldsData = Integer.valueOf(value);
fieldsData = bytes;
}
}

View File

@ -1,93 +0,0 @@
package org.apache.lucene.document;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.util.BytesRef;
/**
* <p>
* Field that stores
* a per-document {@link BytesRef} value. The values are
* stored directly with no sharing, which is a good fit when
* the fields don't share (many) values, such as a title
* field. If values may be shared it's better to use {@link
* DerefBytesDocValuesField}. Here's an example usage:
*
* <pre class="prettyprint">
* document.add(new StraightBytesDocValuesField(name, new BytesRef("hello")));
* </pre>
*
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class StraightBytesDocValuesField extends StoredField {
// TODO: ideally indexer figures out var vs fixed on its own!?
/**
* Type for direct bytes DocValues: all with the same length
*/
public static final FieldType TYPE_FIXED_LEN = new FieldType();
static {
TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_STRAIGHT);
TYPE_FIXED_LEN.freeze();
}
/**
* Type for direct bytes DocValues: can have variable lengths
*/
public static final FieldType TYPE_VAR_LEN = new FieldType();
static {
TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_STRAIGHT);
TYPE_VAR_LEN.freeze();
}
/**
* Create a new variable-length direct DocValues field.
* <p>
* This calls
* {@link StraightBytesDocValuesField#StraightBytesDocValuesField(String, BytesRef, boolean)
* StraightBytesDocValuesField(name, bytes, false}, meaning by default
* it allows for values of different lengths. If your values are all
* the same length, use that constructor instead.
* @param name field name
* @param bytes binary content
* @throws IllegalArgumentException if the field name is null
*/
public StraightBytesDocValuesField(String name, BytesRef bytes) {
super(name, TYPE_VAR_LEN);
fieldsData = bytes;
}
/**
* Create a new fixed or variable length direct DocValues field.
* <p>
* @param name field name
* @param bytes binary content
* @param isFixedLength true if all values have the same length.
* @throws IllegalArgumentException if the field name is null
*/
public StraightBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
fieldsData = bytes;
}
}

View File

@ -156,19 +156,30 @@ public abstract class AtomicReader extends IndexReader {
}
return null;
}
/**
* Returns {@link DocValues} for this field.
* This method may return null if the reader has no per-document
* values stored.
*/
public abstract DocValues docValues(String field) throws IOException;
/**
* Returns {@link DocValues} for this field's normalization values.
* This method may return null if the field has no norms.
*/
public abstract DocValues normValues(String field) throws IOException;
/** Returns {@link NumericDocValues} for this field, or
* null if no {@link NumericDocValues} were indexed for
* this field. The returned instance should only be
* used by a single thread. */
public abstract NumericDocValues getNumericDocValues(String field) throws IOException;
/** Returns {@link BinaryDocValues} for this field, or
* null if no {@link BinaryDocValues} were indexed for
* this field. The returned instance should only be
* used by a single thread. */
public abstract BinaryDocValues getBinaryDocValues(String field) throws IOException;
/** Returns {@link SortedDocValues} for this field, or
* null if no {@link SortedDocValues} were indexed for
* this field. The returned instance should only be
* used by a single thread. */
public abstract SortedDocValues getSortedDocValues(String field) throws IOException;
/** Returns {@link NumericDocValues} representing norms
* for this field, or null if no {@link NumericDocValues}
* were indexed. The returned instance should only be
* used by a single thread. */
public abstract NumericDocValues getNormValues(String field) throws IOException;
/**
* Get the {@link FieldInfos} describing all fields in

View File

@ -0,0 +1,48 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.util.BytesRef;
/**
* A per-document byte[]
*/
public abstract class BinaryDocValues {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected BinaryDocValues() {}
/** Lookup the value for document. */
public abstract void get(int docID, BytesRef result);
/**
* Indicates the value was missing for the document.
*/
public static final byte[] MISSING = new byte[0];
/** An empty BinaryDocValues which returns {@link #MISSING} for every document */
public static final BinaryDocValues EMPTY = new BinaryDocValues() {
@Override
public void get(int docID, BytesRef result) {
result.bytes = MISSING;
result.offset = 0;
result.length = 0;
}
};
}

View File

@ -0,0 +1,133 @@
package org.apache.lucene.index;
/*
* 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.Iterator;
import java.util.NoSuchElementException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.packed.AppendingLongBuffer;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
/** Buffers up pending byte[] per doc, then flushes when
* segment flushes. */
class BinaryDocValuesWriter extends DocValuesWriter {
private final ByteBlockPool pool;
private final AppendingLongBuffer lengths;
private final FieldInfo fieldInfo;
private int addedValues = 0;
public BinaryDocValuesWriter(FieldInfo fieldInfo, Counter iwBytesUsed) {
this.fieldInfo = fieldInfo;
this.pool = new ByteBlockPool(new DirectTrackingAllocator(iwBytesUsed));
this.lengths = new AppendingLongBuffer();
}
public void addValue(int docID, BytesRef value) {
if (docID < addedValues) {
throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed per field)");
}
if (value == null) {
throw new IllegalArgumentException("field=\"" + fieldInfo.name + "\": null value not allowed");
}
if (value.length > (BYTE_BLOCK_SIZE - 2)) {
throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" is too large, must be <= " + (BYTE_BLOCK_SIZE - 2));
}
// Fill in any holes:
while(addedValues < docID) {
addedValues++;
lengths.add(0);
}
addedValues++;
lengths.add(value.length);
pool.append(value);
}
@Override
public void finish(int maxDoc) {
}
@Override
public void flush(SegmentWriteState state, DocValuesConsumer dvConsumer) throws IOException {
final int maxDoc = state.segmentInfo.getDocCount();
dvConsumer.addBinaryField(fieldInfo,
new Iterable<BytesRef>() {
@Override
public Iterator<BytesRef> iterator() {
return new BytesIterator(maxDoc);
}
});
}
@Override
public void abort() {
}
// iterates over the values we have in ram
private class BytesIterator implements Iterator<BytesRef> {
final BytesRef value = new BytesRef();
final AppendingLongBuffer.Iterator lengthsIterator = lengths.iterator();
final int size = lengths.size();
final int maxDoc;
int upto;
long byteOffset;
BytesIterator(int maxDoc) {
this.maxDoc = maxDoc;
}
@Override
public boolean hasNext() {
return upto < maxDoc;
}
@Override
public BytesRef next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
if (upto < size) {
int length = (int) lengthsIterator.next();
value.grow(length);
value.length = length;
pool.readBytes(byteOffset, value.bytes, value.offset, value.length);
byteOffset += length;
} else {
// This is to handle last N documents not having
// this DV field in the end of the segment:
value.length = 0;
}
upto++;
return value;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}
}

View File

@ -31,10 +31,7 @@ import java.util.Map;
import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat; // javadocs
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType; // for javadocs
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
@ -183,7 +180,7 @@ public class CheckIndex {
/** Number of deleted documents. */
public int numDeleted;
/** True if we were able to open a SegmentReader on this
/** True if we were able to open an AtomicReader on this
* segment. */
public boolean openReaderPassed;
@ -343,9 +340,9 @@ public class CheckIndex {
setInfoStream(out, false);
}
private void msg(String msg) {
if (infoStream != null)
infoStream.println(msg);
private static void msg(PrintStream out, String msg) {
if (out != null)
out.println(msg);
}
/** Returns a {@link Status} instance detailing
@ -381,7 +378,7 @@ public class CheckIndex {
try {
sis.read(dir);
} catch (Throwable t) {
msg("ERROR: could not read any segments file in directory");
msg(infoStream, "ERROR: could not read any segments file in directory");
result.missingSegments = true;
if (infoStream != null)
t.printStackTrace(infoStream);
@ -416,7 +413,7 @@ public class CheckIndex {
try {
input = dir.openInput(segmentsFileName, IOContext.DEFAULT);
} catch (Throwable t) {
msg("ERROR: could not open segments file in directory");
msg(infoStream, "ERROR: could not open segments file in directory");
if (infoStream != null)
t.printStackTrace(infoStream);
result.cantOpenSegments = true;
@ -426,7 +423,7 @@ public class CheckIndex {
try {
format = input.readInt();
} catch (Throwable t) {
msg("ERROR: could not read segment file version in directory");
msg(infoStream, "ERROR: could not read segment file version in directory");
if (infoStream != null)
t.printStackTrace(infoStream);
result.missingSegmentVersion = true;
@ -460,7 +457,7 @@ public class CheckIndex {
versionString = oldest.equals(newest) ? ( "version=" + oldest ) : ("versions=[" + oldest + " .. " + newest + "]");
}
msg("Segments file=" + segmentsFileName + " numSegments=" + numSegments
msg(infoStream, "Segments file=" + segmentsFileName + " numSegments=" + numSegments
+ " " + versionString + " format=" + sFormat + userDataString);
if (onlySegments != null) {
@ -472,11 +469,11 @@ public class CheckIndex {
infoStream.print(" " + s);
}
result.segmentsChecked.addAll(onlySegments);
msg(":");
msg(infoStream, ":");
}
if (skip) {
msg("\nERROR: this index appears to be created by a newer version of Lucene than this tool was compiled on; please re-compile this tool on the matching version of Lucene; exiting");
msg(infoStream, "\nERROR: this index appears to be created by a newer version of Lucene than this tool was compiled on; please re-compile this tool on the matching version of Lucene; exiting");
result.toolOutOfDate = true;
return result;
}
@ -497,38 +494,41 @@ public class CheckIndex {
}
Status.SegmentInfoStatus segInfoStat = new Status.SegmentInfoStatus();
result.segmentInfos.add(segInfoStat);
msg(" " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " docCount=" + info.info.getDocCount());
msg(infoStream, " " + (1+i) + " of " + numSegments + ": name=" + info.info.name + " docCount=" + info.info.getDocCount());
segInfoStat.name = info.info.name;
segInfoStat.docCount = info.info.getDocCount();
int toLoseDocCount = info.info.getDocCount();
SegmentReader reader = null;
AtomicReader reader = null;
try {
final Codec codec = info.info.getCodec();
msg(" codec=" + codec);
msg(infoStream, " codec=" + codec);
segInfoStat.codec = codec;
msg(" compound=" + info.info.getUseCompoundFile());
msg(infoStream, " compound=" + info.info.getUseCompoundFile());
segInfoStat.compound = info.info.getUseCompoundFile();
msg(" numFiles=" + info.files().size());
msg(infoStream, " numFiles=" + info.files().size());
segInfoStat.numFiles = info.files().size();
segInfoStat.sizeMB = info.sizeInBytes()/(1024.*1024.);
msg(" size (MB)=" + nf.format(segInfoStat.sizeMB));
msg(infoStream, " size (MB)=" + nf.format(segInfoStat.sizeMB));
Map<String,String> diagnostics = info.info.getDiagnostics();
segInfoStat.diagnostics = diagnostics;
if (diagnostics.size() > 0) {
msg(" diagnostics = " + diagnostics);
msg(infoStream, " diagnostics = " + diagnostics);
}
// TODO: we could append the info attributes() to the msg?
Map<String,String> atts = info.info.attributes();
if (atts != null && !atts.isEmpty()) {
msg(infoStream, " attributes = " + atts);
}
if (!info.hasDeletions()) {
msg(" no deletions");
msg(infoStream, " no deletions");
segInfoStat.hasDeletions = false;
}
else{
msg(" has deletions [delGen=" + info.getDelGen() + "]");
msg(infoStream, " has deletions [delGen=" + info.getDelGen() + "]");
segInfoStat.hasDeletions = true;
segInfoStat.deletionsGen = info.getDelGen();
}
@ -566,7 +566,7 @@ public class CheckIndex {
}
segInfoStat.numDeleted = info.info.getDocCount() - numDocs;
msg("OK [" + (segInfoStat.numDeleted) + " deleted docs]");
msg(infoStream, "OK [" + (segInfoStat.numDeleted) + " deleted docs]");
} else {
if (info.getDelCount() != 0) {
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.info.getDocCount() - numDocs));
@ -580,7 +580,7 @@ public class CheckIndex {
}
}
}
msg("OK");
msg(infoStream, "OK");
}
if (reader.maxDoc() != info.info.getDocCount()) {
throw new RuntimeException("SegmentReader.maxDoc() " + reader.maxDoc() + " != SegmentInfos.docCount " + info.info.getDocCount());
@ -591,22 +591,22 @@ public class CheckIndex {
infoStream.print(" test: fields..............");
}
FieldInfos fieldInfos = reader.getFieldInfos();
msg("OK [" + fieldInfos.size() + " fields]");
msg(infoStream, "OK [" + fieldInfos.size() + " fields]");
segInfoStat.numFields = fieldInfos.size();
// Test Field Norms
segInfoStat.fieldNormStatus = testFieldNorms(fieldInfos, reader);
segInfoStat.fieldNormStatus = testFieldNorms(reader, infoStream);
// Test the Term Index
segInfoStat.termIndexStatus = testPostings(fieldInfos, reader);
segInfoStat.termIndexStatus = testPostings(reader, infoStream, verbose);
// Test Stored Fields
segInfoStat.storedFieldStatus = testStoredFields(info, reader, nf);
segInfoStat.storedFieldStatus = testStoredFields(reader, infoStream);
// Test Term Vectors
segInfoStat.termVectorStatus = testTermVectors(fieldInfos, info, reader, nf);
segInfoStat.docValuesStatus = testDocValues(info, fieldInfos, reader);
segInfoStat.termVectorStatus = testTermVectors(reader, infoStream, verbose, crossCheckTermVectors);
segInfoStat.docValuesStatus = testDocValues(reader, infoStream);
// Rethrow the first exception we encountered
// This will cause stats for failed segments to be incremented properly
@ -622,16 +622,16 @@ public class CheckIndex {
throw new RuntimeException("DocValues test failed");
}
msg("");
msg(infoStream, "");
} catch (Throwable t) {
msg("FAILED");
msg(infoStream, "FAILED");
String comment;
comment = "fixIndex() would remove reference to this segment";
msg(" WARNING: " + comment + "; full exception:");
msg(infoStream, " WARNING: " + comment + "; full exception:");
if (infoStream != null)
t.printStackTrace(infoStream);
msg("");
msg(infoStream, "");
result.totLoseDocCount += toLoseDocCount;
result.numBadSegments++;
continue;
@ -647,16 +647,16 @@ public class CheckIndex {
if (0 == result.numBadSegments) {
result.clean = true;
} else
msg("WARNING: " + result.numBadSegments + " broken segments (containing " + result.totLoseDocCount + " documents) detected");
msg(infoStream, "WARNING: " + result.numBadSegments + " broken segments (containing " + result.totLoseDocCount + " documents) detected");
if ( ! (result.validCounter = (result.maxSegmentName < sis.counter))) {
result.clean = false;
result.newSegments.counter = result.maxSegmentName + 1;
msg("ERROR: Next segment name counter " + sis.counter + " is not greater than max segment name " + result.maxSegmentName);
msg(infoStream, "ERROR: Next segment name counter " + sis.counter + " is not greater than max segment name " + result.maxSegmentName);
}
if (result.clean) {
msg("No problems were detected with this index.\n");
msg(infoStream, "No problems were detected with this index.\n");
}
return result;
@ -664,8 +664,9 @@ public class CheckIndex {
/**
* Test field norms.
* @lucene.experimental
*/
private Status.FieldNormStatus testFieldNorms(FieldInfos fieldInfos, SegmentReader reader) {
public static Status.FieldNormStatus testFieldNorms(AtomicReader reader, PrintStream infoStream) {
final Status.FieldNormStatus status = new Status.FieldNormStatus();
try {
@ -673,21 +674,20 @@ public class CheckIndex {
if (infoStream != null) {
infoStream.print(" test: field norms.........");
}
for (FieldInfo info : fieldInfos) {
for (FieldInfo info : reader.getFieldInfos()) {
if (info.hasNorms()) {
DocValues dv = reader.normValues(info.name);
checkDocValues(dv, info.name, info.getNormType(), reader.maxDoc());
checkNorms(info, reader, infoStream);
++status.totFields;
} else {
if (reader.normValues(info.name) != null) {
if (reader.getNormValues(info.name) != null) {
throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
}
}
}
msg("OK [" + status.totFields + " fields]");
msg(infoStream, "OK [" + status.totFields + " fields]");
} catch (Throwable e) {
msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
@ -701,14 +701,14 @@ public class CheckIndex {
* checks Fields api is consistent with itself.
* searcher is optional, to verify with queries. Can be null.
*/
private Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors) throws IOException {
private static Status.TermIndexStatus checkFields(Fields fields, Bits liveDocs, int maxDoc, FieldInfos fieldInfos, boolean doPrint, boolean isVectors, PrintStream infoStream, boolean verbose) throws IOException {
// TODO: we should probably return our own stats thing...?!
final Status.TermIndexStatus status = new Status.TermIndexStatus();
int computedFieldCount = 0;
if (fields == null) {
msg("OK [no fields/terms]");
msg(infoStream, "OK [no fields/terms]");
return status;
}
@ -769,7 +769,7 @@ public class CheckIndex {
break;
}
checkBounds(term);
assert term.isValid();
// make sure terms arrive in order according to
// the comp
@ -859,7 +859,7 @@ public class CheckIndex {
lastPos = pos;
BytesRef payload = postings.getPayload();
if (payload != null) {
checkBounds(payload);
assert payload.isValid();
}
if (payload != null && payload.length < 1) {
throw new RuntimeException("term " + term + ": doc " + doc + ": pos " + pos + " payload length is out of bounds " + payload.length);
@ -1155,7 +1155,7 @@ public class CheckIndex {
}
if (doPrint) {
msg("OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
msg(infoStream, "OK [" + status.termCount + " terms; " + status.totFreq + " terms/docs pairs; " + status.totPos + " tokens]");
}
if (verbose && status.blockTreeStats != null && infoStream != null && status.termCount > 0) {
@ -1170,8 +1170,17 @@ public class CheckIndex {
/**
* Test the term index.
* @lucene.experimental
*/
private Status.TermIndexStatus testPostings(FieldInfos fieldInfos, SegmentReader reader) {
public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream) {
return testPostings(reader, infoStream, false);
}
/**
* Test the term index.
* @lucene.experimental
*/
public static Status.TermIndexStatus testPostings(AtomicReader reader, PrintStream infoStream, boolean verbose) {
// TODO: we should go and verify term vectors match, if
// crossCheckTermVectors is on...
@ -1186,15 +1195,16 @@ public class CheckIndex {
}
final Fields fields = reader.fields();
status = checkFields(fields, liveDocs, maxDoc, fieldInfos, true, false);
final FieldInfos fieldInfos = reader.getFieldInfos();
status = checkFields(fields, liveDocs, maxDoc, fieldInfos, true, false, infoStream, verbose);
if (liveDocs != null) {
if (infoStream != null) {
infoStream.print(" test (ignoring deletes): terms, freq, prox...");
}
checkFields(fields, null, maxDoc, fieldInfos, true, false);
checkFields(fields, null, maxDoc, fieldInfos, true, false, infoStream, verbose);
}
} catch (Throwable e) {
msg("ERROR: " + e);
msg(infoStream, "ERROR: " + e);
status = new Status.TermIndexStatus();
status.error = e;
if (infoStream != null) {
@ -1206,9 +1216,10 @@ public class CheckIndex {
}
/**
* Test stored fields for a segment.
* Test stored fields.
* @lucene.experimental
*/
private Status.StoredFieldStatus testStoredFields(SegmentInfoPerCommit info, SegmentReader reader, NumberFormat format) {
public static Status.StoredFieldStatus testStoredFields(AtomicReader reader, PrintStream infoStream) {
final Status.StoredFieldStatus status = new Status.StoredFieldStatus();
try {
@ -1218,7 +1229,7 @@ public class CheckIndex {
// Scan stored fields for all documents
final Bits liveDocs = reader.getLiveDocs();
for (int j = 0; j < info.info.getDocCount(); ++j) {
for (int j = 0; j < reader.maxDoc(); ++j) {
// Intentionally pull even deleted documents to
// make sure they too are not corrupt:
StoredDocument doc = reader.document(j);
@ -1233,10 +1244,10 @@ public class CheckIndex {
throw new RuntimeException("docCount=" + status.docCount + " but saw " + status.docCount + " undeleted docs");
}
msg("OK [" + status.totFields + " total field count; avg " +
format.format((((float) status.totFields)/status.docCount)) + " fields per doc]");
msg(infoStream, "OK [" + status.totFields + " total field count; avg " +
NumberFormat.getInstance(Locale.ROOT).format((((float) status.totFields)/status.docCount)) + " fields per doc]");
} catch (Throwable e) {
msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
@ -1246,124 +1257,33 @@ public class CheckIndex {
return status;
}
/** Helper method to verify values (either docvalues or norms), also checking
* type and size against fieldinfos/segmentinfo
/**
* Test docvalues.
* @lucene.experimental
*/
private void checkDocValues(DocValues docValues, String fieldName, DocValues.Type expectedType, int expectedDocs) throws IOException {
if (docValues == null) {
throw new RuntimeException("field: " + fieldName + " omits docvalues but should have them!");
}
DocValues.Type type = docValues.getType();
if (type != expectedType) {
throw new RuntimeException("field: " + fieldName + " has type: " + type + " but fieldInfos says:" + expectedType);
}
final Source values = docValues.getDirectSource();
int size = docValues.getValueSize();
for (int i = 0; i < expectedDocs; i++) {
switch (type) {
case BYTES_FIXED_SORTED:
case BYTES_VAR_SORTED:
case BYTES_FIXED_DEREF:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_STRAIGHT:
BytesRef bytes = new BytesRef();
values.getBytes(i, bytes);
if (size != -1 && size != bytes.length) {
throw new RuntimeException("field: " + fieldName + " returned wrongly sized bytes, was: " + bytes.length + " should be: " + size);
}
break;
case FLOAT_32:
assert size == 4;
values.getFloat(i);
break;
case FLOAT_64:
assert size == 8;
values.getFloat(i);
break;
case VAR_INTS:
assert size == -1;
values.getInt(i);
break;
case FIXED_INTS_16:
assert size == 2;
values.getInt(i);
break;
case FIXED_INTS_32:
assert size == 4;
values.getInt(i);
break;
case FIXED_INTS_64:
assert size == 8;
values.getInt(i);
break;
case FIXED_INTS_8:
assert size == 1;
values.getInt(i);
break;
default:
throw new IllegalArgumentException("Field: " + fieldName
+ " - no such DocValues type: " + type);
}
}
if (type == DocValues.Type.BYTES_FIXED_SORTED || type == DocValues.Type.BYTES_VAR_SORTED) {
// check sorted bytes
SortedSource sortedValues = values.asSortedSource();
Comparator<BytesRef> comparator = sortedValues.getComparator();
int maxOrd = sortedValues.getValueCount() - 1;
FixedBitSet seenOrds = new FixedBitSet(sortedValues.getValueCount());
int lastOrd = -1;
BytesRef lastBytes = new BytesRef();
for (int i = 0; i < expectedDocs; i++) {
int ord = sortedValues.ord(i);
if (ord < 0 || ord > maxOrd) {
throw new RuntimeException("field: " + fieldName + " ord is out of bounds: " + ord);
}
BytesRef bytes = new BytesRef();
sortedValues.getByOrd(ord, bytes);
if (lastOrd != -1) {
int ordComp = Integer.signum(new Integer(ord).compareTo(new Integer(lastOrd)));
int bytesComp = Integer.signum(comparator.compare(bytes, lastBytes));
if (ordComp != bytesComp) {
throw new RuntimeException("field: " + fieldName + " ord comparison is wrong: " + ordComp + " comparator claims: " + bytesComp);
}
}
lastOrd = ord;
lastBytes = bytes;
seenOrds.set(ord);
}
if (seenOrds.cardinality() != sortedValues.getValueCount()) {
// TODO: find the bug here and figure out a workaround (we can implement in LUCENE-4547's back compat layer maybe)
// basically ord 0 is unused by any docs: so the sortedbytes ords are all off-by-one
// does it always happen? e.g. maybe only if there are missing values? or a bug in its merge optimizations?
// throw new RuntimeException("dv for field: " + fieldName + " has holes in its ords, valueCount=" + sortedValues.getValueCount() + " but only used: " + seenOrds.cardinality());
}
}
}
private Status.DocValuesStatus testDocValues(SegmentInfoPerCommit info,
FieldInfos fieldInfos,
SegmentReader reader) {
public static Status.DocValuesStatus testDocValues(AtomicReader reader,
PrintStream infoStream) {
final Status.DocValuesStatus status = new Status.DocValuesStatus();
try {
if (infoStream != null) {
infoStream.print(" test: docvalues...........");
}
for (FieldInfo fieldInfo : fieldInfos) {
for (FieldInfo fieldInfo : reader.getFieldInfos()) {
if (fieldInfo.hasDocValues()) {
status.totalValueFields++;
final DocValues docValues = reader.docValues(fieldInfo.name);
checkDocValues(docValues, fieldInfo.name, fieldInfo.getDocValuesType(), reader.maxDoc());
checkDocValues(fieldInfo, reader, infoStream);
} else {
if (reader.docValues(fieldInfo.name) != null) {
if (reader.getBinaryDocValues(fieldInfo.name) != null ||
reader.getNumericDocValues(fieldInfo.name) != null ||
reader.getSortedDocValues(fieldInfo.name) != null) {
throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
}
}
}
msg("OK [" + status.docCount + " total doc count; " + status.totalValueFields + " docvalues fields]");
msg(infoStream, "OK [" + status.docCount + " total doc count; " + status.totalValueFields + " docvalues fields]");
} catch (Throwable e) {
msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);
@ -1372,37 +1292,94 @@ public class CheckIndex {
return status;
}
// basic value checks
private static void checkBounds(BytesRef b) {
if (b.bytes == null) {
throw new RuntimeException("bytes is null");
private static void checkBinaryDocValues(String fieldName, AtomicReader reader, BinaryDocValues dv) {
BytesRef scratch = new BytesRef();
for (int i = 0; i < reader.maxDoc(); i++) {
dv.get(i, scratch);
assert scratch.isValid();
}
if (b.length < 0) {
throw new RuntimeException("length is negative: " + b.length);
}
private static void checkSortedDocValues(String fieldName, AtomicReader reader, SortedDocValues dv) {
checkBinaryDocValues(fieldName, reader, dv);
final int maxOrd = dv.getValueCount()-1;
FixedBitSet seenOrds = new FixedBitSet(dv.getValueCount());
int maxOrd2 = -1;
for (int i = 0; i < reader.maxDoc(); i++) {
int ord = dv.getOrd(i);
if (ord < 0 || ord > maxOrd) {
throw new RuntimeException("ord out of bounds: " + ord);
}
maxOrd2 = Math.max(maxOrd2, ord);
seenOrds.set(ord);
}
if (b.length > b.bytes.length) {
throw new RuntimeException("length is out of bounds: " + b.length + ", bytes.length=" + b.bytes.length);
if (maxOrd != maxOrd2) {
throw new RuntimeException("dv for field: " + fieldName + " reports wrong maxOrd=" + maxOrd + " but this is not the case: " + maxOrd2);
}
if (b.offset < 0) {
throw new RuntimeException("offset is negative: " + b.offset);
if (seenOrds.cardinality() != dv.getValueCount()) {
throw new RuntimeException("dv for field: " + fieldName + " has holes in its ords, valueCount=" + dv.getValueCount() + " but only used: " + seenOrds.cardinality());
}
if (b.offset > b.bytes.length) {
throw new RuntimeException("offset out of bounds: " + b.offset + ", length=" + b.length);
BytesRef lastValue = null;
BytesRef scratch = new BytesRef();
for (int i = 0; i <= maxOrd; i++) {
dv.lookupOrd(i, scratch);
assert scratch.isValid();
if (lastValue != null) {
if (scratch.compareTo(lastValue) <= 0) {
throw new RuntimeException("dv for field: " + fieldName + " has ords out of order: " + lastValue + " >=" + scratch);
}
}
lastValue = BytesRef.deepCopyOf(scratch);
}
if (b.offset + b.length < 0) {
throw new RuntimeException("offset+length is negative: offset=" + b.offset + ",length=" + b.length);
}
private static void checkNumericDocValues(String fieldName, AtomicReader reader, NumericDocValues ndv) {
for (int i = 0; i < reader.maxDoc(); i++) {
ndv.get(i);
}
if (b.offset + b.length > b.bytes.length) {
throw new RuntimeException("offset+length out of bounds: offset=" + b.offset + ",length=" + b.length + ",bytes.length=" + b.bytes.length);
}
private static void checkDocValues(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws Exception {
switch(fi.getDocValuesType()) {
case SORTED:
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
break;
case BINARY:
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
break;
case NUMERIC:
checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
break;
default:
throw new AssertionError();
}
}
private static void checkNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
switch(fi.getNormType()) {
case NUMERIC:
checkNumericDocValues(fi.name, reader, reader.getNormValues(fi.name));
break;
default:
throw new AssertionError("wtf: " + fi.getNormType());
}
}
/**
* Test term vectors for a segment.
* Test term vectors.
* @lucene.experimental
*/
private Status.TermVectorStatus testTermVectors(FieldInfos fieldInfos, SegmentInfoPerCommit info, SegmentReader reader, NumberFormat format) {
final Status.TermVectorStatus status = new Status.TermVectorStatus();
public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream) {
return testTermVectors(reader, infoStream, false, false);
}
/**
* Test term vectors.
* @lucene.experimental
*/
public static Status.TermVectorStatus testTermVectors(AtomicReader reader, PrintStream infoStream, boolean verbose, boolean crossCheckTermVectors) {
final Status.TermVectorStatus status = new Status.TermVectorStatus();
final FieldInfos fieldInfos = reader.getFieldInfos();
final Bits onlyDocIsDeleted = new FixedBitSet(1);
try {
@ -1430,7 +1407,7 @@ public class CheckIndex {
TermsEnum termsEnum = null;
TermsEnum postingsTermsEnum = null;
for (int j = 0; j < info.info.getDocCount(); ++j) {
for (int j = 0; j < reader.maxDoc(); ++j) {
// Intentionally pull/visit (but don't count in
// stats) deleted documents to make sure they too
// are not corrupt:
@ -1441,10 +1418,10 @@ public class CheckIndex {
if (tfv != null) {
// First run with no deletions:
checkFields(tfv, null, 1, fieldInfos, false, true);
checkFields(tfv, null, 1, fieldInfos, false, true, infoStream, verbose);
// Again, with the one doc deleted:
checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true);
checkFields(tfv, onlyDocIsDeleted, 1, fieldInfos, false, true, infoStream, verbose);
// Only agg stats if the doc is live:
final boolean doStats = liveDocs == null || liveDocs.get(j);
@ -1608,10 +1585,10 @@ public class CheckIndex {
}
}
float vectorAvg = status.docCount == 0 ? 0 : status.totVectors / (float)status.docCount;
msg("OK [" + status.totVectors + " total vector count; avg " +
format.format(vectorAvg) + " term/freq vector fields per doc]");
msg(infoStream, "OK [" + status.totVectors + " total vector count; avg " +
NumberFormat.getInstance(Locale.ROOT).format(vectorAvg) + " term/freq vector fields per doc]");
} catch (Throwable e) {
msg("ERROR [" + String.valueOf(e.getMessage()) + "]");
msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
status.error = e;
if (infoStream != null) {
e.printStackTrace(infoStream);

View File

@ -24,6 +24,5 @@ abstract class DocConsumer {
abstract void finishDocument() throws IOException;
abstract void flush(final SegmentWriteState state) throws IOException;
abstract void abort();
abstract boolean freeRAM();
abstract void doAfterFlush();
}

View File

@ -28,15 +28,9 @@ abstract class DocFieldConsumer {
/** Called when an aborting exception is hit */
abstract void abort();
/** Called when DocumentsWriterPerThread is using too much RAM.
* The consumer should free RAM, if possible, returning
* true if any RAM was in fact freed. */
abstract boolean freeRAM();
abstract void startDocument() throws IOException;
abstract DocFieldConsumerPerField addField(FieldInfo fi);
abstract void finishDocument() throws IOException;
}

View File

@ -25,16 +25,10 @@ import java.util.HashSet;
import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
import org.apache.lucene.index.TypePromoter.TypeCompatibility;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.Counter;
/**
* This is a DocConsumer that gathers all fields under the
@ -47,7 +41,7 @@ import org.apache.lucene.util.IOUtils;
final class DocFieldProcessor extends DocConsumer {
final DocFieldConsumer consumer;
final StoredFieldsConsumer fieldsWriter;
final StoredFieldsConsumer storedConsumer;
final Codec codec;
// Holds all fields seen in current doc
@ -62,11 +56,14 @@ final class DocFieldProcessor extends DocConsumer {
int fieldGen;
final DocumentsWriterPerThread.DocState docState;
public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) {
final Counter bytesUsed;
public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer, StoredFieldsConsumer storedConsumer) {
this.docState = docWriter.docState;
this.codec = docWriter.codec;
this.bytesUsed = docWriter.bytesUsed;
this.consumer = consumer;
fieldsWriter = new StoredFieldsConsumer(docWriter);
this.storedConsumer = storedConsumer;
}
@Override
@ -78,15 +75,10 @@ final class DocFieldProcessor extends DocConsumer {
childFields.put(f.getFieldInfo().name, f);
}
fieldsWriter.flush(state);
consumer.flush(childFields, state);
assert fields.size() == totalFieldCount;
for (DocValuesConsumerHolder consumer : docValues.values()) {
consumer.docValuesConsumer.finish(state.segmentInfo.getDocCount());
}
// close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
IOUtils.close(perDocConsumer);
storedConsumer.flush(state);
consumer.flush(childFields, state);
// Important to save after asking consumer to flush so
// consumer can alter the FieldInfo* if necessary. EG,
@ -113,11 +105,9 @@ final class DocFieldProcessor extends DocConsumer {
field = next;
}
}
IOUtils.closeWhileHandlingException(perDocConsumer);
// TODO add abort to PerDocConsumer!
try {
fieldsWriter.abort();
storedConsumer.abort();
} catch (Throwable t) {
if (th == null) {
th = t;
@ -132,16 +122,6 @@ final class DocFieldProcessor extends DocConsumer {
}
}
try {
if (perDocConsumer != null) {
perDocConsumer.abort();
}
} catch (Throwable t) {
if (th == null) {
th = t;
}
}
// If any errors occured, throw it.
if (th != null) {
if (th instanceof RuntimeException) throw (RuntimeException) th;
@ -151,11 +131,6 @@ final class DocFieldProcessor extends DocConsumer {
}
}
@Override
public boolean freeRAM() {
return consumer.freeRAM();
}
public Collection<DocFieldConsumerPerField> fields() {
Collection<DocFieldConsumerPerField> fields = new HashSet<DocFieldConsumerPerField>();
for(int i=0;i<fieldHash.length;i++) {
@ -176,8 +151,6 @@ final class DocFieldProcessor extends DocConsumer {
fieldHash = new DocFieldProcessorPerField[2];
hashMask = 1;
totalFieldCount = 0;
perDocConsumer = null;
docValues.clear();
}
private void rehash() {
@ -207,7 +180,7 @@ final class DocFieldProcessor extends DocConsumer {
public void processDocument(FieldInfos.Builder fieldInfos) throws IOException {
consumer.startDocument();
fieldsWriter.startDocument();
storedConsumer.startDocument();
fieldCount = 0;
@ -226,38 +199,12 @@ final class DocFieldProcessor extends DocConsumer {
fp.addField(field);
}
for (StorableField field: docState.doc.storableFields()) {
final String fieldName = field.name();
IndexableFieldType ft = field.fieldType();
DocFieldProcessorPerField fp = processField(fieldInfos, thisFieldGen, fieldName, ft);
if (ft.stored()) {
fieldsWriter.addField(field, fp.fieldInfo);
}
final DocValues.Type dvType = ft.docValueType();
if (dvType != null) {
DocValuesConsumerHolder docValuesConsumer = docValuesConsumer(dvType,
docState, fp.fieldInfo);
DocValuesConsumer consumer = docValuesConsumer.docValuesConsumer;
if (docValuesConsumer.compatibility == null) {
consumer.add(docState.docID, field);
docValuesConsumer.compatibility = new TypeCompatibility(dvType,
consumer.getValueSize());
} else if (docValuesConsumer.compatibility.isCompatible(dvType,
TypePromoter.getValueSize(dvType, field.binaryValue()))) {
consumer.add(docState.docID, field);
} else {
docValuesConsumer.compatibility.isCompatible(dvType,
TypePromoter.getValueSize(dvType, field.binaryValue()));
TypeCompatibility compatibility = docValuesConsumer.compatibility;
throw new IllegalArgumentException("Incompatible DocValues type: "
+ dvType.name() + " size: "
+ TypePromoter.getValueSize(dvType, field.binaryValue())
+ " expected: " + " type: " + compatibility.getBaseType()
+ " size: " + compatibility.getBaseSize());
}
}
FieldInfo fieldInfo = fieldInfos.addOrUpdate(fieldName, ft);
storedConsumer.addField(docState.docID, field, fieldInfo);
}
// If we are writing vectors then we must visit
@ -280,6 +227,7 @@ final class DocFieldProcessor extends DocConsumer {
private DocFieldProcessorPerField processField(FieldInfos.Builder fieldInfos,
final int thisFieldGen, final String fieldName, IndexableFieldType ft) {
// Make sure we have a PerField allocated
final int hashPos = fieldName.hashCode() & hashMask;
DocFieldProcessorPerField fp = fieldHash[hashPos];
@ -305,7 +253,7 @@ final class DocFieldProcessor extends DocConsumer {
rehash();
}
} else {
fieldInfos.addOrUpdate(fp.fieldInfo.name, ft);
fp.fieldInfo.update(ft);
}
if (thisFieldGen != fp.lastGen) {
@ -336,54 +284,9 @@ final class DocFieldProcessor extends DocConsumer {
@Override
void finishDocument() throws IOException {
try {
fieldsWriter.finishDocument();
storedConsumer.finishDocument();
} finally {
consumer.finishDocument();
}
}
private static class DocValuesConsumerHolder {
// Only used to enforce that same DV field name is never
// added more than once per doc:
int docID;
final DocValuesConsumer docValuesConsumer;
TypeCompatibility compatibility;
public DocValuesConsumerHolder(DocValuesConsumer docValuesConsumer) {
this.docValuesConsumer = docValuesConsumer;
}
}
final private Map<String, DocValuesConsumerHolder> docValues = new HashMap<String, DocValuesConsumerHolder>();
private PerDocConsumer perDocConsumer;
DocValuesConsumerHolder docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo)
throws IOException {
DocValuesConsumerHolder docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
if (docValuesConsumerAndDocID != null) {
if (docState.docID == docValuesConsumerAndDocID.docID) {
throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
}
assert docValuesConsumerAndDocID.docID < docState.docID;
docValuesConsumerAndDocID.docID = docState.docID;
return docValuesConsumerAndDocID;
}
if (perDocConsumer == null) {
PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
perDocConsumer = docState.docWriter.codec.docValuesFormat().docsConsumer(perDocWriteState);
if (perDocConsumer == null) {
throw new IllegalStateException("codec=" + docState.docWriter.codec + " does not support docValues: from docValuesFormat().docsConsumer(...) returned null; field=" + fieldInfo.name);
}
}
DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
assert fieldInfo.getDocValuesType() == null || fieldInfo.getDocValuesType() == valueType;
fieldInfo.setDocValuesType(valueType);
docValuesConsumerAndDocID = new DocValuesConsumerHolder(docValuesConsumer);
docValuesConsumerAndDocID.docID = docState.docID;
docValues.put(fieldInfo.name, docValuesConsumerAndDocID);
return docValuesConsumerAndDocID;
}
}

View File

@ -17,7 +17,12 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RamUsageEstimator;
/**
@ -28,16 +33,19 @@ final class DocFieldProcessorPerField {
final DocFieldConsumerPerField consumer;
final FieldInfo fieldInfo;
private final Counter bytesUsed;
DocFieldProcessorPerField next;
int lastGen = -1;
int fieldCount;
IndexableField[] fields = new IndexableField[1];
private final Map<FieldInfo,String> dvFields = new HashMap<FieldInfo,String>();
public DocFieldProcessorPerField(final DocFieldProcessor docFieldProcessor, final FieldInfo fieldInfo) {
this.consumer = docFieldProcessor.consumer.addField(fieldInfo);
this.fieldInfo = fieldInfo;
this.bytesUsed = docFieldProcessor.bytesUsed;
}
public void addField(IndexableField field) {

View File

@ -77,14 +77,8 @@ final class DocInverter extends DocFieldConsumer {
}
}
@Override
public boolean freeRAM() {
return consumer.freeRAM();
}
@Override
public DocFieldConsumerPerField addField(FieldInfo fi) {
return new DocInverterPerField(this, fi);
}
}

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