mirror of
https://github.com/apache/lucene.git
synced 2025-02-22 01:56:16 +00:00
LUCENE-3628: Cut over Norms to DocValues
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1227676 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d44531d05f
commit
e4cb8e3a17
@ -619,6 +619,9 @@ New features
|
||||
* LUCENE-3638: Added sugar methods to IndexReader and IndexSearcher to
|
||||
load only certain fields when loading a document. (Peter Chang via
|
||||
Mike McCandless)
|
||||
|
||||
* LUCENE-3628: Norms are represented as DocValues. IndexReader exposes
|
||||
a #normValues(String) method to obtain norms per field. (Simon Willnauer)
|
||||
|
||||
Optimizations
|
||||
|
||||
|
@ -48,6 +48,7 @@ import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TermState;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.memory.MemoryIndexNormDocValues.SingleByteSource;
|
||||
import org.apache.lucene.search.Collector;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
@ -1082,34 +1083,6 @@ public class MemoryIndex {
|
||||
private void setSearcher(IndexSearcher searcher) {
|
||||
this.searcher = searcher;
|
||||
}
|
||||
|
||||
/** performance hack: cache norms to avoid repeated expensive calculations */
|
||||
private byte[] cachedNorms;
|
||||
private String cachedFieldName;
|
||||
private SimilarityProvider cachedSimilarity;
|
||||
|
||||
@Override
|
||||
public byte[] norms(String fieldName) {
|
||||
byte[] norms = cachedNorms;
|
||||
SimilarityProvider sim = getSimilarityProvider();
|
||||
if (!fieldName.equals(cachedFieldName) || sim != cachedSimilarity) { // not cached?
|
||||
Info info = getInfo(fieldName);
|
||||
Similarity fieldSim = sim.get(fieldName);
|
||||
int numTokens = info != null ? info.numTokens : 0;
|
||||
int numOverlapTokens = info != null ? info.numOverlapTokens : 0;
|
||||
float boost = info != null ? info.getBoost() : 1.0f;
|
||||
FieldInvertState invertState = new FieldInvertState(0, numTokens, numOverlapTokens, 0, boost);
|
||||
byte norm = fieldSim.computeNorm(invertState);
|
||||
norms = new byte[] {norm};
|
||||
|
||||
// cache it for future reuse
|
||||
cachedNorms = norms;
|
||||
cachedFieldName = fieldName;
|
||||
cachedSimilarity = sim;
|
||||
if (DEBUG) System.err.println("MemoryIndexReader.norms: " + fieldName + ":" + norm + ":" + numTokens);
|
||||
}
|
||||
return norms;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numDocs() {
|
||||
@ -1160,6 +1133,34 @@ public class MemoryIndex {
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
/** performance hack: cache norms to avoid repeated expensive calculations */
|
||||
private DocValues cachedNormValues;
|
||||
private String cachedFieldName;
|
||||
private SimilarityProvider cachedSimilarity;
|
||||
|
||||
@Override
|
||||
public DocValues normValues(String field) throws IOException {
|
||||
DocValues norms = cachedNormValues;
|
||||
SimilarityProvider sim = getSimilarityProvider();
|
||||
if (!field.equals(cachedFieldName) || sim != cachedSimilarity) { // not cached?
|
||||
Info info = getInfo(field);
|
||||
Similarity fieldSim = sim.get(field);
|
||||
int numTokens = info != null ? info.numTokens : 0;
|
||||
int numOverlapTokens = info != null ? info.numOverlapTokens : 0;
|
||||
float boost = info != null ? info.getBoost() : 1.0f;
|
||||
FieldInvertState invertState = new FieldInvertState(0, numTokens, numOverlapTokens, 0, boost);
|
||||
byte norm = fieldSim.computeNorm(invertState);
|
||||
SingleByteSource singleByteSource = new SingleByteSource(new byte[] {norm});
|
||||
norms = new MemoryIndexNormDocValues(singleByteSource);
|
||||
// cache it for future reuse
|
||||
cachedNormValues = norms;
|
||||
cachedFieldName = field;
|
||||
cachedSimilarity = sim;
|
||||
if (DEBUG) System.err.println("MemoryIndexReader.norms: " + field + ":" + norm + ":" + numTokens);
|
||||
}
|
||||
return norms;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -0,0 +1,78 @@
|
||||
package org.apache.lucene.index.memory;
|
||||
/**
|
||||
* 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;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
*
|
||||
* @lucene.internal
|
||||
*/
|
||||
class MemoryIndexNormDocValues extends DocValues {
|
||||
|
||||
private final Source source;
|
||||
|
||||
MemoryIndexNormDocValues(Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type type() {
|
||||
return source.type();
|
||||
}
|
||||
|
||||
public static class SingleByteSource extends Source {
|
||||
|
||||
private final byte[] bytes;
|
||||
|
||||
protected SingleByteSource(byte[] bytes) {
|
||||
super(Type.BYTES_FIXED_STRAIGHT);
|
||||
this.bytes = bytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.bytes = bytes;
|
||||
ref.offset = docID;
|
||||
ref.length = 1;
|
||||
return ref;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasArray() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getArray() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -17,10 +17,7 @@ package org.apache.lucene.codecs;
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStreamWriter;
|
||||
import java.io.Writer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
@ -42,7 +39,6 @@ import org.apache.lucene.util.fst.ByteSequenceOutputs;
|
||||
import org.apache.lucene.util.fst.BytesRefFSTEnum;
|
||||
import org.apache.lucene.util.fst.FST;
|
||||
import org.apache.lucene.util.fst.NoOutputs;
|
||||
import org.apache.lucene.util.fst.Util;
|
||||
|
||||
/*
|
||||
TODO:
|
||||
@ -641,6 +637,7 @@ public class BlockTreeTermsWriter extends FieldsConsumer {
|
||||
}
|
||||
|
||||
// for debugging
|
||||
@SuppressWarnings("unused")
|
||||
private String toString(BytesRef b) {
|
||||
try {
|
||||
return b.utf8ToString() + " " + b;
|
||||
|
@ -18,11 +18,14 @@ package org.apache.lucene.codecs;
|
||||
*/
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.lucene40.values.Writer;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
* Abstract API that consumes {@link DocValue}s.
|
||||
@ -35,6 +38,9 @@ import org.apache.lucene.util.Bits;
|
||||
*/
|
||||
public abstract class DocValuesConsumer {
|
||||
|
||||
protected Source currentMergeSource;
|
||||
protected final BytesRef spare = new BytesRef();
|
||||
|
||||
/**
|
||||
* Adds the given {@link DocValue} instance to this
|
||||
* {@link DocValuesConsumer}
|
||||
@ -83,6 +89,7 @@ public abstract class DocValuesConsumer {
|
||||
hasMerged = true;
|
||||
merge(new SingleSubMergeState(docValues[readerIDX], mergeState.docBase[readerIDX], reader.reader.maxDoc(),
|
||||
reader.liveDocs));
|
||||
mergeState.checkAbort.work(reader.reader.maxDoc());
|
||||
}
|
||||
}
|
||||
// only finish if no exception is thrown!
|
||||
@ -99,10 +106,112 @@ public abstract class DocValuesConsumer {
|
||||
* @throws IOException
|
||||
* if an {@link IOException} occurs
|
||||
*/
|
||||
// TODO: can't we have a default implementation here that merges naively with our apis?
|
||||
// this is how stored fields and term vectors work. its a pain to have to impl merging
|
||||
// (should be an optimization to override it)
|
||||
protected abstract void merge(SingleSubMergeState mergeState) throws IOException;
|
||||
protected void merge(SingleSubMergeState state) 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 = state.reader.getDirectSource();
|
||||
assert source != null;
|
||||
setNextMergeSource(source); // set the current enum we are working on - the
|
||||
// impl. will get the correct reference for the type
|
||||
// it supports
|
||||
int docID = state.docBase;
|
||||
final Bits liveDocs = state.liveDocs;
|
||||
final int docCount = state.docCount;
|
||||
for (int i = 0; i < docCount; i++) {
|
||||
if (liveDocs == null || liveDocs.get(i)) {
|
||||
mergeDoc(docID++, i);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Records the specified <tt>long</tt> value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* <tt>long</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record <tt>long</tt> values
|
||||
*/
|
||||
protected void add(int docID, long value) throws IOException {
|
||||
throw new UnsupportedOperationException("override this method to support integer types");
|
||||
}
|
||||
|
||||
/**
|
||||
* Records the specified <tt>double</tt> value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* <tt>double</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record <tt>double</tt> values
|
||||
*/
|
||||
protected void add(int docID, double value) throws IOException {
|
||||
throw new UnsupportedOperationException("override this method to support floating point types");
|
||||
}
|
||||
|
||||
/**
|
||||
* Records the specified {@link BytesRef} value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* {@link BytesRef} values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record {@link BytesRef} values
|
||||
*/
|
||||
protected void add(int docID, BytesRef value) throws IOException {
|
||||
throw new UnsupportedOperationException("override this method to support byte types");
|
||||
}
|
||||
|
||||
/**
|
||||
* 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} set to <i>setNextMergeSource(Source)</i>.
|
||||
* <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 {@link Writer} implementation. The given document
|
||||
* ID must always be greater than the previous ID or <tt>0</tt> if called the
|
||||
* first time.
|
||||
*/
|
||||
protected void mergeDoc(int docID, int sourceDoc)
|
||||
throws IOException {
|
||||
switch(currentMergeSource.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:
|
||||
add(docID, currentMergeSource.getBytes(sourceDoc, spare));
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
case FIXED_INTS_32:
|
||||
case FIXED_INTS_64:
|
||||
case FIXED_INTS_8:
|
||||
case VAR_INTS:
|
||||
add(docID, currentMergeSource.getInt(sourceDoc));
|
||||
break;
|
||||
case FLOAT_32:
|
||||
case FLOAT_64:
|
||||
add(docID, currentMergeSource.getFloat(sourceDoc));
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the next {@link Source} to consume values from on calls to
|
||||
* {@link #mergeDoc(int, int)}
|
||||
*
|
||||
* @param mergeSource
|
||||
* the next {@link Source}, this must not be null
|
||||
*/
|
||||
protected final void setNextMergeSource(Source mergeSource) {
|
||||
currentMergeSource = mergeSource;
|
||||
}
|
||||
|
||||
/**
|
||||
* Specialized auxiliary MergeState is necessary since we don't want to
|
||||
|
@ -41,7 +41,7 @@ import org.apache.lucene.util.BytesRef;
|
||||
*/
|
||||
// TODO: this needs to go under lucene40 codec (its specific to its impl)
|
||||
public abstract class DocValuesReaderBase extends PerDocProducer {
|
||||
|
||||
|
||||
protected abstract void closeInternal(Collection<? extends Closeable> closeables) throws IOException;
|
||||
protected abstract Map<String, DocValues> docValues();
|
||||
|
||||
@ -68,14 +68,14 @@ public abstract class DocValuesReaderBase extends PerDocProducer {
|
||||
try {
|
||||
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
if (canLoad(fieldInfo)) {
|
||||
final String field = fieldInfo.name;
|
||||
// TODO can we have a compound file per segment and codec for
|
||||
// docvalues?
|
||||
final String id = DocValuesWriterBase.docValuesId(segment,
|
||||
fieldInfo.number);
|
||||
values.put(field,
|
||||
loadDocValues(docCount, dir, id, fieldInfo.getDocValuesType(), context));
|
||||
loadDocValues(docCount, dir, id, getDocValuesType(fieldInfo), context));
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
@ -88,6 +88,18 @@ public abstract class DocValuesReaderBase extends PerDocProducer {
|
||||
return values;
|
||||
}
|
||||
|
||||
protected boolean canLoad(FieldInfo info) {
|
||||
return info.hasDocValues();
|
||||
}
|
||||
|
||||
protected Type getDocValuesType(FieldInfo info) {
|
||||
return info.getDocValuesType();
|
||||
}
|
||||
|
||||
protected boolean anyDocValuesFields(FieldInfos infos) {
|
||||
return infos.anyDocValuesFields();
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads a {@link DocValues} instance depending on the given {@link Type}.
|
||||
* Codecs that use different implementations for a certain {@link Type} can
|
||||
|
@ -23,7 +23,6 @@ import java.util.Comparator;
|
||||
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;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadoc
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
@ -38,7 +37,6 @@ import org.apache.lucene.util.Counter;
|
||||
//TODO: this needs to go under lucene40 codec (its specific to its impl)
|
||||
public abstract class DocValuesWriterBase extends PerDocConsumer {
|
||||
protected final String segmentName;
|
||||
protected final String segmentSuffix;
|
||||
private final Counter bytesUsed;
|
||||
protected final IOContext context;
|
||||
private final boolean fasterButMoreRam;
|
||||
@ -58,7 +56,6 @@ public abstract class DocValuesWriterBase extends PerDocConsumer {
|
||||
*/
|
||||
protected DocValuesWriterBase(PerDocWriteState state, boolean fasterButMoreRam) {
|
||||
this.segmentName = state.segmentName;
|
||||
this.segmentSuffix = state.segmentSuffix;
|
||||
this.bytesUsed = state.bytesUsed;
|
||||
this.context = state.context;
|
||||
this.fasterButMoreRam = fasterButMoreRam;
|
||||
|
@ -20,19 +20,19 @@ package org.apache.lucene.codecs;
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/**
|
||||
* format for normalization factors
|
||||
*/
|
||||
public abstract class NormsFormat {
|
||||
/** Note: separateNormsDir should not be used! */
|
||||
public abstract NormsReader normsReader(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context, Directory separateNormsDir) throws IOException;
|
||||
public abstract NormsWriter normsWriter(SegmentWriteState state) throws IOException;
|
||||
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
|
||||
@Deprecated
|
||||
public abstract PerDocProducer docsProducer(SegmentReadState state, Directory separateNormsDir) throws IOException;
|
||||
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
|
||||
public abstract void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException;
|
||||
|
||||
/**
|
||||
|
@ -1,70 +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.Arrays;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
// simple api just for now before switching to docvalues apis
|
||||
public abstract class NormsWriter implements Closeable {
|
||||
|
||||
// TODO: I think IW should set info.normValueType from Similarity,
|
||||
// and then this method just returns DocValuesConsumer
|
||||
public abstract void startField(FieldInfo info) throws IOException;
|
||||
public abstract void writeNorm(byte norm) throws IOException;
|
||||
public abstract void finish(int numDocs) throws IOException;
|
||||
|
||||
public int merge(MergeState mergeState) throws IOException {
|
||||
int numMergedDocs = 0;
|
||||
for (FieldInfo fi : mergeState.fieldInfos) {
|
||||
if (fi.isIndexed && !fi.omitNorms) {
|
||||
startField(fi);
|
||||
int numMergedDocsForField = 0;
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
byte normBuffer[] = reader.reader.norms(fi.name);
|
||||
if (normBuffer == null) {
|
||||
// Can be null if this segment doesn't have
|
||||
// any docs with this field
|
||||
normBuffer = new byte[maxDoc];
|
||||
Arrays.fill(normBuffer, (byte)0);
|
||||
}
|
||||
// this segment has deleted docs, so we have to
|
||||
// check for every doc if it is deleted or not
|
||||
final Bits liveDocs = reader.liveDocs;
|
||||
for (int k = 0; k < maxDoc; k++) {
|
||||
if (liveDocs == null || liveDocs.get(k)) {
|
||||
writeNorm(normBuffer[k]);
|
||||
numMergedDocsForField++;
|
||||
}
|
||||
}
|
||||
mergeState.checkAbort.work(maxDoc);
|
||||
}
|
||||
assert numMergedDocs == 0 || numMergedDocs == numMergedDocsForField;
|
||||
numMergedDocs = numMergedDocsForField;
|
||||
}
|
||||
}
|
||||
finish(numMergedDocs);
|
||||
return numMergedDocs;
|
||||
}
|
||||
}
|
@ -20,7 +20,9 @@ import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
|
||||
/**
|
||||
* Abstract API that consumes per document values. Concrete implementations of
|
||||
@ -32,7 +34,7 @@ import org.apache.lucene.index.MergeState;
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class PerDocConsumer implements Closeable{
|
||||
public abstract class PerDocConsumer implements Closeable {
|
||||
/** Adds a new DocValuesField */
|
||||
public abstract DocValuesConsumer addValuesField(DocValues.Type type, FieldInfo field)
|
||||
throws IOException;
|
||||
@ -46,14 +48,57 @@ public abstract class PerDocConsumer implements Closeable{
|
||||
|
||||
for (FieldInfo fieldInfo : mergeState.fieldInfos) {
|
||||
mergeState.fieldInfo = fieldInfo; // set the field we are merging
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
if (canMerge(fieldInfo)) {
|
||||
for (int i = 0; i < docValues.length; i++) {
|
||||
docValues[i] = mergeState.readers.get(i).reader.docValues(fieldInfo.name);
|
||||
docValues[i] = getDocValuesForMerge(mergeState.readers.get(i).reader, fieldInfo);
|
||||
}
|
||||
final DocValuesConsumer docValuesConsumer = addValuesField(fieldInfo.getDocValuesType(), fieldInfo);
|
||||
final DocValuesConsumer docValuesConsumer = addValuesField(getDocValuesType(fieldInfo), 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 IndexReader#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(IndexReader 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();
|
||||
}
|
||||
|
@ -31,7 +31,6 @@ import org.apache.lucene.codecs.SegmentInfosFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.TermVectorsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfosFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
|
||||
@ -65,7 +64,7 @@ public class Lucene3xCodec extends Codec {
|
||||
private final SegmentInfosFormat infosFormat = new Lucene40SegmentInfosFormat();
|
||||
|
||||
// TODO: this should really be a different impl
|
||||
private final NormsFormat normsFormat = new Lucene40NormsFormat();
|
||||
private final NormsFormat normsFormat = new Lucene3xNormsFormat();
|
||||
|
||||
// 3.x doesn't support docvalues
|
||||
private final DocValuesFormat docValuesFormat = new DocValuesFormat() {
|
||||
|
@ -0,0 +1,65 @@
|
||||
package org.apache.lucene.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* 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.Set;
|
||||
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/**
|
||||
* Read-Only Lucene 3.x Norms Format
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Lucene3xNormsFormat extends NormsFormat {
|
||||
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xNormsProducer.files(dir, info, files);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene3xNormsProducer.separateFiles(dir, info, files);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
throw new IllegalArgumentException("this codec can only be used for reading");
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return docsProducer(state, null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocProducer docsProducer(SegmentReadState state,
|
||||
Directory separateNormsDir) throws IOException {
|
||||
return new Lucene3xNormsProducer(state.dir, state.segmentInfo, state.fieldInfos, state.context, separateNormsDir);
|
||||
}
|
||||
}
|
@ -1,4 +1,4 @@
|
||||
package org.apache.lucene.codecs.lucene40;
|
||||
package org.apache.lucene.codecs.lucene3x;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
@ -24,7 +24,10 @@ import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.lucene.codecs.NormsReader;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
@ -32,14 +35,29 @@ import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.MapBackedSet;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
public class Lucene40NormsReader extends NormsReader {
|
||||
// this would be replaced by Source/SourceCache in a dv impl.
|
||||
// for now we have our own mini-version
|
||||
final Map<String,Norm> norms = new HashMap<String,Norm>();
|
||||
/**
|
||||
* Reads Lucene 3.x norms format and exposes it via DocValues API
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class Lucene3xNormsProducer extends PerDocProducer {
|
||||
|
||||
/** norms header placeholder */
|
||||
static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
|
||||
|
||||
/** Extension of norms file */
|
||||
static final String NORMS_EXTENSION = "nrm";
|
||||
|
||||
/** Extension of separate norms file
|
||||
* @deprecated */
|
||||
@Deprecated
|
||||
static final String SEPARATE_NORMS_EXTENSION = "s";
|
||||
|
||||
final Map<String,NormsDocValues> norms = new HashMap<String,NormsDocValues>();
|
||||
// any .nrm or .sNN files we have open at any time.
|
||||
// TODO: just a list, and double-close() separate norms files?
|
||||
final Set<IndexInput> openFiles = new MapBackedSet<IndexInput>(new IdentityHashMap<IndexInput,Boolean>());
|
||||
@ -49,20 +67,20 @@ public class Lucene40NormsReader extends NormsReader {
|
||||
|
||||
// note: just like segmentreader in 3.x, we open up all the files here (including separate norms) up front.
|
||||
// but we just don't do any seeks or reading yet.
|
||||
public Lucene40NormsReader(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context, Directory separateNormsDir) throws IOException {
|
||||
public Lucene3xNormsProducer(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context, Directory separateNormsDir) throws IOException {
|
||||
maxdoc = info.docCount;
|
||||
String segmentName = info.name;
|
||||
Map<Integer,Long> normGen = info.getNormGen();
|
||||
boolean success = false;
|
||||
try {
|
||||
long nextNormSeek = Lucene40NormsWriter.NORMS_HEADER.length; //skip header (header unused for now)
|
||||
long nextNormSeek = NORMS_HEADER.length; //skip header (header unused for now)
|
||||
for (FieldInfo fi : fields) {
|
||||
if (fi.isIndexed && !fi.omitNorms) {
|
||||
String fileName = getNormFilename(segmentName, normGen, fi.number);
|
||||
Directory d = hasSeparateNorms(normGen, fi.number) ? separateNormsDir : dir;
|
||||
|
||||
// singleNormFile means multiple norms share this file
|
||||
boolean singleNormFile = IndexFileNames.matchesExtension(fileName, Lucene40NormsWriter.NORMS_EXTENSION);
|
||||
boolean singleNormFile = IndexFileNames.matchesExtension(fileName, NORMS_EXTENSION);
|
||||
IndexInput normInput = null;
|
||||
long normSeek;
|
||||
|
||||
@ -90,19 +108,16 @@ public class Lucene40NormsReader extends NormsReader {
|
||||
if (isUnversioned) {
|
||||
normSeek = 0;
|
||||
} else {
|
||||
normSeek = Lucene40NormsWriter.NORMS_HEADER.length;
|
||||
normSeek = NORMS_HEADER.length;
|
||||
}
|
||||
}
|
||||
|
||||
Norm norm = new Norm();
|
||||
norm.file = normInput;
|
||||
norm.offset = normSeek;
|
||||
NormsDocValues norm = new NormsDocValues(normInput, normSeek);
|
||||
norms.put(fi.name, norm);
|
||||
nextNormSeek += maxdoc; // increment also if some norms are separate
|
||||
}
|
||||
}
|
||||
// TODO: change to a real check? see LUCENE-3619
|
||||
assert singleNormStream == null || nextNormSeek == singleNormStream.length();
|
||||
assert singleNormStream == null || nextNormSeek == singleNormStream.length() : singleNormStream != null ? "len: " + singleNormStream.length() + " expected: " + nextNormSeek : "null";
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
@ -112,12 +127,10 @@ public class Lucene40NormsReader extends NormsReader {
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] norms(String name) throws IOException {
|
||||
Norm norm = norms.get(name);
|
||||
return norm == null ? null : norm.bytes();
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
return norms.get(field);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
@ -130,10 +143,10 @@ public class Lucene40NormsReader extends NormsReader {
|
||||
|
||||
private static String getNormFilename(String segmentName, Map<Integer,Long> normGen, int number) {
|
||||
if (hasSeparateNorms(normGen, number)) {
|
||||
return IndexFileNames.fileNameFromGeneration(segmentName, Lucene40NormsWriter.SEPARATE_NORMS_EXTENSION + number, normGen.get(number));
|
||||
return IndexFileNames.fileNameFromGeneration(segmentName, SEPARATE_NORMS_EXTENSION + number, normGen.get(number));
|
||||
} else {
|
||||
// single file for all norms
|
||||
return IndexFileNames.fileNameFromGeneration(segmentName, Lucene40NormsWriter.NORMS_EXTENSION, SegmentInfo.WITHOUT_GEN);
|
||||
return IndexFileNames.fileNameFromGeneration(segmentName, NORMS_EXTENSION, SegmentInfo.WITHOUT_GEN);
|
||||
}
|
||||
}
|
||||
|
||||
@ -146,34 +159,38 @@ public class Lucene40NormsReader extends NormsReader {
|
||||
return gen != null && gen.longValue() != SegmentInfo.NO;
|
||||
}
|
||||
|
||||
class Norm {
|
||||
IndexInput file;
|
||||
long offset;
|
||||
byte bytes[];
|
||||
static final class NormSource extends Source {
|
||||
protected NormSource(byte[] bytes) {
|
||||
super(Type.BYTES_FIXED_STRAIGHT);
|
||||
this.bytes = bytes;
|
||||
}
|
||||
|
||||
final byte bytes[];
|
||||
|
||||
synchronized byte[] bytes() throws IOException {
|
||||
if (bytes == null) {
|
||||
bytes = new byte[maxdoc];
|
||||
// some norms share fds
|
||||
synchronized(file) {
|
||||
file.seek(offset);
|
||||
file.readBytes(bytes, 0, bytes.length, false);
|
||||
}
|
||||
// we are done with this file
|
||||
if (file != singleNormStream) {
|
||||
openFiles.remove(file);
|
||||
file.close();
|
||||
file = null;
|
||||
}
|
||||
}
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.bytes = bytes;
|
||||
ref.offset = docID;
|
||||
ref.length = 1;
|
||||
return ref;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasArray() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getArray() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
// TODO: This is what SI always did... but we can do this cleaner?
|
||||
// like first FI that has norms but doesn't have separate norms?
|
||||
final String normsFileName = IndexFileNames.segmentFileName(info.name, "", Lucene40NormsWriter.NORMS_EXTENSION);
|
||||
final String normsFileName = IndexFileNames.segmentFileName(info.name, "", NORMS_EXTENSION);
|
||||
if (dir.fileExists(normsFileName)) {
|
||||
files.add(normsFileName);
|
||||
}
|
||||
@ -188,9 +205,49 @@ public class Lucene40NormsReader extends NormsReader {
|
||||
long gen = entry.getValue();
|
||||
if (gen >= SegmentInfo.YES) {
|
||||
// Definitely a separate norm file, with generation:
|
||||
files.add(IndexFileNames.fileNameFromGeneration(info.name, Lucene40NormsWriter.SEPARATE_NORMS_EXTENSION + entry.getKey(), gen));
|
||||
files.add(IndexFileNames.fileNameFromGeneration(info.name, SEPARATE_NORMS_EXTENSION + entry.getKey(), gen));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private class NormsDocValues extends DocValues {
|
||||
private final IndexInput file;
|
||||
private final long offset;
|
||||
public NormsDocValues(IndexInput normInput, long normSeek) {
|
||||
this.file = normInput;
|
||||
this.offset = normSeek;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return new NormSource(bytes());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return getSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type type() {
|
||||
return Type.BYTES_FIXED_STRAIGHT;
|
||||
}
|
||||
|
||||
byte[] bytes() throws IOException {
|
||||
byte[] bytes = new byte[maxdoc];
|
||||
// some norms share fds
|
||||
synchronized(file) {
|
||||
file.seek(offset);
|
||||
file.readBytes(bytes, 0, bytes.length, false);
|
||||
}
|
||||
// we are done with this file
|
||||
if (file != singleNormStream) {
|
||||
openFiles.remove(file);
|
||||
file.close();
|
||||
}
|
||||
return bytes;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
@ -28,6 +28,7 @@ import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.CompoundFileDirectory;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Default PerDocConsumer implementation that uses compound file.
|
||||
@ -36,11 +37,13 @@ import org.apache.lucene.store.Directory;
|
||||
public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
|
||||
private final Directory mainDirectory;
|
||||
private Directory directory;
|
||||
private final String segmentSuffix;
|
||||
public final static String DOC_VALUES_SEGMENT_SUFFIX = "dv";
|
||||
|
||||
final static String DOC_VALUES_SEGMENT_SUFFIX = "dv";
|
||||
|
||||
public Lucene40DocValuesConsumer(PerDocWriteState state) throws IOException {
|
||||
|
||||
public Lucene40DocValuesConsumer(PerDocWriteState state, String segmentSuffix) throws IOException {
|
||||
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?
|
||||
}
|
||||
@ -50,7 +53,7 @@ public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
|
||||
// lazy init
|
||||
if (directory == null) {
|
||||
directory = new CompoundFileDirectory(mainDirectory,
|
||||
IndexFileNames.segmentFileName(segmentName, DOC_VALUES_SEGMENT_SUFFIX,
|
||||
IndexFileNames.segmentFileName(segmentName, segmentSuffix,
|
||||
IndexFileNames.COMPOUND_FILE_EXTENSION), context, true);
|
||||
}
|
||||
return directory;
|
||||
@ -75,4 +78,15 @@ public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {}
|
||||
IOUtils.deleteFilesIgnoringExceptions(mainDirectory, IndexFileNames.segmentFileName(
|
||||
segmentName, segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
|
||||
IndexFileNames.segmentFileName(segmentName, segmentSuffix,
|
||||
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
}
|
||||
}
|
||||
|
@ -32,12 +32,12 @@ public class Lucene40DocValuesFormat extends DocValuesFormat {
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new Lucene40DocValuesConsumer(state);
|
||||
return new Lucene40DocValuesConsumer(state, Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return new Lucene40DocValuesProducer(state);
|
||||
return new Lucene40DocValuesProducer(state, Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -39,16 +39,15 @@ import org.apache.lucene.util.IOUtils;
|
||||
public class Lucene40DocValuesProducer extends DocValuesReaderBase {
|
||||
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) throws IOException {
|
||||
if (state.fieldInfos.anyDocValuesFields()) {
|
||||
public Lucene40DocValuesProducer(SegmentReadState state, String segmentSuffix) throws IOException {
|
||||
if (anyDocValuesFields(state.fieldInfos)) {
|
||||
cfs = new CompoundFileDirectory(state.dir,
|
||||
IndexFileNames.segmentFileName(state.segmentInfo.name,
|
||||
Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION),
|
||||
segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
|
||||
state.context, false);
|
||||
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.docCount, cfs, state.context);
|
||||
} else {
|
||||
|
@ -1,5 +1,4 @@
|
||||
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
|
||||
@ -16,38 +15,117 @@ package org.apache.lucene.codecs.lucene40;
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.NormsReader;
|
||||
import org.apache.lucene.codecs.NormsWriter;
|
||||
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.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/**
|
||||
* Norms Format for the default codec.
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class Lucene40NormsFormat extends NormsFormat {
|
||||
|
||||
private final static String NORMS_SEGMENT_SUFFIX = "nrm";
|
||||
|
||||
@Override
|
||||
public NormsReader normsReader(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context, Directory separateNormsDir) throws IOException {
|
||||
return new Lucene40NormsReader(dir, info, fields, context, separateNormsDir);
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new Lucene40NormsDocValuesConsumer(state, NORMS_SEGMENT_SUFFIX);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NormsWriter normsWriter(SegmentWriteState state) throws IOException {
|
||||
return new Lucene40NormsWriter(state.directory, state.segmentName, state.context);
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return new Lucene40NormsDocValuesProducer(state, NORMS_SEGMENT_SUFFIX);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40NormsReader.files(dir, info, files);
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files)
|
||||
throws IOException {
|
||||
Lucene40NormsDocValuesConsumer.files(dir, info, files);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
Lucene40NormsReader.separateFiles(dir, info, files);
|
||||
public PerDocProducer docsProducer(SegmentReadState state,
|
||||
Directory separateNormsDir) throws IOException {
|
||||
return docsProducer(state);
|
||||
}
|
||||
|
||||
|
||||
public static class Lucene40NormsDocValuesProducer extends Lucene40DocValuesProducer {
|
||||
|
||||
public Lucene40NormsDocValuesProducer(SegmentReadState state,
|
||||
String segmentSuffix) throws IOException {
|
||||
super(state, segmentSuffix);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean canLoad(FieldInfo info) {
|
||||
return !info.omitNorms && info.isIndexed;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Type getDocValuesType(FieldInfo info) {
|
||||
return Type.BYTES_FIXED_STRAIGHT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean anyDocValuesFields(FieldInfos infos) {
|
||||
return infos.hasNorms();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static class Lucene40NormsDocValuesConsumer extends Lucene40DocValuesConsumer {
|
||||
|
||||
public Lucene40NormsDocValuesConsumer(PerDocWriteState state,
|
||||
String segmentSuffix) throws IOException {
|
||||
super(state, segmentSuffix);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DocValues getDocValuesForMerge(IndexReader reader, FieldInfo info)
|
||||
throws IOException {
|
||||
return reader.normValues(info.name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean canMerge(FieldInfo info) {
|
||||
return !info.omitNorms && info.isIndexed;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Type getDocValuesType(FieldInfo info) {
|
||||
return Type.BYTES_FIXED_STRAIGHT;
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo, Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (!fieldInfo.omitNorms && fieldInfo.isIndexed) {
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
files.add(IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(segmentInfo.name, NORMS_SEGMENT_SUFFIX, IndexFileNames.COMPOUND_FILE_EXTENSION));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
@ -1,130 +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 java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.NormsWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
public class Lucene40NormsWriter extends NormsWriter {
|
||||
private IndexOutput out;
|
||||
private int normCount = 0;
|
||||
|
||||
/** norms header placeholder */
|
||||
static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
|
||||
|
||||
/** Extension of norms file */
|
||||
static final String NORMS_EXTENSION = "nrm";
|
||||
|
||||
/** Extension of separate norms file
|
||||
* @deprecated */
|
||||
@Deprecated
|
||||
static final String SEPARATE_NORMS_EXTENSION = "s";
|
||||
|
||||
public Lucene40NormsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
|
||||
boolean success = false;
|
||||
try {
|
||||
out = directory.createOutput(normsFileName, context);
|
||||
out.writeBytes(NORMS_HEADER, 0, NORMS_HEADER.length);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startField(FieldInfo info) throws IOException {
|
||||
assert info.omitNorms == false;
|
||||
normCount++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeNorm(byte norm) throws IOException {
|
||||
out.writeByte(norm);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int numDocs) throws IOException {
|
||||
if (4+normCount*(long)numDocs != out.getFilePointer()) {
|
||||
throw new RuntimeException(".nrm file size mismatch: expected=" + (4+normCount*(long)numDocs) + " actual=" + out.getFilePointer());
|
||||
}
|
||||
}
|
||||
|
||||
/** we override merge and bulk-merge norms when there are no deletions */
|
||||
@Override
|
||||
public int merge(MergeState mergeState) throws IOException {
|
||||
int numMergedDocs = 0;
|
||||
for (FieldInfo fi : mergeState.fieldInfos) {
|
||||
if (fi.isIndexed && !fi.omitNorms) {
|
||||
startField(fi);
|
||||
int numMergedDocsForField = 0;
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
byte normBuffer[] = reader.reader.norms(fi.name);
|
||||
if (normBuffer == null) {
|
||||
// Can be null if this segment doesn't have
|
||||
// any docs with this field
|
||||
normBuffer = new byte[maxDoc];
|
||||
Arrays.fill(normBuffer, (byte)0);
|
||||
}
|
||||
if (reader.liveDocs == null) {
|
||||
//optimized case for segments without deleted docs
|
||||
out.writeBytes(normBuffer, maxDoc);
|
||||
numMergedDocsForField += maxDoc;
|
||||
} else {
|
||||
// this segment has deleted docs, so we have to
|
||||
// check for every doc if it is deleted or not
|
||||
final Bits liveDocs = reader.liveDocs;
|
||||
for (int k = 0; k < maxDoc; k++) {
|
||||
if (liveDocs.get(k)) {
|
||||
numMergedDocsForField++;
|
||||
out.writeByte(normBuffer[k]);
|
||||
}
|
||||
}
|
||||
}
|
||||
mergeState.checkAbort.work(maxDoc);
|
||||
}
|
||||
assert numMergedDocs == 0 || numMergedDocs == numMergedDocsForField;
|
||||
numMergedDocs = numMergedDocsForField;
|
||||
}
|
||||
}
|
||||
finish(numMergedDocs);
|
||||
return numMergedDocs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(out);
|
||||
} finally {
|
||||
out = null;
|
||||
}
|
||||
}
|
||||
}
|
@ -122,14 +122,9 @@ public final class Lucene40StoredFieldsWriter extends StoredFieldsWriter {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {}
|
||||
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {}
|
||||
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {}
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory,
|
||||
IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION),
|
||||
IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
|
||||
}
|
||||
|
||||
public final void writeField(FieldInfo info, IndexableField field) throws IOException {
|
||||
|
@ -202,18 +202,9 @@ public final class Lucene40TermVectorsWriter extends TermVectorsWriter {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {}
|
||||
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_INDEX_EXTENSION));
|
||||
} catch (IOException ignored) {}
|
||||
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_DOCUMENTS_EXTENSION));
|
||||
} catch (IOException ignored) {}
|
||||
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {}
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_INDEX_EXTENSION),
|
||||
IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_DOCUMENTS_EXTENSION),
|
||||
IndexFileNames.segmentFileName(segment, "", Lucene40TermVectorsReader.VECTORS_FIELDS_EXTENSION));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -22,6 +22,7 @@ import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
@ -122,7 +123,7 @@ public final class Bytes {
|
||||
* @throws IOException
|
||||
* if the files for the writer can not be created.
|
||||
*/
|
||||
public static Writer getWriter(Directory dir, String id, Mode mode,
|
||||
public static DocValuesConsumer getWriter(Directory dir, String id, Mode mode,
|
||||
boolean fixedSize, Comparator<BytesRef> sortComparator,
|
||||
Counter bytesUsed, IOContext context, boolean fasterButMoreRam)
|
||||
throws IOException {
|
||||
@ -295,7 +296,8 @@ public final class Bytes {
|
||||
* skipped; they will be filled with 0 bytes.
|
||||
*/
|
||||
@Override
|
||||
public abstract void add(int docID, BytesRef bytes) throws IOException;
|
||||
protected
|
||||
abstract void add(int docID, BytesRef bytes) throws IOException;
|
||||
|
||||
@Override
|
||||
public abstract void finish(int docCount) throws IOException;
|
||||
@ -431,7 +433,7 @@ public final class Bytes {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
protected void add(int docID, BytesRef bytes) throws IOException {
|
||||
if (bytes.length == 0) { // default value - skip it
|
||||
return;
|
||||
}
|
||||
|
@ -64,7 +64,7 @@ class FixedStraightBytesImpl {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
protected void add(int docID, BytesRef bytes) throws IOException {
|
||||
assert lastDocID < docID;
|
||||
|
||||
if (size == -1) {
|
||||
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene40.values;
|
||||
*/
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
@ -44,7 +45,7 @@ public class Floats {
|
||||
protected static final int VERSION_START = 0;
|
||||
protected static final int VERSION_CURRENT = VERSION_START;
|
||||
|
||||
public static Writer getWriter(Directory dir, String id, Counter bytesUsed,
|
||||
public static DocValuesConsumer getWriter(Directory dir, String id, Counter bytesUsed,
|
||||
IOContext context, Type type) throws IOException {
|
||||
return new FloatsWriter(dir, id, bytesUsed, context, type);
|
||||
}
|
||||
@ -79,7 +80,7 @@ public class Floats {
|
||||
assert template != null;
|
||||
}
|
||||
|
||||
public void add(int docID, double v) throws IOException {
|
||||
protected void add(int docID, double v) throws IOException {
|
||||
template.toBytes(v, bytesRef);
|
||||
add(docID, bytesRef);
|
||||
}
|
||||
|
@ -19,6 +19,7 @@ package org.apache.lucene.codecs.lucene40.values;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
@ -42,7 +43,7 @@ public final class Ints {
|
||||
private Ints() {
|
||||
}
|
||||
|
||||
public static Writer getWriter(Directory dir, String id, Counter bytesUsed,
|
||||
public static DocValuesConsumer getWriter(Directory dir, String id, Counter bytesUsed,
|
||||
Type type, IOContext context) throws IOException {
|
||||
return type == Type.VAR_INTS ? new PackedIntValues.PackedIntsWriter(dir, id,
|
||||
bytesUsed, context) : new IntsWriter(dir, id, bytesUsed, context, type);
|
||||
@ -103,7 +104,7 @@ public final class Ints {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, long v) throws IOException {
|
||||
protected void add(int docID, long v) throws IOException {
|
||||
template.toBytes(v, bytesRef);
|
||||
add(docID, bytesRef);
|
||||
}
|
||||
|
@ -63,7 +63,7 @@ class PackedIntValues {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, long v) throws IOException {
|
||||
protected void add(int docID, long v) throws IOException {
|
||||
assert lastDocId < docID;
|
||||
if (!started) {
|
||||
started = true;
|
||||
|
@ -80,7 +80,7 @@ class VarStraightBytesImpl {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, BytesRef bytes) throws IOException {
|
||||
protected void add(int docID, BytesRef bytes) throws IOException {
|
||||
assert !merge;
|
||||
if (bytes.length == 0) {
|
||||
return; // default
|
||||
|
@ -20,11 +20,9 @@ import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.Counter;
|
||||
|
||||
@ -41,7 +39,6 @@ import org.apache.lucene.util.Counter;
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public abstract class Writer extends DocValuesConsumer {
|
||||
protected Source currentMergeSource;
|
||||
protected final Counter bytesUsed;
|
||||
|
||||
/**
|
||||
@ -66,102 +63,6 @@ public abstract class Writer extends DocValuesConsumer {
|
||||
*/
|
||||
public static final String DATA_EXTENSION = "dat";
|
||||
|
||||
/**
|
||||
* Records the specified <tt>long</tt> value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* <tt>long</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record <tt>long</tt> values
|
||||
*/
|
||||
public void add(int docID, long value) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Records the specified <tt>double</tt> value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* <tt>double</tt> values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record <tt>double</tt> values
|
||||
*/
|
||||
public void add(int docID, double value) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* Records the specified {@link BytesRef} value for the docID or throws an
|
||||
* {@link UnsupportedOperationException} if this {@link Writer} doesn't record
|
||||
* {@link BytesRef} values.
|
||||
*
|
||||
* @throws UnsupportedOperationException
|
||||
* if this writer doesn't record {@link BytesRef} values
|
||||
*/
|
||||
public void add(int docID, BytesRef value) throws IOException {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
/**
|
||||
* 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} set to <i>setNextMergeSource(Source)</i>.
|
||||
* <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 {@link Writer} implementation. The given document
|
||||
* ID must always be greater than the previous ID or <tt>0</tt> if called the
|
||||
* first time.
|
||||
*/
|
||||
protected abstract void mergeDoc(int docID, int sourceDoc) throws IOException;
|
||||
|
||||
/**
|
||||
* Sets the next {@link Source} to consume values from on calls to
|
||||
* {@link #mergeDoc(int, int)}
|
||||
*
|
||||
* @param mergeSource
|
||||
* the next {@link Source}, this must not be null
|
||||
*/
|
||||
protected void setNextMergeSource(Source mergeSource) {
|
||||
currentMergeSource = mergeSource;
|
||||
}
|
||||
|
||||
/**
|
||||
* Finish writing and close any files and resources used by this Writer.
|
||||
*
|
||||
* @param docCount
|
||||
* the total number of documents for this writer. This must be
|
||||
* greater that or equal to the largest document id passed to one of
|
||||
* the add methods after the {@link Writer} was created.
|
||||
*/
|
||||
public abstract void finish(int docCount) throws IOException;
|
||||
|
||||
@Override
|
||||
protected void merge(SingleSubMergeState state) 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 = state.reader.getDirectSource();
|
||||
assert source != null;
|
||||
setNextMergeSource(source); // set the current enum we are working on - the
|
||||
// impl. will get the correct reference for the type
|
||||
// it supports
|
||||
int docID = state.docBase;
|
||||
final Bits liveDocs = state.liveDocs;
|
||||
final int docCount = state.docCount;
|
||||
for (int i = 0; i < docCount; i++) {
|
||||
if (liveDocs == null || liveDocs.get(i)) {
|
||||
mergeDoc(docID++, i);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Factory method to create a {@link Writer} instance for a given type. This
|
||||
* method returns default implementations for each of the different types
|
||||
@ -181,7 +82,7 @@ public abstract class Writer extends DocValuesConsumer {
|
||||
* @return a new {@link Writer} instance for the given {@link Type}
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Writer create(Type type, String id, Directory directory,
|
||||
public static DocValuesConsumer create(Type type, String id, Directory directory,
|
||||
Comparator<BytesRef> comp, Counter bytesUsed, IOContext context, boolean fasterButMoreRam) throws IOException {
|
||||
if (comp == null) {
|
||||
comp = BytesRef.getUTF8SortedAsUnicodeComparator();
|
||||
|
@ -18,6 +18,7 @@ package org.apache.lucene.codecs.sep;
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesWriterBase;
|
||||
@ -28,6 +29,7 @@ import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Implementation of PerDocConsumer that uses separate files.
|
||||
@ -35,10 +37,11 @@ import org.apache.lucene.store.Directory;
|
||||
*/
|
||||
public class SepDocValuesConsumer extends DocValuesWriterBase {
|
||||
private final Directory directory;
|
||||
|
||||
private final FieldInfos fieldInfos;
|
||||
public SepDocValuesConsumer(PerDocWriteState state) throws IOException {
|
||||
super(state);
|
||||
this.directory = state.directory;
|
||||
fieldInfos = state.fieldInfos;
|
||||
}
|
||||
|
||||
@Override
|
||||
@ -46,13 +49,16 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
|
||||
return directory;
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough")
|
||||
public static void files(Directory dir, SegmentInfo segmentInfo,
|
||||
Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = segmentInfo.getFieldInfos();
|
||||
files(dir, segmentInfo.getFieldInfos(), segmentInfo.name, files);
|
||||
}
|
||||
|
||||
@SuppressWarnings("fallthrough")
|
||||
private static void files(Directory dir,FieldInfos fieldInfos, String segmentName, Set<String> files) {
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (fieldInfo.hasDocValues()) {
|
||||
String filename = docValuesId(segmentInfo.name, fieldInfo.number);
|
||||
String filename = docValuesId(segmentName, fieldInfo.number);
|
||||
switch (fieldInfo.getDocValuesType()) {
|
||||
case BYTES_FIXED_DEREF:
|
||||
case BYTES_VAR_DEREF:
|
||||
@ -61,8 +67,13 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
|
||||
case BYTES_VAR_SORTED:
|
||||
files.add(IndexFileNames.segmentFileName(filename, "",
|
||||
Writer.INDEX_EXTENSION));
|
||||
try {
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
|
||||
Writer.INDEX_EXTENSION));
|
||||
} catch (IOException e) {
|
||||
// don't throw checked exception - dir is only used in assert
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
// until here all types use an index
|
||||
case BYTES_FIXED_STRAIGHT:
|
||||
case FLOAT_32:
|
||||
@ -74,8 +85,13 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
|
||||
case FIXED_INTS_8:
|
||||
files.add(IndexFileNames.segmentFileName(filename, "",
|
||||
Writer.DATA_EXTENSION));
|
||||
try {
|
||||
assert dir.fileExists(IndexFileNames.segmentFileName(filename, "",
|
||||
Writer.DATA_EXTENSION));
|
||||
} catch (IOException e) {
|
||||
// don't throw checked exception - dir is only used in assert
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
break;
|
||||
default:
|
||||
assert false;
|
||||
@ -83,4 +99,11 @@ public class SepDocValuesConsumer extends DocValuesWriterBase {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
Set<String> files = new HashSet<String>();
|
||||
files(directory, fieldInfos, segmentName, files);
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, files.toArray(new String[0]));
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,276 @@
|
||||
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.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
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.IOUtils;
|
||||
|
||||
/**
|
||||
* Writes plain-text norms
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextNormsConsumer extends PerDocConsumer {
|
||||
|
||||
/** Extension of norms file */
|
||||
static final String NORMS_EXTENSION = "len";
|
||||
final static BytesRef END = new BytesRef("END");
|
||||
final static BytesRef FIELD = new BytesRef("field ");
|
||||
final static BytesRef DOC = new BytesRef(" doc ");
|
||||
final static BytesRef NORM = new BytesRef(" norm ");
|
||||
|
||||
private NormsWriter writer;
|
||||
|
||||
private final Directory directory;
|
||||
|
||||
private final String segment;
|
||||
|
||||
private final IOContext context;
|
||||
|
||||
public SimpleTextNormsConsumer(Directory directory, String segment,
|
||||
IOContext context) throws IOException {
|
||||
this.directory = directory;
|
||||
this.segment = segment;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (writer != null) {
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DocValues getDocValuesForMerge(IndexReader reader, FieldInfo info)
|
||||
throws IOException {
|
||||
return reader.normValues(info.name);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean canMerge(FieldInfo info) {
|
||||
return !info.omitNorms && info.isIndexed;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Type getDocValuesType(FieldInfo info) {
|
||||
return Type.BYTES_FIXED_STRAIGHT;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
|
||||
throws IOException {
|
||||
return new SimpleTextNormsDocValuesConsumer(fieldInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
if (writer != null) {
|
||||
try {
|
||||
writer.abort();
|
||||
} catch (IOException e) {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private class SimpleTextNormsDocValuesConsumer extends DocValuesConsumer {
|
||||
// Holds all docID/norm pairs we've seen
|
||||
int[] docIDs = new int[1];
|
||||
byte[] norms = new byte[1];
|
||||
int upto;
|
||||
private final FieldInfo fi;
|
||||
|
||||
public SimpleTextNormsDocValuesConsumer(FieldInfo fieldInfo) {
|
||||
fi = fieldInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, DocValue docValue) throws IOException {
|
||||
add(docID, docValue.getBytes());
|
||||
}
|
||||
|
||||
protected void add(int docID, BytesRef value) throws IOException {
|
||||
if (docIDs.length <= upto) {
|
||||
assert docIDs.length == upto;
|
||||
docIDs = ArrayUtil.grow(docIDs, 1 + upto);
|
||||
}
|
||||
if (norms.length <= upto) {
|
||||
assert norms.length == upto;
|
||||
norms = ArrayUtil.grow(norms, 1 + upto);
|
||||
}
|
||||
assert value.length == 1;
|
||||
norms[upto] = value.bytes[value.offset];
|
||||
docIDs[upto] = docID;
|
||||
upto++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
final NormsWriter normsWriter = getNormsWriter();
|
||||
boolean success = false;
|
||||
try {
|
||||
int uptoDoc = 0;
|
||||
normsWriter.setNumTotalDocs(docCount);
|
||||
if (upto > 0) {
|
||||
normsWriter.startField(fi);
|
||||
int docID = 0;
|
||||
for (; docID < docCount; docID++) {
|
||||
if (uptoDoc < upto && docIDs[uptoDoc] == docID) {
|
||||
normsWriter.writeNorm(norms[uptoDoc]);
|
||||
uptoDoc++;
|
||||
} else {
|
||||
normsWriter.writeNorm((byte) 0);
|
||||
}
|
||||
}
|
||||
// we should have consumed every norm
|
||||
assert uptoDoc == upto;
|
||||
|
||||
} else {
|
||||
// Fill entire field with default norm:
|
||||
normsWriter.startField(fi);
|
||||
for (; upto < docCount; upto++)
|
||||
normsWriter.writeNorm((byte) 0);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
normsWriter.abort();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public NormsWriter getNormsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
writer = new NormsWriter(directory, segment, context);
|
||||
}
|
||||
return writer;
|
||||
}
|
||||
|
||||
private static class NormsWriter {
|
||||
|
||||
private final IndexOutput output;
|
||||
private int numTotalDocs = 0;
|
||||
private int docid = 0;
|
||||
|
||||
private final BytesRef scratch = new BytesRef();
|
||||
|
||||
|
||||
public NormsWriter(Directory directory, String segment, IOContext context)
|
||||
throws IOException {
|
||||
final String normsFileName = IndexFileNames.segmentFileName(segment, "",
|
||||
NORMS_EXTENSION);
|
||||
output = directory.createOutput(normsFileName, context);
|
||||
|
||||
}
|
||||
|
||||
public void startField(FieldInfo info) throws IOException {
|
||||
assert info.omitNorms == false;
|
||||
docid = 0;
|
||||
write(FIELD);
|
||||
write(info.name);
|
||||
newLine();
|
||||
}
|
||||
|
||||
public void writeNorm(byte norm) throws IOException {
|
||||
write(DOC);
|
||||
write(Integer.toString(docid));
|
||||
newLine();
|
||||
|
||||
write(NORM);
|
||||
write(norm);
|
||||
newLine();
|
||||
docid++;
|
||||
}
|
||||
|
||||
public void finish(int numDocs) throws IOException {
|
||||
if (docid != numDocs) {
|
||||
throw new RuntimeException(
|
||||
"mergeNorms produced an invalid result: docCount is " + numDocs
|
||||
+ " but only saw " + docid + " file=" + output.toString()
|
||||
+ "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
write(END);
|
||||
newLine();
|
||||
}
|
||||
|
||||
private void write(String s) throws IOException {
|
||||
SimpleTextUtil.write(output, s, scratch);
|
||||
}
|
||||
|
||||
private void write(BytesRef bytes) throws IOException {
|
||||
SimpleTextUtil.write(output, bytes);
|
||||
}
|
||||
|
||||
private void write(byte b) throws IOException {
|
||||
scratch.grow(1);
|
||||
scratch.bytes[scratch.offset] = b;
|
||||
scratch.length = 1;
|
||||
SimpleTextUtil.write(output, scratch);
|
||||
}
|
||||
|
||||
private void newLine() throws IOException {
|
||||
SimpleTextUtil.writeNewline(output);
|
||||
}
|
||||
|
||||
public void setNumTotalDocs(int numTotalDocs) {
|
||||
assert this.numTotalDocs == 0 || numTotalDocs == this.numTotalDocs;
|
||||
this.numTotalDocs = numTotalDocs;
|
||||
}
|
||||
|
||||
public void abort() throws IOException {
|
||||
IOUtils.close(output);
|
||||
}
|
||||
|
||||
public void finish() throws IOException {
|
||||
finish(numTotalDocs);
|
||||
IOUtils.close(output);
|
||||
}
|
||||
}
|
||||
|
||||
public static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
FieldInfos fieldInfos = info.getFieldInfos();
|
||||
|
||||
for (FieldInfo fieldInfo : fieldInfos) {
|
||||
if (!fieldInfo.omitNorms && fieldInfo.isIndexed) {
|
||||
files.add(IndexFileNames.segmentFileName(info.name, "",
|
||||
NORMS_EXTENSION));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -21,13 +21,12 @@ import java.io.IOException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.NormsReader;
|
||||
import org.apache.lucene.codecs.NormsWriter;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.apache.lucene.index.SegmentReadState;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
|
||||
/**
|
||||
* plain-text norms format
|
||||
@ -36,19 +35,28 @@ import org.apache.lucene.store.IOContext;
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextNormsFormat extends NormsFormat {
|
||||
|
||||
|
||||
@Override
|
||||
public NormsReader normsReader(Directory dir, SegmentInfo info, FieldInfos fields, IOContext context, Directory separateNormsDir) throws IOException {
|
||||
return new SimpleTextNormsReader(dir, info, fields, context);
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new SimpleTextNormsConsumer(state.directory, state.segmentName, state.context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public NormsWriter normsWriter(SegmentWriteState state) throws IOException {
|
||||
return new SimpleTextNormsWriter(state.directory, state.segmentName, state.context);
|
||||
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
|
||||
return new SimpleTextNormsProducer(state.dir, state.segmentInfo, state.fieldInfos, state.context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
SimpleTextNormsReader.files(dir, info, files);
|
||||
public void files(Directory dir, SegmentInfo info, Set<String> files)
|
||||
throws IOException {
|
||||
SimpleTextNormsConsumer.files(dir, info, files);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public PerDocProducer docsProducer(SegmentReadState state,
|
||||
Directory separateNormsDir) throws IOException {
|
||||
return docsProducer(state);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -17,12 +17,21 @@ package org.apache.lucene.codecs.simpletext;
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.DOC;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.END;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.FIELD;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.NORM;
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsConsumer.NORMS_EXTENSION;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.codecs.NormsReader;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.FieldInfos;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.SegmentInfo;
|
||||
@ -33,18 +42,17 @@ import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
|
||||
import static org.apache.lucene.codecs.simpletext.SimpleTextNormsWriter.*;
|
||||
|
||||
/**
|
||||
* Reads plain-text norms
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextNormsReader extends NormsReader {
|
||||
private Map<String,byte[]> norms = new HashMap<String,byte[]>();
|
||||
public class SimpleTextNormsProducer extends PerDocProducer {
|
||||
|
||||
public SimpleTextNormsReader(Directory directory, SegmentInfo si, FieldInfos fields, IOContext context) throws IOException {
|
||||
Map<String,NormsDocValues> norms = new HashMap<String,NormsDocValues>();
|
||||
|
||||
public SimpleTextNormsProducer(Directory directory, SegmentInfo si, FieldInfos fields, IOContext context) throws IOException {
|
||||
if (fields.hasNorms()) {
|
||||
readNorms(directory.openInput(IndexFileNames.segmentFileName(si.name, "", NORMS_EXTENSION), context), si.docCount);
|
||||
}
|
||||
@ -58,7 +66,7 @@ public class SimpleTextNormsReader extends NormsReader {
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
while (!scratch.equals(END)) {
|
||||
assert StringHelper.startsWith(scratch, FIELD);
|
||||
String fieldName = readString(FIELD.length, scratch);
|
||||
final String fieldName = readString(FIELD.length, scratch);
|
||||
byte bytes[] = new byte[maxDoc];
|
||||
for (int i = 0; i < bytes.length; i++) {
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
@ -67,7 +75,7 @@ public class SimpleTextNormsReader extends NormsReader {
|
||||
assert StringHelper.startsWith(scratch, NORM);
|
||||
bytes[i] = scratch.bytes[scratch.offset + NORM.length];
|
||||
}
|
||||
norms.put(fieldName, bytes);
|
||||
norms.put(fieldName, new NormsDocValues(new Norm(bytes)));
|
||||
SimpleTextUtil.readLine(in, scratch);
|
||||
assert StringHelper.startsWith(scratch, FIELD) || scratch.equals(END);
|
||||
}
|
||||
@ -81,11 +89,6 @@ public class SimpleTextNormsReader extends NormsReader {
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] norms(String name) throws IOException {
|
||||
return norms.get(name);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
norms = null;
|
||||
@ -94,7 +97,7 @@ public class SimpleTextNormsReader extends NormsReader {
|
||||
static void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
|
||||
// TODO: This is what SI always did... but we can do this cleaner?
|
||||
// like first FI that has norms but doesn't have separate norms?
|
||||
final String normsFileName = IndexFileNames.segmentFileName(info.name, "", SimpleTextNormsWriter.NORMS_EXTENSION);
|
||||
final String normsFileName = IndexFileNames.segmentFileName(info.name, "", SimpleTextNormsConsumer.NORMS_EXTENSION);
|
||||
if (dir.fileExists(normsFileName)) {
|
||||
files.add(normsFileName);
|
||||
}
|
||||
@ -103,4 +106,58 @@ public class SimpleTextNormsReader extends NormsReader {
|
||||
private String readString(int offset, BytesRef scratch) {
|
||||
return new String(scratch.bytes, scratch.offset+offset, scratch.length-offset, IOUtils.CHARSET_UTF_8);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
return norms.get(field);
|
||||
}
|
||||
|
||||
private class NormsDocValues extends DocValues {
|
||||
private final Source source;
|
||||
public NormsDocValues(Source source) {
|
||||
this.source = source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source load() throws IOException {
|
||||
return source;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Source getDirectSource() throws IOException {
|
||||
return getSource();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Type type() {
|
||||
return Type.BYTES_FIXED_STRAIGHT;
|
||||
}
|
||||
}
|
||||
|
||||
static final class Norm extends Source {
|
||||
protected Norm(byte[] bytes) {
|
||||
super(Type.BYTES_FIXED_STRAIGHT);
|
||||
this.bytes = bytes;
|
||||
}
|
||||
final byte bytes[];
|
||||
|
||||
@Override
|
||||
public BytesRef getBytes(int docID, BytesRef ref) {
|
||||
ref.bytes = bytes;
|
||||
ref.offset = docID;
|
||||
ref.length = 1;
|
||||
return ref;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasArray() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getArray() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
@ -1,114 +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.NormsWriter;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
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.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Writes plain-text norms
|
||||
* <p>
|
||||
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public class SimpleTextNormsWriter extends NormsWriter {
|
||||
private IndexOutput out;
|
||||
private int docid = 0;
|
||||
|
||||
/** Extension of norms file */
|
||||
static final String NORMS_EXTENSION = "len";
|
||||
|
||||
private final BytesRef scratch = new BytesRef();
|
||||
|
||||
final static BytesRef END = new BytesRef("END");
|
||||
final static BytesRef FIELD = new BytesRef("field ");
|
||||
final static BytesRef DOC = new BytesRef(" doc ");
|
||||
final static BytesRef NORM = new BytesRef(" norm ");
|
||||
|
||||
public SimpleTextNormsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
|
||||
out = directory.createOutput(normsFileName, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startField(FieldInfo info) throws IOException {
|
||||
assert info.omitNorms == false;
|
||||
docid = 0;
|
||||
write(FIELD);
|
||||
write(info.name);
|
||||
newLine();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeNorm(byte norm) throws IOException {
|
||||
write(DOC);
|
||||
write(Integer.toString(docid));
|
||||
newLine();
|
||||
|
||||
write(NORM);
|
||||
write(norm);
|
||||
newLine();
|
||||
docid++;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int numDocs) throws IOException {
|
||||
if (docid != numDocs) {
|
||||
throw new RuntimeException("mergeNorms produced an invalid result: docCount is " + numDocs
|
||||
+ " but only saw " + docid + " file=" + out.toString() + "; now aborting this merge to prevent index corruption");
|
||||
}
|
||||
write(END);
|
||||
newLine();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
IOUtils.close(out);
|
||||
} finally {
|
||||
out = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void write(String s) throws IOException {
|
||||
SimpleTextUtil.write(out, s, scratch);
|
||||
}
|
||||
|
||||
private void write(BytesRef bytes) throws IOException {
|
||||
SimpleTextUtil.write(out, bytes);
|
||||
}
|
||||
|
||||
private void write(byte b) throws IOException {
|
||||
scratch.grow(1);
|
||||
scratch.bytes[scratch.offset] = b;
|
||||
scratch.length = 1;
|
||||
SimpleTextUtil.write(out, scratch);
|
||||
}
|
||||
|
||||
private void newLine() throws IOException {
|
||||
SimpleTextUtil.writeNewline(out);
|
||||
}
|
||||
}
|
@ -166,9 +166,7 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {}
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
|
||||
} catch (IOException ignored) {}
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -149,10 +149,7 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
|
||||
try {
|
||||
close();
|
||||
} catch (IOException ignored) {}
|
||||
|
||||
try {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION));
|
||||
} catch (IOException ignored) {}
|
||||
IOUtils.deleteFilesIgnoringExceptions(directory, IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -117,11 +117,6 @@ abstract class BaseMultiReader<R extends IndexReader> extends IndexReader {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized byte[] norms(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiNorms.norms, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level norms");
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq(String field, BytesRef t) throws IOException {
|
||||
ensureOpen();
|
||||
@ -157,4 +152,9 @@ abstract class BaseMultiReader<R extends IndexReader> extends IndexReader {
|
||||
public DocValues docValues(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiDocValues#getDocValues, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level DocValues");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues normValues(String field) throws IOException {
|
||||
throw new UnsupportedOperationException("please use MultiDocValues#getNormValues, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Norm DocValues ");
|
||||
}
|
||||
}
|
||||
|
@ -131,6 +131,17 @@ final class DocFieldProcessor extends DocConsumer {
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
PerDocConsumer perDocConsumer = perDocConsumers.get(0);
|
||||
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;
|
||||
@ -329,7 +340,6 @@ final class DocFieldProcessor extends DocConsumer {
|
||||
perDocConsumer = dvFormat.docsConsumer(perDocWriteState);
|
||||
perDocConsumers.put(0, perDocConsumer);
|
||||
}
|
||||
|
||||
DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
|
||||
fieldInfo.setDocValuesType(valueType);
|
||||
|
||||
|
@ -18,7 +18,6 @@ package org.apache.lucene.index;
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Reader;
|
||||
import org.apache.lucene.analysis.TokenStream;
|
||||
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
|
||||
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
@ -34,7 +33,6 @@ import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
|
||||
|
||||
final class DocInverterPerField extends DocFieldConsumerPerField {
|
||||
|
||||
final private DocInverter parent;
|
||||
final FieldInfo fieldInfo;
|
||||
final InvertedDocConsumerPerField consumer;
|
||||
final InvertedDocEndConsumerPerField endConsumer;
|
||||
@ -42,7 +40,6 @@ final class DocInverterPerField extends DocFieldConsumerPerField {
|
||||
final FieldInvertState fieldState;
|
||||
|
||||
public DocInverterPerField(DocInverter parent, FieldInfo fieldInfo) {
|
||||
this.parent = parent;
|
||||
this.fieldInfo = fieldInfo;
|
||||
docState = parent.docState;
|
||||
fieldState = parent.fieldState;
|
||||
|
@ -20,8 +20,6 @@ import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Type; // javadocs
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/**
|
||||
@ -32,36 +30,6 @@ import org.apache.lucene.util.BytesRef;
|
||||
*/
|
||||
public interface DocValue {
|
||||
|
||||
/**
|
||||
* Sets the given <code>long</code> value.
|
||||
*/
|
||||
public void setInt(long value);
|
||||
|
||||
/**
|
||||
* Sets the given <code>float</code> value.
|
||||
*/
|
||||
public void setFloat(float value);
|
||||
|
||||
/**
|
||||
* Sets the given <code>double</code> value.
|
||||
*/
|
||||
public void setFloat(double value);
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value and the field's {@link Type}. The
|
||||
* comparator for this field is set to <code>null</code>. If a
|
||||
* <code>null</code> comparator is set the default comparator for the given
|
||||
* {@link Type} is used.
|
||||
*/
|
||||
public void setBytes(BytesRef value, DocValues.Type type);
|
||||
|
||||
/**
|
||||
* Sets the given {@link BytesRef} value, the field's {@link Type} and the
|
||||
* field's comparator. If the {@link Comparator} is set to <code>null</code>
|
||||
* the default for the given {@link Type} is used instead.
|
||||
*/
|
||||
public void setBytes(BytesRef value, DocValues.Type type, Comparator<BytesRef> comp);
|
||||
|
||||
/**
|
||||
* Returns the set {@link BytesRef} or <code>null</code> if not set.
|
||||
*/
|
||||
@ -82,19 +50,4 @@ public interface DocValue {
|
||||
*/
|
||||
public long getInt();
|
||||
|
||||
/**
|
||||
* Sets the {@link BytesRef} comparator for this field. If the field has a
|
||||
* numeric {@link Type} the comparator will be ignored.
|
||||
*/
|
||||
public void setBytesComparator(Comparator<BytesRef> comp);
|
||||
|
||||
/**
|
||||
* Sets the {@link Type}
|
||||
*/
|
||||
public void setDocValuesType(DocValues.Type type);
|
||||
|
||||
/**
|
||||
* Returns the {@link Type}
|
||||
*/
|
||||
public DocValues.Type docValueType();
|
||||
}
|
||||
|
@ -331,12 +331,6 @@ public class FilterIndexReader extends IndexReader {
|
||||
return in.hasNorms(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] norms(String f) throws IOException {
|
||||
ensureOpen();
|
||||
return in.norms(f);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq(String field, BytesRef t) throws IOException {
|
||||
ensureOpen();
|
||||
@ -419,6 +413,12 @@ public class FilterIndexReader extends IndexReader {
|
||||
ensureOpen();
|
||||
return in.docValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues normValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return in.normValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexCommit getIndexCommit() throws IOException {
|
||||
|
@ -29,6 +29,7 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.DocumentStoredFieldVisitor;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.search.SearcherManager; // javadocs
|
||||
import org.apache.lucene.store.*;
|
||||
import org.apache.lucene.util.Bits;
|
||||
@ -735,7 +736,17 @@ public abstract class IndexReader implements Closeable {
|
||||
*
|
||||
* @see org.apache.lucene.document.Field#setBoost(float)
|
||||
*/
|
||||
public abstract byte[] norms(String field) throws IOException;
|
||||
// TODO: cut over to source once we support other formats like float
|
||||
public final byte[] norms(String field) throws IOException {
|
||||
DocValues docValues = normValues(field);
|
||||
if (docValues != null) {
|
||||
Source source = docValues.getSource();
|
||||
assert source.hasArray(); // TODO cut over to source
|
||||
return (byte[])source.getArray();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Returns {@link Fields} for this reader.
|
||||
@ -1056,6 +1067,8 @@ public abstract class IndexReader implements Closeable {
|
||||
* using {@link ReaderUtil#gatherSubReaders} and iterate
|
||||
* through them yourself. */
|
||||
public abstract DocValues docValues(String field) throws IOException;
|
||||
|
||||
public abstract DocValues normValues(String field) throws IOException;
|
||||
|
||||
private volatile Fields fields;
|
||||
|
||||
|
@ -1,5 +1,7 @@
|
||||
package org.apache.lucene.index;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
@ -18,6 +20,6 @@ package org.apache.lucene.index;
|
||||
*/
|
||||
|
||||
abstract class InvertedDocEndConsumerPerField {
|
||||
abstract void finish();
|
||||
abstract void finish() throws IOException;
|
||||
abstract void abort();
|
||||
}
|
||||
|
@ -17,6 +17,7 @@ package org.apache.lucene.index;
|
||||
* limitations under the License.
|
||||
*/
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Array;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
@ -33,6 +34,13 @@ import org.apache.lucene.util.ReaderUtil.Gather;
|
||||
* @lucene.internal
|
||||
*/
|
||||
public class MultiDocValues extends DocValues {
|
||||
|
||||
private static DocValuesPuller DEFAULT_PULLER = new DocValuesPuller();
|
||||
private static final DocValuesPuller NORMS_PULLER = new DocValuesPuller() {
|
||||
public DocValues pull(IndexReader reader, String field) throws IOException {
|
||||
return reader.normValues(field);
|
||||
}
|
||||
};
|
||||
|
||||
public static class DocValuesSlice {
|
||||
public final static DocValuesSlice[] EMPTY_ARRAY = new DocValuesSlice[0];
|
||||
@ -46,6 +54,12 @@ public class MultiDocValues extends DocValues {
|
||||
this.length = length;
|
||||
}
|
||||
}
|
||||
|
||||
private static class DocValuesPuller {
|
||||
public DocValues pull(IndexReader reader, String field) throws IOException {
|
||||
return reader.docValues(field);
|
||||
}
|
||||
}
|
||||
|
||||
private DocValuesSlice[] slices;
|
||||
private int[] starts;
|
||||
@ -58,7 +72,6 @@ public class MultiDocValues extends DocValues {
|
||||
this.type = promotedType.type();
|
||||
this.valueSize = promotedType.getValueSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a single {@link DocValues} instance for this field, merging
|
||||
* their values on the fly.
|
||||
@ -68,15 +81,32 @@ public class MultiDocValues extends DocValues {
|
||||
* sub-readers (using {@link Gather}) and iterate through them yourself.
|
||||
*/
|
||||
public static DocValues getDocValues(IndexReader r, final String field) throws IOException {
|
||||
return getDocValues(r, field, DEFAULT_PULLER);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a single {@link DocValues} instance for this norms field, merging
|
||||
* their values on the fly.
|
||||
*
|
||||
* <p>
|
||||
* <b>NOTE</b>: this is a slow way to access DocValues. It's better to get the
|
||||
* sub-readers (using {@link Gather}) and iterate through them yourself.
|
||||
*/
|
||||
public static DocValues getNormDocValues(IndexReader r, final String field) throws IOException {
|
||||
return getDocValues(r, field, NORMS_PULLER);
|
||||
}
|
||||
|
||||
|
||||
private static DocValues getDocValues(IndexReader r, final String field, final DocValuesPuller puller) throws IOException {
|
||||
final IndexReader[] subs = r.getSequentialSubReaders();
|
||||
if (subs == null) {
|
||||
// already an atomic reader
|
||||
return r.docValues(field);
|
||||
return puller.pull(r, field);
|
||||
} else if (subs.length == 0) {
|
||||
// no fields
|
||||
return null;
|
||||
} else if (subs.length == 1) {
|
||||
return getDocValues(subs[0], field);
|
||||
return getDocValues(subs[0], field, puller);
|
||||
} else {
|
||||
final List<DocValuesSlice> slices = new ArrayList<DocValuesSlice>();
|
||||
|
||||
@ -89,7 +119,7 @@ public class MultiDocValues extends DocValues {
|
||||
new ReaderUtil.Gather(r) {
|
||||
@Override
|
||||
protected void add(int base, IndexReader r) throws IOException {
|
||||
final DocValues d = r.docValues(field);
|
||||
final DocValues d = puller.pull(r, field);
|
||||
if (d != null) {
|
||||
TypePromoter incoming = TypePromoter.create(d.type(), d.getValueSize());
|
||||
promotedType[0] = promotedType[0].promote(incoming);
|
||||
@ -195,6 +225,7 @@ public class MultiDocValues extends DocValues {
|
||||
private final int[] starts;
|
||||
private final DocValuesSlice[] slices;
|
||||
private boolean direct;
|
||||
private Object cachedArray; // cached array if supported
|
||||
|
||||
public MultiSource(DocValuesSlice[] slices, int[] starts, boolean direct, Type type) {
|
||||
super(type);
|
||||
@ -243,6 +274,77 @@ public class MultiDocValues extends DocValues {
|
||||
final int doc = ensureSource(docID);
|
||||
return current.getBytes(doc, bytesRef);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasArray() {
|
||||
boolean oneRealSource = false;
|
||||
for (DocValuesSlice slice : slices) {
|
||||
try {
|
||||
Source source = slice.docValues.getSource();
|
||||
if (source instanceof EmptySource) {
|
||||
/*
|
||||
* empty source marks a gap in the array skip if we encounter one
|
||||
*/
|
||||
continue;
|
||||
}
|
||||
oneRealSource = true;
|
||||
if (!source.hasArray()) {
|
||||
return false;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("load failed", e);
|
||||
}
|
||||
}
|
||||
return oneRealSource;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object getArray() {
|
||||
if (!hasArray()) {
|
||||
return null;
|
||||
}
|
||||
try {
|
||||
Class<?> componentType = null;
|
||||
Object[] arrays = new Object[slices.length];
|
||||
int numDocs = 0;
|
||||
for (int i = 0; i < slices.length; i++) {
|
||||
DocValuesSlice slice = slices[i];
|
||||
Source source = slice.docValues.getSource();
|
||||
Object array = null;
|
||||
if (!(source instanceof EmptySource)) {
|
||||
// EmptySource is skipped - marks a gap in the array
|
||||
array = source.getArray();
|
||||
}
|
||||
numDocs += slice.length;
|
||||
if (array != null) {
|
||||
if (componentType == null) {
|
||||
componentType = array.getClass().getComponentType();
|
||||
}
|
||||
assert componentType == array.getClass().getComponentType();
|
||||
}
|
||||
arrays[i] = array;
|
||||
}
|
||||
assert componentType != null;
|
||||
synchronized (this) {
|
||||
if (cachedArray != null) {
|
||||
return cachedArray;
|
||||
}
|
||||
final Object globalArray = Array.newInstance(componentType, numDocs);
|
||||
|
||||
for (int i = 0; i < slices.length; i++) {
|
||||
DocValuesSlice slice = slices[i];
|
||||
if (arrays[i] != null) {
|
||||
assert slice.length == Array.getLength(arrays[i]);
|
||||
System.arraycopy(arrays[i], 0, globalArray, slice.start,
|
||||
slice.length);
|
||||
}
|
||||
}
|
||||
return cachedArray = globalArray;
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("load failed", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: this is dup of DocValues.getDefaultSource()?
|
||||
@ -269,7 +371,7 @@ public class MultiDocValues extends DocValues {
|
||||
}
|
||||
}
|
||||
|
||||
private static class EmptyFixedSource extends Source {
|
||||
private static class EmptyFixedSource extends EmptySource {
|
||||
private final int valueSize;
|
||||
|
||||
public EmptyFixedSource(Type type, int valueSize) {
|
||||
|
@ -21,8 +21,12 @@ import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.NormsWriter;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
// TODO FI: norms could actually be stored as doc store
|
||||
@ -33,14 +37,21 @@ import org.apache.lucene.util.IOUtils;
|
||||
*/
|
||||
|
||||
final class NormsConsumer extends InvertedDocEndConsumer {
|
||||
final NormsFormat normsFormat;
|
||||
private final NormsFormat normsFormat;
|
||||
private PerDocConsumer consumer;
|
||||
private final DocumentsWriterPerThread dwpt;
|
||||
|
||||
public NormsConsumer(DocumentsWriterPerThread dwpt) {
|
||||
normsFormat = dwpt.codec.normsFormat();
|
||||
this.dwpt = dwpt;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {}
|
||||
public void abort(){
|
||||
if (consumer != null) {
|
||||
consumer.abort();
|
||||
}
|
||||
}
|
||||
|
||||
// We only write the _X.nrm file at flush
|
||||
void files(Collection<String> files) {}
|
||||
@ -49,50 +60,39 @@ final class NormsConsumer extends InvertedDocEndConsumer {
|
||||
* not disabled */
|
||||
@Override
|
||||
public void flush(Map<FieldInfo,InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
|
||||
if (!state.fieldInfos.hasNorms()) {
|
||||
return;
|
||||
}
|
||||
|
||||
NormsWriter normsOut = null;
|
||||
boolean success = false;
|
||||
boolean anythingFlushed = false;
|
||||
try {
|
||||
normsOut = normsFormat.normsWriter(state);
|
||||
|
||||
for (FieldInfo fi : state.fieldInfos) {
|
||||
final NormsConsumerPerField toWrite = (NormsConsumerPerField) fieldsToFlush.get(fi);
|
||||
int upto = 0;
|
||||
// we must check the final value of omitNorms for the fieldinfo, it could have
|
||||
// changed for this field since the first time we added it.
|
||||
if (!fi.omitNorms && toWrite != null && toWrite.upto > 0) {
|
||||
normsOut.startField(fi);
|
||||
int docID = 0;
|
||||
for (; docID < state.numDocs; docID++) {
|
||||
if (upto < toWrite.upto && toWrite.docIDs[upto] == docID) {
|
||||
normsOut.writeNorm(toWrite.norms[upto]);
|
||||
upto++;
|
||||
} else {
|
||||
normsOut.writeNorm((byte) 0);
|
||||
if (state.fieldInfos.hasNorms()) {
|
||||
for (FieldInfo fi : state.fieldInfos) {
|
||||
final NormsConsumerPerField toWrite = (NormsConsumerPerField) fieldsToFlush.get(fi);
|
||||
if (!fi.omitNorms) {
|
||||
if (toWrite != null) {
|
||||
anythingFlushed = true;
|
||||
toWrite.flush(state.numDocs);
|
||||
} else if (fi.isIndexed) {
|
||||
anythingFlushed = true;
|
||||
// we must check the final value of omitNorms for the fieldinfo, it could have
|
||||
// changed for this field since the first time we added it.
|
||||
final DocValuesConsumer valuesConsumer = newConsumer(new PerDocWriteState(state), fi);
|
||||
final DocValuesField value = new DocValuesField("");
|
||||
value.setBytes(new BytesRef(new byte[] {0x00}), Type.BYTES_FIXED_STRAIGHT);
|
||||
valuesConsumer.add(state.numDocs-1, value);
|
||||
valuesConsumer.finish(state.numDocs);
|
||||
}
|
||||
}
|
||||
|
||||
// we should have consumed every norm
|
||||
assert upto == toWrite.upto;
|
||||
|
||||
toWrite.reset();
|
||||
} else if (fi.isIndexed && !fi.omitNorms) {
|
||||
// Fill entire field with default norm:
|
||||
normsOut.startField(fi);
|
||||
for(;upto<state.numDocs;upto++)
|
||||
normsOut.writeNorm((byte) 0);
|
||||
}
|
||||
}
|
||||
normsOut.finish(state.numDocs);
|
||||
}
|
||||
|
||||
success = true;
|
||||
if (!anythingFlushed && consumer != null) {
|
||||
consumer.abort();
|
||||
}
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(normsOut);
|
||||
IOUtils.close(consumer);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(normsOut);
|
||||
IOUtils.closeWhileHandlingException(consumer);
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -106,6 +106,16 @@ final class NormsConsumer extends InvertedDocEndConsumer {
|
||||
@Override
|
||||
InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField,
|
||||
FieldInfo fieldInfo) {
|
||||
return new NormsConsumerPerField(docInverterPerField, fieldInfo);
|
||||
return new NormsConsumerPerField(docInverterPerField, fieldInfo, this);
|
||||
}
|
||||
|
||||
DocValuesConsumer newConsumer(PerDocWriteState perDocWriteState,
|
||||
FieldInfo fieldInfo) throws IOException {
|
||||
if (consumer == null) {
|
||||
consumer = normsFormat.docsConsumer(perDocWriteState);
|
||||
}
|
||||
DocValuesConsumer addValuesField = consumer.addValuesField(
|
||||
Type.BYTES_FIXED_STRAIGHT, fieldInfo);
|
||||
return addValuesField;
|
||||
}
|
||||
}
|
||||
|
@ -1,5 +1,4 @@
|
||||
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
|
||||
@ -16,65 +15,74 @@ package org.apache.lucene.index;
|
||||
* 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.document.DocValuesField;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.util.ArrayUtil;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
|
||||
/** Taps into DocInverter, as an InvertedDocEndConsumer,
|
||||
* which is called at the end of inverting each field. We
|
||||
* just look at the length for the field (docState.length)
|
||||
* and record the norm. */
|
||||
|
||||
final class NormsConsumerPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsConsumerPerField> {
|
||||
|
||||
final FieldInfo fieldInfo;
|
||||
final DocumentsWriterPerThread.DocState docState;
|
||||
final Similarity similarity;
|
||||
public class NormsConsumerPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsConsumerPerField> {
|
||||
private final FieldInfo fieldInfo;
|
||||
private final DocumentsWriterPerThread.DocState docState;
|
||||
private final Similarity similarity;
|
||||
private final FieldInvertState fieldState;
|
||||
private DocValuesConsumer consumer;
|
||||
private final DocValuesField value = new DocValuesField("");
|
||||
private final BytesRef spare = new BytesRef(1);
|
||||
private final NormsConsumer parent;
|
||||
|
||||
// Holds all docID/norm pairs we've seen
|
||||
int[] docIDs = new int[1];
|
||||
byte[] norms = new byte[1];
|
||||
int upto;
|
||||
|
||||
final FieldInvertState fieldState;
|
||||
|
||||
public void reset() {
|
||||
// Shrink back if we are overallocated now:
|
||||
docIDs = ArrayUtil.shrink(docIDs, upto);
|
||||
norms = ArrayUtil.shrink(norms, upto);
|
||||
upto = 0;
|
||||
}
|
||||
|
||||
public NormsConsumerPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
|
||||
public NormsConsumerPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo, NormsConsumer parent) {
|
||||
this.fieldInfo = fieldInfo;
|
||||
this.parent = parent;
|
||||
docState = docInverterPerField.docState;
|
||||
fieldState = docInverterPerField.fieldState;
|
||||
similarity = docState.similarityProvider.get(fieldInfo.name);
|
||||
}
|
||||
spare.length = 1;
|
||||
spare.offset = 0;
|
||||
|
||||
}
|
||||
@Override
|
||||
void abort() {
|
||||
upto = 0;
|
||||
}
|
||||
|
||||
public int compareTo(NormsConsumerPerField other) {
|
||||
return fieldInfo.name.compareTo(other.fieldInfo.name);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
void finish() {
|
||||
void finish() throws IOException {
|
||||
if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
|
||||
if (docIDs.length <= upto) {
|
||||
assert docIDs.length == upto;
|
||||
docIDs = ArrayUtil.grow(docIDs, 1+upto);
|
||||
}
|
||||
if (norms.length <= upto) {
|
||||
assert norms.length == upto;
|
||||
norms = ArrayUtil.grow(norms, 1+upto);
|
||||
}
|
||||
norms[upto] = similarity.computeNorm(fieldState);
|
||||
docIDs[upto] = docState.docID;
|
||||
upto++;
|
||||
DocValuesConsumer consumer = getConsumer();
|
||||
spare.bytes[0] = similarity.computeNorm(fieldState);
|
||||
value.setBytes(spare, Type.BYTES_FIXED_STRAIGHT);
|
||||
consumer.add(docState.docID, value);
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
void flush(int docCount) throws IOException {
|
||||
DocValuesConsumer consumer = this.consumer;
|
||||
if (consumer == null && fieldInfo.isIndexed) {
|
||||
consumer = getConsumer();
|
||||
spare.bytes[0] = 0x00;
|
||||
value.setBytes(spare, Type.BYTES_FIXED_STRAIGHT);
|
||||
consumer.add(docCount-1, value);
|
||||
}
|
||||
if (consumer != null) {
|
||||
consumer.finish(docCount);
|
||||
}
|
||||
}
|
||||
|
||||
private DocValuesConsumer getConsumer() throws IOException {
|
||||
if (consumer == null) {
|
||||
consumer = parent.newConsumer(docState.docWriter.newPerDocWriteState(""), fieldInfo);
|
||||
}
|
||||
return consumer;
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
void abort() {
|
||||
//
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -48,7 +48,7 @@ public class ParallelReader extends IndexReader {
|
||||
private SortedMap<String,IndexReader> fieldToReader = new TreeMap<String,IndexReader>();
|
||||
private Map<IndexReader,Collection<String>> readerToFields = new HashMap<IndexReader,Collection<String>>();
|
||||
private List<IndexReader> storedFieldReaders = new ArrayList<IndexReader>();
|
||||
private Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
|
||||
private Map<String, DocValues> normsCache = new HashMap<String,DocValues>();
|
||||
private final ReaderContext topLevelReaderContext = new AtomicReaderContext(this);
|
||||
private int maxDoc;
|
||||
private int numDocs;
|
||||
@ -336,27 +336,6 @@ public class ParallelReader extends IndexReader {
|
||||
return reader==null ? false : reader.hasNorms(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized byte[] norms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
IndexReader reader = fieldToReader.get(field);
|
||||
|
||||
if (reader==null)
|
||||
return null;
|
||||
|
||||
byte[] bytes = normsCache.get(field);
|
||||
if (bytes != null)
|
||||
return bytes;
|
||||
if (!hasNorms(field))
|
||||
return null;
|
||||
if (normsCache.containsKey(field)) // cached omitNorms, not missing key
|
||||
return null;
|
||||
|
||||
bytes = MultiNorms.norms(reader, field);
|
||||
normsCache.put(field, bytes);
|
||||
return bytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int docFreq(String field, BytesRef term) throws IOException {
|
||||
ensureOpen();
|
||||
@ -427,4 +406,16 @@ public class ParallelReader extends IndexReader {
|
||||
IndexReader reader = fieldToReader.get(field);
|
||||
return reader == null ? null : MultiDocValues.getDocValues(reader, field);
|
||||
}
|
||||
|
||||
// TODO: I suspect this is completely untested!!!!!
|
||||
@Override
|
||||
public synchronized DocValues normValues(String field) throws IOException {
|
||||
DocValues values = normsCache.get(field);
|
||||
if (values == null) {
|
||||
IndexReader reader = fieldToReader.get(field);
|
||||
values = reader == null ? null : MultiDocValues.getNormDocValues(reader, field);
|
||||
normsCache.put(field, values);
|
||||
}
|
||||
return values;
|
||||
}
|
||||
}
|
||||
|
@ -25,7 +25,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FieldsProducer;
|
||||
import org.apache.lucene.codecs.NormsReader;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
@ -54,7 +53,7 @@ final class SegmentCoreReaders {
|
||||
|
||||
final FieldsProducer fields;
|
||||
final PerDocProducer perDocProducer;
|
||||
final NormsReader norms;
|
||||
final PerDocProducer norms;
|
||||
|
||||
final Directory dir;
|
||||
final Directory cfsDir;
|
||||
@ -120,7 +119,7 @@ final class SegmentCoreReaders {
|
||||
// ask codec for its Norms:
|
||||
// TODO: since we don't write any norms file if there are no norms,
|
||||
// kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
|
||||
norms = codec.normsFormat().normsReader(cfsDir, si, fieldInfos, context, dir);
|
||||
norms = codec.normsFormat().docsProducer(segmentReadState, dir);
|
||||
perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
|
||||
|
||||
final Directory storeDir;
|
||||
|
@ -27,7 +27,6 @@ import java.util.Map;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.FieldInfosWriter;
|
||||
import org.apache.lucene.codecs.FieldsConsumer;
|
||||
import org.apache.lucene.codecs.NormsWriter;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.StoredFieldsWriter;
|
||||
import org.apache.lucene.codecs.TermVectorsWriter;
|
||||
@ -125,8 +124,7 @@ final class SegmentMerger {
|
||||
mergePerDoc(segmentWriteState);
|
||||
|
||||
if (mergeState.fieldInfos.hasNorms()) {
|
||||
int numMerged = mergeNorms(segmentWriteState);
|
||||
assert numMerged == mergeState.mergedDocCount;
|
||||
mergeNorms(segmentWriteState);
|
||||
}
|
||||
|
||||
if (mergeState.fieldInfos.hasVectors()) {
|
||||
@ -379,20 +377,24 @@ final class SegmentMerger {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private int mergeNorms(SegmentWriteState segmentWriteState) throws IOException {
|
||||
final NormsWriter writer = codec.normsFormat().normsWriter(segmentWriteState);
|
||||
|
||||
|
||||
private void mergeNorms(SegmentWriteState segmentWriteState) throws IOException {
|
||||
final PerDocConsumer docsConsumer = codec.normsFormat()
|
||||
.docsConsumer(new PerDocWriteState(segmentWriteState));
|
||||
// TODO: remove this check when 3.x indexes are no longer supported
|
||||
// (3.x indexes don't have docvalues)
|
||||
if (docsConsumer == null) {
|
||||
return;
|
||||
}
|
||||
boolean success = false;
|
||||
try {
|
||||
int numMerged = writer.merge(mergeState);
|
||||
docsConsumer.merge(mergeState);
|
||||
success = true;
|
||||
return numMerged;
|
||||
} finally {
|
||||
if (success) {
|
||||
IOUtils.close(writer);
|
||||
IOUtils.close(docsConsumer);
|
||||
} else {
|
||||
IOUtils.closeWhileHandlingException(writer);
|
||||
IOUtils.closeWhileHandlingException(docsConsumer);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -18,16 +18,15 @@ package org.apache.lucene.index;
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.codecs.StoredFieldsReader;
|
||||
import org.apache.lucene.codecs.TermVectorsReader;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.FieldInfo.IndexOptions;
|
||||
import org.apache.lucene.search.FieldCache; // javadocs
|
||||
import org.apache.lucene.store.IOContext;
|
||||
@ -262,12 +261,6 @@ public final class SegmentReader extends IndexReader {
|
||||
return fi != null && fi.isIndexed && !fi.omitNorms;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] norms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
return core.norms.norms(field);
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public TermVectorsReader getTermVectorsReader() {
|
||||
ensureOpen();
|
||||
@ -352,6 +345,17 @@ public final class SegmentReader extends IndexReader {
|
||||
}
|
||||
return perDoc.docValues(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValues normValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
final PerDocProducer perDoc = core.norms;
|
||||
if (perDoc == null) {
|
||||
return null;
|
||||
}
|
||||
return perDoc.docValues(field);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Called when the shared core for this SegmentReader
|
||||
|
@ -34,7 +34,7 @@ import org.apache.lucene.index.MultiReader; // javadoc
|
||||
* IndexReader#getSequentialSubReaders}) to emulate an
|
||||
* atomic reader. This requires implementing the postings
|
||||
* APIs on-the-fly, using the static methods in {@link
|
||||
* MultiFields}, {@link MultiNorms}, {@link MultiDocValues},
|
||||
* MultiFields}, {@link MultiDocValues},
|
||||
* by stepping through the sub-readers to merge fields/terms,
|
||||
* appending docs, etc.
|
||||
*
|
||||
@ -53,7 +53,7 @@ import org.apache.lucene.index.MultiReader; // javadoc
|
||||
public final class SlowMultiReaderWrapper extends FilterIndexReader {
|
||||
|
||||
private final ReaderContext readerContext;
|
||||
private final Map<String,byte[]> normsCache = new HashMap<String,byte[]>();
|
||||
private final Map<String, DocValues> normsCache = new HashMap<String, DocValues>();
|
||||
|
||||
public SlowMultiReaderWrapper(IndexReader other) {
|
||||
super(other);
|
||||
@ -76,7 +76,17 @@ public final class SlowMultiReaderWrapper extends FilterIndexReader {
|
||||
ensureOpen();
|
||||
return MultiDocValues.getDocValues(in, field);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public synchronized DocValues normValues(String field) throws IOException {
|
||||
ensureOpen();
|
||||
DocValues values = normsCache.get(field);
|
||||
if (values == null) {
|
||||
values = MultiDocValues.getNormDocValues(in, field);
|
||||
normsCache.put(field, values);
|
||||
}
|
||||
return values;
|
||||
}
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
ensureOpen();
|
||||
@ -87,22 +97,6 @@ public final class SlowMultiReaderWrapper extends FilterIndexReader {
|
||||
public IndexReader[] getSequentialSubReaders() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized byte[] norms(String field) throws IOException {
|
||||
ensureOpen();
|
||||
byte[] bytes = normsCache.get(field);
|
||||
if (bytes != null)
|
||||
return bytes;
|
||||
if (!hasNorms(field))
|
||||
return null;
|
||||
if (normsCache.containsKey(field)) // cached omitNorms, not missing key
|
||||
return null;
|
||||
|
||||
bytes = MultiNorms.norms(in, field);
|
||||
normsCache.put(field, bytes);
|
||||
return bytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReaderContext getTopReaderContext() {
|
||||
|
@ -91,7 +91,7 @@ final class CompoundFileWriter implements Closeable{
|
||||
// all entries that are written to a sep. file but not yet moved into CFS
|
||||
private final Queue<FileEntry> pendingEntries = new LinkedList<FileEntry>();
|
||||
private boolean closed = false;
|
||||
private volatile IndexOutput dataOut;
|
||||
private IndexOutput dataOut;
|
||||
private final AtomicBoolean outputTaken = new AtomicBoolean(false);
|
||||
final String entryTableName;
|
||||
final String dataFileName;
|
||||
@ -113,16 +113,25 @@ final class CompoundFileWriter implements Closeable{
|
||||
IndexFileNames.stripExtension(name), "",
|
||||
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION);
|
||||
dataFileName = name;
|
||||
boolean success = false;
|
||||
try {
|
||||
dataOut = directory.createOutput(dataFileName, IOContext.DEFAULT);
|
||||
dataOut.writeVInt(FORMAT_CURRENT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(dataOut);
|
||||
|
||||
}
|
||||
|
||||
private synchronized IndexOutput getOutput() throws IOException {
|
||||
if (dataOut == null) {
|
||||
IndexOutput dataOutput = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
dataOutput = directory.createOutput(dataFileName, IOContext.DEFAULT);
|
||||
dataOutput.writeVInt(FORMAT_CURRENT);
|
||||
dataOut = dataOutput;
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(dataOutput);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return dataOut;
|
||||
}
|
||||
|
||||
/** Returns the directory of the compound file. */
|
||||
@ -154,6 +163,7 @@ final class CompoundFileWriter implements Closeable{
|
||||
}
|
||||
closed = true;
|
||||
// open the compound stream
|
||||
getOutput();
|
||||
assert dataOut != null;
|
||||
long finalLength = dataOut.getFilePointer();
|
||||
assert assertFileLength(finalLength, dataOut);
|
||||
@ -246,7 +256,7 @@ final class CompoundFileWriter implements Closeable{
|
||||
seenIDs.add(id);
|
||||
final DirectCFSIndexOutput out;
|
||||
if (outputTaken.compareAndSet(false, true)) {
|
||||
out = new DirectCFSIndexOutput(dataOut, entry, false);
|
||||
out = new DirectCFSIndexOutput(getOutput(), entry, false);
|
||||
outputLocked = true;
|
||||
success = true;
|
||||
} else {
|
||||
@ -280,7 +290,7 @@ final class CompoundFileWriter implements Closeable{
|
||||
try {
|
||||
while (!pendingEntries.isEmpty()) {
|
||||
FileEntry entry = pendingEntries.poll();
|
||||
copyFileEntry(dataOut, entry);
|
||||
copyFileEntry(getOutput(), entry);
|
||||
entries.put(entry.file, entry);
|
||||
}
|
||||
} finally {
|
||||
|
@ -30,6 +30,8 @@ import java.nio.charset.Charset;
|
||||
import java.nio.charset.CharsetDecoder;
|
||||
import java.nio.charset.CodingErrorAction;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
|
||||
/** This class emulates the new Java 7 "Try-With-Resources" statement.
|
||||
* Remove once Lucene is on Java 7.
|
||||
* @lucene.internal */
|
||||
@ -318,6 +320,16 @@ public final class IOUtils {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public static void deleteFilesIgnoringExceptions(Directory dir, String... files) {
|
||||
for (String name : files) {
|
||||
try {
|
||||
dir.deleteFile(name);
|
||||
} catch (IOException ignored) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -0,0 +1,277 @@
|
||||
package org.apache.lucene.codecs.preflexrw;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.DocValues.Type;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.IndexFileNames;
|
||||
import org.apache.lucene.index.MergeState;
|
||||
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.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
|
||||
/**
|
||||
* Writes and Merges Lucene 3.x norms format
|
||||
* @lucene.experimental
|
||||
*/
|
||||
class PreFlexNormsConsumer extends PerDocConsumer {
|
||||
|
||||
/** norms header placeholder */
|
||||
private static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
|
||||
|
||||
/** Extension of norms file */
|
||||
private static final String NORMS_EXTENSION = "nrm";
|
||||
|
||||
/** Extension of separate norms file
|
||||
* @deprecated */
|
||||
@Deprecated
|
||||
private static final String SEPARATE_NORMS_EXTENSION = "s";
|
||||
|
||||
private final Directory directory;
|
||||
|
||||
private final String segment;
|
||||
|
||||
private final IOContext context;
|
||||
|
||||
private NormsWriter writer;
|
||||
|
||||
public PreFlexNormsConsumer(Directory directory, String segment, IOContext context){
|
||||
this.directory = directory;
|
||||
this.segment = segment;
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void merge(MergeState mergeState) throws IOException {
|
||||
getNormsWriter().merge(mergeState);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (writer != null) {
|
||||
writer.finish();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesConsumer addValuesField(Type type, FieldInfo fieldInfo)
|
||||
throws IOException {
|
||||
return new Lucene3xNormsDocValuesConsumer(fieldInfo);
|
||||
}
|
||||
|
||||
class Lucene3xNormsDocValuesConsumer extends DocValuesConsumer {
|
||||
// Holds all docID/norm pairs we've seen
|
||||
private int[] docIDs = new int[1];
|
||||
private byte[] norms = new byte[1];
|
||||
private int upto;
|
||||
private final FieldInfo fi;
|
||||
|
||||
Lucene3xNormsDocValuesConsumer(FieldInfo fieldInfo) {
|
||||
fi = fieldInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finish(int docCount) throws IOException {
|
||||
final NormsWriter normsWriter = getNormsWriter();
|
||||
boolean success = false;
|
||||
try {
|
||||
int uptoDoc = 0;
|
||||
normsWriter.setNumTotalDocs(docCount);
|
||||
if (upto > 0) {
|
||||
normsWriter.startField(fi);
|
||||
int docID = 0;
|
||||
for (; docID < docCount; docID++) {
|
||||
if (uptoDoc < upto && docIDs[uptoDoc] == docID) {
|
||||
normsWriter.writeNorm(norms[uptoDoc]);
|
||||
uptoDoc++;
|
||||
} else {
|
||||
normsWriter.writeNorm((byte) 0);
|
||||
}
|
||||
}
|
||||
// we should have consumed every norm
|
||||
assert uptoDoc == upto;
|
||||
|
||||
} else {
|
||||
// Fill entire field with default norm:
|
||||
normsWriter.startField(fi);
|
||||
for (; upto < docCount; upto++)
|
||||
normsWriter.writeNorm((byte) 0);
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
normsWriter.abort();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(int docID, DocValue docValue) throws IOException {
|
||||
add(docID, docValue.getBytes());
|
||||
}
|
||||
|
||||
protected void add(int docID, BytesRef value) throws IOException {
|
||||
if (docIDs.length <= upto) {
|
||||
assert docIDs.length == upto;
|
||||
docIDs = ArrayUtil.grow(docIDs, 1 + upto);
|
||||
}
|
||||
if (norms.length <= upto) {
|
||||
assert norms.length == upto;
|
||||
norms = ArrayUtil.grow(norms, 1 + upto);
|
||||
}
|
||||
assert value.length == 1;
|
||||
norms[upto] = value.bytes[value.offset];
|
||||
|
||||
docIDs[upto] = docID;
|
||||
upto++;
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
public NormsWriter getNormsWriter() throws IOException {
|
||||
if (writer == null) {
|
||||
writer = new NormsWriter(directory, segment, context);
|
||||
}
|
||||
return writer;
|
||||
}
|
||||
|
||||
private static class NormsWriter {
|
||||
|
||||
private final IndexOutput output;
|
||||
private int normCount = 0;
|
||||
private int numTotalDocs = 0;
|
||||
|
||||
public NormsWriter(Directory directory, String segment, IOContext context) throws IOException {
|
||||
final String normsFileName = IndexFileNames.segmentFileName(segment, "", NORMS_EXTENSION);
|
||||
boolean success = false;
|
||||
IndexOutput out = null;
|
||||
try {
|
||||
out = directory.createOutput(normsFileName, context);
|
||||
output = out;
|
||||
output.writeBytes(NORMS_HEADER, 0, NORMS_HEADER.length);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
IOUtils.closeWhileHandlingException(out);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
public void setNumTotalDocs(int numTotalDocs) {
|
||||
assert this.numTotalDocs == 0 || numTotalDocs == this.numTotalDocs;
|
||||
this.numTotalDocs = numTotalDocs;
|
||||
}
|
||||
|
||||
public void startField(FieldInfo info) throws IOException {
|
||||
assert info.omitNorms == false;
|
||||
normCount++;
|
||||
}
|
||||
|
||||
public void writeNorm(byte norm) throws IOException {
|
||||
output.writeByte(norm);
|
||||
}
|
||||
|
||||
public void abort() throws IOException {
|
||||
IOUtils.close(output);
|
||||
}
|
||||
|
||||
public void finish() throws IOException {
|
||||
IOUtils.close(output);
|
||||
|
||||
if (4+normCount*(long)numTotalDocs != output.getFilePointer()) {
|
||||
throw new IOException(".nrm file size mismatch: expected=" + (4+normCount*(long)numTotalDocs) + " actual=" + output.getFilePointer());
|
||||
}
|
||||
}
|
||||
// TODO: we can actually use the defaul DV merge here and drop this specific stuff entirely
|
||||
/** we override merge and bulk-merge norms when there are no deletions */
|
||||
public void merge(MergeState mergeState) throws IOException {
|
||||
int numMergedDocs = 0;
|
||||
for (FieldInfo fi : mergeState.fieldInfos) {
|
||||
if (fi.isIndexed && !fi.omitNorms) {
|
||||
startField(fi);
|
||||
int numMergedDocsForField = 0;
|
||||
for (MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
|
||||
final int maxDoc = reader.reader.maxDoc();
|
||||
byte[] normBuffer;
|
||||
DocValues normValues = reader.reader.normValues(fi.name);
|
||||
if (normValues == null) {
|
||||
// Can be null if this segment doesn't have
|
||||
// any docs with this field
|
||||
normBuffer = new byte[maxDoc];
|
||||
Arrays.fill(normBuffer, (byte)0);
|
||||
} else {
|
||||
Source directSource = normValues.getDirectSource();
|
||||
assert directSource.hasArray();
|
||||
normBuffer = (byte[]) directSource.getArray();
|
||||
}
|
||||
if (reader.liveDocs == null) {
|
||||
//optimized case for segments without deleted docs
|
||||
output.writeBytes(normBuffer, maxDoc);
|
||||
numMergedDocsForField += maxDoc;
|
||||
} else {
|
||||
// this segment has deleted docs, so we have to
|
||||
// check for every doc if it is deleted or not
|
||||
final Bits liveDocs = reader.liveDocs;
|
||||
for (int k = 0; k < maxDoc; k++) {
|
||||
if (liveDocs.get(k)) {
|
||||
numMergedDocsForField++;
|
||||
output.writeByte(normBuffer[k]);
|
||||
}
|
||||
}
|
||||
}
|
||||
mergeState.checkAbort.work(maxDoc);
|
||||
}
|
||||
assert numMergedDocs == 0 || numMergedDocs == numMergedDocsForField;
|
||||
numMergedDocs = numMergedDocsForField;
|
||||
}
|
||||
}
|
||||
this.numTotalDocs = numMergedDocs;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void abort() {
|
||||
try {
|
||||
try {
|
||||
if (writer != null) {
|
||||
writer.abort();
|
||||
}
|
||||
} finally {
|
||||
directory.deleteFile(IndexFileNames.segmentFileName(segment, "",
|
||||
NORMS_EXTENSION));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
@ -17,6 +17,7 @@ package org.apache.lucene.codecs.preflexrw;
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.NormsFormat;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
@ -27,7 +28,8 @@ import org.apache.lucene.util.LuceneTestCase;
|
||||
*/
|
||||
public class PreFlexRWCodec extends Lucene3xCodec {
|
||||
private final PostingsFormat postings = new PreFlexRWPostingsFormat();
|
||||
|
||||
private final NormsFormat norms = new PreFlexRWNormsFormat();
|
||||
|
||||
@Override
|
||||
public PostingsFormat postingsFormat() {
|
||||
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
|
||||
@ -36,4 +38,13 @@ public class PreFlexRWCodec extends Lucene3xCodec {
|
||||
return super.postingsFormat();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public NormsFormat normsFormat() {
|
||||
if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
|
||||
return norms;
|
||||
} else {
|
||||
return super.normsFormat();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1,5 +1,4 @@
|
||||
package org.apache.lucene.codecs;
|
||||
|
||||
package org.apache.lucene.codecs.preflexrw;
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
@ -16,11 +15,17 @@ package org.apache.lucene.codecs;
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
|
||||
//simple api just for now before switching to docvalues apis
|
||||
public abstract class NormsReader implements Closeable {
|
||||
public abstract byte[] norms(String name) throws IOException;
|
||||
import org.apache.lucene.codecs.PerDocConsumer;
|
||||
import org.apache.lucene.codecs.lucene3x.Lucene3xNormsFormat;
|
||||
import org.apache.lucene.index.PerDocWriteState;
|
||||
|
||||
public class PreFlexRWNormsFormat extends Lucene3xNormsFormat {
|
||||
|
||||
@Override
|
||||
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
|
||||
return new PreFlexNormsConsumer(state.directory, state.segmentName, state.context);
|
||||
}
|
||||
|
||||
}
|
@ -20,10 +20,11 @@ package org.apache.lucene.codecs.lucene40;
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesConsumer;
|
||||
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.codecs.lucene40.values.Writer;
|
||||
import org.apache.lucene.index.DocValue;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
@ -58,12 +59,13 @@ public class TestDocValues extends LuceneTestCase {
|
||||
|
||||
public void runTestBytes(final Bytes.Mode mode, final boolean fixedSize)
|
||||
throws IOException {
|
||||
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
valueHolder.comp = COMP;
|
||||
final BytesRef bytesRef = new BytesRef();
|
||||
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Bytes.getWriter(dir, "test", mode, fixedSize, COMP, trackBytes, newIOContext(random),
|
||||
DocValuesConsumer w = Bytes.getWriter(dir, "test", mode, fixedSize, COMP, trackBytes, newIOContext(random),
|
||||
random.nextBoolean());
|
||||
int maxDoc = 220;
|
||||
final String[] values = new String[maxDoc];
|
||||
@ -79,7 +81,8 @@ public class TestDocValues extends LuceneTestCase {
|
||||
values[2 * i] = s;
|
||||
|
||||
UnicodeUtil.UTF16toUTF8(s, 0, s.length(), bytesRef);
|
||||
w.add(2 * i, bytesRef);
|
||||
valueHolder.bytes = bytesRef;
|
||||
w.add(2 * i, valueHolder);
|
||||
}
|
||||
w.finish(maxDoc);
|
||||
assertEquals(0, trackBytes.get());
|
||||
@ -167,12 +170,15 @@ public class TestDocValues extends LuceneTestCase {
|
||||
Type.FIXED_INTS_64, Type.FIXED_INTS_64,
|
||||
Type.FIXED_INTS_64, Type.VAR_INTS, Type.VAR_INTS,
|
||||
Type.VAR_INTS, };
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
for (int i = 0; i < minMax.length; i++) {
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.VAR_INTS, newIOContext(random));
|
||||
w.add(0, minMax[i][0]);
|
||||
w.add(1, minMax[i][1]);
|
||||
DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.VAR_INTS, newIOContext(random));
|
||||
valueHolder.intValue = minMax[i][0];
|
||||
w.add(0, valueHolder);
|
||||
valueHolder.intValue = minMax[i][1];
|
||||
w.add(1, valueHolder);
|
||||
w.finish(2);
|
||||
assertEquals(0, trackBytes.get());
|
||||
DocValues r = Ints.getValues(dir, "test", 2, Type.VAR_INTS, newIOContext(random));
|
||||
@ -200,12 +206,14 @@ public class TestDocValues extends LuceneTestCase {
|
||||
}
|
||||
|
||||
public void testGetInt8Array() throws IOException {
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
byte[] sourceArray = new byte[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_8, newIOContext(random));
|
||||
DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_8, newIOContext(random));
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, (long) sourceArray[i]);
|
||||
valueHolder.intValue = (long) sourceArray[i];
|
||||
w.add(i, valueHolder);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
DocValues r = Ints.getValues(dir, "test", sourceArray.length, Type.FIXED_INTS_8, newIOContext(random));
|
||||
@ -221,12 +229,14 @@ public class TestDocValues extends LuceneTestCase {
|
||||
}
|
||||
|
||||
public void testGetInt16Array() throws IOException {
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
short[] sourceArray = new short[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_16, newIOContext(random));
|
||||
DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_16, newIOContext(random));
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, (long) sourceArray[i]);
|
||||
valueHolder.intValue = (long) sourceArray[i];
|
||||
w.add(i, valueHolder);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
DocValues r = Ints.getValues(dir, "test", sourceArray.length, Type.FIXED_INTS_16, newIOContext(random));
|
||||
@ -242,12 +252,14 @@ public class TestDocValues extends LuceneTestCase {
|
||||
}
|
||||
|
||||
public void testGetInt64Array() throws IOException {
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
long[] sourceArray = new long[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_64, newIOContext(random));
|
||||
DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_64, newIOContext(random));
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, sourceArray[i]);
|
||||
valueHolder.intValue = sourceArray[i];
|
||||
w.add(i, valueHolder);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
DocValues r = Ints.getValues(dir, "test", sourceArray.length, Type.FIXED_INTS_64, newIOContext(random));
|
||||
@ -263,12 +275,14 @@ public class TestDocValues extends LuceneTestCase {
|
||||
}
|
||||
|
||||
public void testGetInt32Array() throws IOException {
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
int[] sourceArray = new int[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_32, newIOContext(random));
|
||||
DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, Type.FIXED_INTS_32, newIOContext(random));
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, (long) sourceArray[i]);
|
||||
valueHolder.intValue = (long) sourceArray[i];
|
||||
w.add(i, valueHolder);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
DocValues r = Ints.getValues(dir, "test", sourceArray.length, Type.FIXED_INTS_32, newIOContext(random));
|
||||
@ -284,12 +298,14 @@ public class TestDocValues extends LuceneTestCase {
|
||||
}
|
||||
|
||||
public void testGetFloat32Array() throws IOException {
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
float[] sourceArray = new float[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), Type.FLOAT_32);
|
||||
DocValuesConsumer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), Type.FLOAT_32);
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, sourceArray[i]);
|
||||
valueHolder.floatValue = sourceArray[i];
|
||||
w.add(i, valueHolder);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
DocValues r = Floats.getValues(dir, "test", 3, newIOContext(random), Type.FLOAT_32);
|
||||
@ -305,12 +321,14 @@ public class TestDocValues extends LuceneTestCase {
|
||||
}
|
||||
|
||||
public void testGetFloat64Array() throws IOException {
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
double[] sourceArray = new double[] {1,2,3};
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), Type.FLOAT_64);
|
||||
DocValuesConsumer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), Type.FLOAT_64);
|
||||
for (int i = 0; i < sourceArray.length; i++) {
|
||||
w.add(i, sourceArray[i]);
|
||||
valueHolder.floatValue = sourceArray[i];
|
||||
w.add(i, valueHolder);
|
||||
}
|
||||
w.finish(sourceArray.length);
|
||||
DocValues r = Floats.getValues(dir, "test", 3, newIOContext(random), Type.FLOAT_64);
|
||||
@ -326,17 +344,18 @@ public class TestDocValues extends LuceneTestCase {
|
||||
}
|
||||
|
||||
private void testInts(Type type, int maxBit) throws IOException {
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
long maxV = 1;
|
||||
final int NUM_VALUES = 333 + random.nextInt(333);
|
||||
final long[] values = new long[NUM_VALUES];
|
||||
for (int rx = 1; rx < maxBit; rx++, maxV *= 2) {
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Ints.getWriter(dir, "test", trackBytes, type, newIOContext(random));
|
||||
DocValuesConsumer w = Ints.getWriter(dir, "test", trackBytes, type, newIOContext(random));
|
||||
for (int i = 0; i < NUM_VALUES; i++) {
|
||||
final long v = random.nextLong() % (1 + maxV);
|
||||
values[i] = v;
|
||||
w.add(i, v);
|
||||
valueHolder.intValue = values[i] = v;
|
||||
w.add(i, valueHolder);
|
||||
}
|
||||
final int additionalDocs = 1 + random.nextInt(9);
|
||||
w.finish(NUM_VALUES + additionalDocs);
|
||||
@ -362,16 +381,17 @@ public class TestDocValues extends LuceneTestCase {
|
||||
}
|
||||
|
||||
private void runTestFloats(Type type, double delta) throws IOException {
|
||||
DocValueHolder valueHolder = new DocValueHolder();
|
||||
Directory dir = newDirectory();
|
||||
final Counter trackBytes = Counter.newCounter();
|
||||
Writer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), type);
|
||||
DocValuesConsumer w = Floats.getWriter(dir, "test", trackBytes, newIOContext(random), type);
|
||||
final int NUM_VALUES = 777 + random.nextInt(777);;
|
||||
final double[] values = new double[NUM_VALUES];
|
||||
for (int i = 0; i < NUM_VALUES; i++) {
|
||||
final double v = type == Type.FLOAT_32 ? random.nextFloat() : random
|
||||
.nextDouble();
|
||||
values[i] = v;
|
||||
w.add(i, v);
|
||||
valueHolder.floatValue = values[i] = v;
|
||||
w.add(i, valueHolder);
|
||||
}
|
||||
final int additionalValues = 1 + random.nextInt(10);
|
||||
w.finish(NUM_VALUES + additionalValues);
|
||||
@ -411,4 +431,31 @@ public class TestDocValues extends LuceneTestCase {
|
||||
return getSource(values).asSortedSource();
|
||||
}
|
||||
|
||||
public static class DocValueHolder implements DocValue {
|
||||
BytesRef bytes;
|
||||
long intValue;
|
||||
double floatValue;
|
||||
Comparator<BytesRef> comp;
|
||||
@Override
|
||||
public BytesRef getBytes() {
|
||||
return bytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Comparator<BytesRef> bytesComparator() {
|
||||
return comp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double getFloat() {
|
||||
return floatValue;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getInt() {
|
||||
return intValue;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -26,7 +26,6 @@ import java.util.List;
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.DocValuesField;
|
||||
@ -417,6 +416,124 @@ public class TestDocValuesIndexing extends LuceneTestCase {
|
||||
w.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
public void testGetArrayNumerics() throws CorruptIndexException, IOException {
|
||||
Directory d = newDirectory();
|
||||
IndexWriterConfig cfg = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random));
|
||||
IndexWriter w = new IndexWriter(d, cfg);
|
||||
final int numValues = 50 + atLeast(10);
|
||||
final List<Type> numVariantList = new ArrayList<Type>(NUMERICS);
|
||||
Collections.shuffle(numVariantList, random);
|
||||
for (Type val : numVariantList) {
|
||||
indexValues(w, numValues, val, numVariantList,
|
||||
false, 7);
|
||||
IndexReader r = IndexReader.open(w, true);
|
||||
DocValues docValues = getDocValues(r, val.name());
|
||||
assertNotNull(docValues);
|
||||
// make sure we don't get a direct source since they don't support getArray()
|
||||
Source source = docValues.getSource();
|
||||
|
||||
switch (source.type()) {
|
||||
case FIXED_INTS_8:
|
||||
{
|
||||
assertTrue(source.hasArray());
|
||||
byte[] values = (byte[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
assertEquals((long)values[i], source.getInt(i));
|
||||
}
|
||||
}
|
||||
break;
|
||||
case FIXED_INTS_16:
|
||||
{
|
||||
assertTrue(source.hasArray());
|
||||
short[] values = (short[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
assertEquals((long)values[i], source.getInt(i));
|
||||
}
|
||||
}
|
||||
break;
|
||||
case FIXED_INTS_32:
|
||||
{
|
||||
assertTrue(source.hasArray());
|
||||
int[] values = (int[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
assertEquals((long)values[i], source.getInt(i));
|
||||
}
|
||||
}
|
||||
break;
|
||||
case FIXED_INTS_64:
|
||||
{
|
||||
assertTrue(source.hasArray());
|
||||
long[] values = (long[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
assertEquals(values[i], source.getInt(i));
|
||||
}
|
||||
}
|
||||
break;
|
||||
case VAR_INTS:
|
||||
assertFalse(source.hasArray());
|
||||
break;
|
||||
case FLOAT_32:
|
||||
{
|
||||
assertTrue(source.hasArray());
|
||||
float[] values = (float[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
assertEquals((double)values[i], source.getFloat(i), 0.0d);
|
||||
}
|
||||
}
|
||||
break;
|
||||
case FLOAT_64:
|
||||
{
|
||||
assertTrue(source.hasArray());
|
||||
double[] values = (double[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
assertEquals(values[i], source.getFloat(i), 0.0d);
|
||||
}
|
||||
}
|
||||
break;
|
||||
default:
|
||||
fail("unexpected value " + source.type());
|
||||
}
|
||||
r.close();
|
||||
}
|
||||
w.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
public void testGetArrayBytes() throws CorruptIndexException, IOException {
|
||||
Directory d = newDirectory();
|
||||
IndexWriterConfig cfg = newIndexWriterConfig(TEST_VERSION_CURRENT,
|
||||
new MockAnalyzer(random));
|
||||
IndexWriter w = new IndexWriter(d, cfg);
|
||||
final int numValues = 50 + atLeast(10);
|
||||
// only single byte fixed straight supports getArray()
|
||||
indexValues(w, numValues, Type.BYTES_FIXED_STRAIGHT, null, false, 1);
|
||||
IndexReader r = IndexReader.open(w, true);
|
||||
DocValues docValues = getDocValues(r, Type.BYTES_FIXED_STRAIGHT.name());
|
||||
assertNotNull(docValues);
|
||||
// make sure we don't get a direct source since they don't support
|
||||
// getArray()
|
||||
Source source = docValues.getSource();
|
||||
|
||||
switch (source.type()) {
|
||||
case BYTES_FIXED_STRAIGHT: {
|
||||
BytesRef ref = new BytesRef();
|
||||
assertTrue(source.hasArray());
|
||||
byte[] values = (byte[]) source.getArray();
|
||||
for (int i = 0; i < numValues; i++) {
|
||||
source.getBytes(i, ref);
|
||||
assertEquals(1, ref.length);
|
||||
assertEquals(values[i], ref.bytes[ref.offset]);
|
||||
}
|
||||
}
|
||||
break;
|
||||
default:
|
||||
fail("unexpected value " + source.type());
|
||||
}
|
||||
r.close();
|
||||
w.close();
|
||||
d.close();
|
||||
}
|
||||
|
||||
private DocValues getDocValues(IndexReader reader, String field) throws IOException {
|
||||
return MultiDocValues.getDocValues(reader, field);
|
||||
|
@ -30,13 +30,11 @@ import org.apache.lucene.index.RandomIndexWriter;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.index.IndexReader.AtomicReaderContext;
|
||||
import org.apache.lucene.search.similarities.DefaultSimilarityProvider;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.apache.lucene.search.similarities.SimilarityProvider;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
import org.apache.lucene.util.TermContext;
|
||||
|
||||
/**
|
||||
* Tests the use of indexdocvalues in scoring.
|
||||
|
Loading…
x
Reference in New Issue
Block a user