mirror of https://github.com/apache/lucene.git
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:
commit
a181dc0735
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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>
|
|
@ -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>();
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 -->
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
}
|
|
@ -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++;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
});
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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><segment>_<fieldNumber>.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 --> Header, PackedType, MinValue,
|
||||
* <li>{@code VAR_INTS} .dat --> Header, PackedType, MinValue,
|
||||
* DefaultValue, PackedStream</li>
|
||||
* <li>{@link Type#FIXED_INTS_8 FIXED_INTS_8} .dat --> Header, ValueSize,
|
||||
* <li>{@code FIXED_INTS_8} .dat --> Header, ValueSize,
|
||||
* {@link DataOutput#writeByte Byte}<sup>maxdoc</sup></li>
|
||||
* <li>{@link Type#FIXED_INTS_16 FIXED_INTS_16} .dat --> Header, ValueSize,
|
||||
* <li>{@code FIXED_INTS_16} .dat --> Header, ValueSize,
|
||||
* {@link DataOutput#writeShort Short}<sup>maxdoc</sup></li>
|
||||
* <li>{@link Type#FIXED_INTS_32 FIXED_INTS_32} .dat --> Header, ValueSize,
|
||||
* <li>{@code FIXED_INTS_32} .dat --> Header, ValueSize,
|
||||
* {@link DataOutput#writeInt Int32}<sup>maxdoc</sup></li>
|
||||
* <li>{@link Type#FIXED_INTS_64 FIXED_INTS_64} .dat --> Header, ValueSize,
|
||||
* <li>{@code FIXED_INTS_64} .dat --> Header, ValueSize,
|
||||
* {@link DataOutput#writeLong Int64}<sup>maxdoc</sup></li>
|
||||
* <li>{@link Type#FLOAT_32 FLOAT_32} .dat --> Header, ValueSize,
|
||||
* Float32<sup>maxdoc</sup></li>
|
||||
* <li>{@link Type#FLOAT_64 FLOAT_64} .dat --> Header, ValueSize,
|
||||
* Float64<sup>maxdoc</sup></li>
|
||||
* <li>{@link Type#BYTES_FIXED_STRAIGHT BYTES_FIXED_STRAIGHT} .dat --> Header, ValueSize,
|
||||
* <li>{@code FLOAT_32} .dat --> Header, ValueSize, Float32<sup>maxdoc</sup></li>
|
||||
* <li>{@code FLOAT_64} .dat --> Header, ValueSize, Float64<sup>maxdoc</sup></li>
|
||||
* <li>{@code BYTES_FIXED_STRAIGHT} .dat --> Header, ValueSize,
|
||||
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>maxdoc</sup></li>
|
||||
* <li>{@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} .idx --> Header, MaxAddress,
|
||||
* Addresses</li>
|
||||
* <li>{@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} .dat --> 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 --> Header, NumValues,
|
||||
* Addresses</li>
|
||||
* <li>{@link Type#BYTES_FIXED_DEREF BYTES_FIXED_DEREF} .dat --> Header, ValueSize,
|
||||
* <li>{@code BYTES_VAR_STRAIGHT} .idx --> Header, TotalBytes, Addresses</li>
|
||||
* <li>{@code BYTES_VAR_STRAIGHT} .dat --> Header,
|
||||
({@link DataOutput#writeByte Byte} * <i>variable ValueSize</i>)<sup>maxdoc</sup></li>
|
||||
* <li>{@code BYTES_FIXED_DEREF} .idx --> Header, NumValues, Addresses</li>
|
||||
* <li>{@code BYTES_FIXED_DEREF} .dat --> Header, ValueSize,
|
||||
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>NumValues</sup></li>
|
||||
* <li>{@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF} .idx --> Header, TotalVarBytes,
|
||||
* Addresses</li>
|
||||
* <li>{@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF} .dat --> Header,
|
||||
* <li>{@code BYTES_VAR_DEREF} .idx --> Header, TotalVarBytes, Addresses</li>
|
||||
* <li>{@code BYTES_VAR_DEREF} .dat --> Header,
|
||||
* (LengthPrefix + {@link DataOutput#writeByte Byte} * <i>variable ValueSize</i>)<sup>NumValues</sup></li>
|
||||
* <li>{@link Type#BYTES_FIXED_SORTED BYTES_FIXED_SORTED} .idx --> Header, NumValues,
|
||||
* Ordinals</li>
|
||||
* <li>{@link Type#BYTES_FIXED_SORTED BYTES_FIXED_SORTED} .dat --> Header, ValueSize,
|
||||
* <li>{@code BYTES_FIXED_SORTED} .idx --> Header, NumValues, Ordinals</li>
|
||||
* <li>{@code BYTES_FIXED_SORTED} .dat --> Header, ValueSize,
|
||||
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>NumValues</sup></li>
|
||||
* <li>{@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED} .idx --> Header, TotalVarBytes,
|
||||
* Addresses, Ordinals</li>
|
||||
* <li>{@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED} .dat --> Header,
|
||||
* <li>{@code BYTES_VAR_SORTED} .idx --> Header, TotalVarBytes, Addresses, Ordinals</li>
|
||||
* <li>{@code BYTES_VAR_SORTED} .dat --> 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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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];
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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) --> Header,<FieldNumber,EntryType,Entry><sup>NumFields</sup></p>
|
||||
* <ul>
|
||||
* <li>Entry --> NumericEntry | BinaryEntry | SortedEntry</li>
|
||||
* <li>NumericEntry --> DataOffset,CompressionType,PackedVersion</li>
|
||||
* <li>BinaryEntry --> DataOffset,DataLength,MinLength,MaxLength,PackedVersion?,BlockSize?</li>
|
||||
* <li>SortedEntry --> DataOffset,ValueCount</li>
|
||||
* <li>FieldNumber,PackedVersion,MinLength,MaxLength,BlockSize,ValueCount --> {@link DataOutput#writeVInt VInt}</li>
|
||||
* <li>DataOffset,DataLength --> {@link DataOutput#writeLong Int64}</li>
|
||||
* <li>EntryType,CompressionType --> {@link DataOutput#writeByte Byte}</li>
|
||||
* <li>Header --> {@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 --> delta-compressed. For each block of 4096 integers, every integer is delta-encoded
|
||||
* from the minimum value within the block.
|
||||
* <li>1 --> 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 --> 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) --> Header,<NumericData | BinaryData | SortedData><sup>NumFields</sup></p>
|
||||
* <ul>
|
||||
* <li>NumericData --> DeltaCompressedNumerics | TableCompressedNumerics | UncompressedNumerics</li>
|
||||
* <li>BinaryData --> {@link DataOutput#writeByte Byte}<sup>DataLength</sup>,Addresses</li>
|
||||
* <li>SortedData --> {@link FST FST<Int64>}</li>
|
||||
* <li>DeltaCompressedNumerics --> {@link BlockPackedWriter BlockPackedInts(blockSize=4096)}</li>
|
||||
* <li>TableCompressedNumerics --> TableSize,{@link DataOutput#writeLong Int64}<sup>TableSize</sup>,{@link PackedInts PackedInts}</li>
|
||||
* <li>UncompressedNumerics --> {@link DataOutput#writeByte Byte}<sup>maxdoc</sup></li>
|
||||
* <li>Addresses --> {@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";
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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) --> Header,FieldsCount, <FieldName,FieldNumber,
|
||||
* FieldBits,DocValuesBits,Attributes> <sup>FieldsCount</sup></p>
|
||||
* <p>Data types:
|
||||
* <ul>
|
||||
* <li>Header --> {@link CodecUtil#checkHeader CodecHeader}</li>
|
||||
* <li>FieldsCount --> {@link DataOutput#writeVInt VInt}</li>
|
||||
* <li>FieldName --> {@link DataOutput#writeString String}</li>
|
||||
* <li>FieldBits, DocValuesBits --> {@link DataOutput#writeByte Byte}</li>
|
||||
* <li>FieldNumber --> {@link DataOutput#writeInt VInt}</li>
|
||||
* <li>Attributes --> {@link DataOutput#writeStringStringMap Map<String,String>}</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;
|
||||
}
|
|
@ -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 + ")");
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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";
|
||||
}
|
|
@ -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>
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue