progress towards full cutover to 2.0 apis

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/lucene4547@1433646 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Robert Muir 2013-01-15 21:00:39 +00:00
parent 1e29841a26
commit 8ef9811938
139 changed files with 382 additions and 12432 deletions

View File

@ -25,8 +25,8 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.SimpleDVProducer;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
@ -75,10 +75,10 @@ class DiskDocValuesProducer extends SimpleDVProducer {
private void readFields(IndexInput meta, FieldInfos infos) throws IOException {
int fieldNumber = meta.readVInt();
while (fieldNumber != -1) {
DocValues.Type type = infos.fieldInfo(fieldNumber).getDocValuesType();
if (DocValues.isNumber(type) || DocValues.isFloat(type)) {
DocValuesType type = infos.fieldInfo(fieldNumber).getDocValuesType();
if (type == DocValuesType.NUMERIC) {
numerics.put(fieldNumber, readNumericEntry(meta));
} else if (DocValues.isBytes(type)) {
} else if (type == DocValuesType.BINARY) {
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
if (b.minLength != b.maxLength) {
@ -88,7 +88,7 @@ class DiskDocValuesProducer extends SimpleDVProducer {
// variable length byte[]: read addresses as a numeric dv field
numerics.put(fieldNumber, readNumericEntry(meta));
}
} else if (DocValues.isSortedBytes(type)) {
} else if (type == DocValuesType.SORTED) {
BinaryEntry b = readBinaryEntry(meta);
binaries.put(fieldNumber, b);
if (b.minLength != b.maxLength) {

View File

@ -1,91 +0,0 @@
package org.apache.lucene.codecs.sep;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.codecs.lucene40.values.Bytes;
import org.apache.lucene.codecs.lucene40.values.Floats;
import org.apache.lucene.codecs.lucene40.values.Ints;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
/**
* Implementation of PerDocProducer that uses separate files.
* @lucene.experimental
*/
public class SepDocValuesProducer extends PerDocProducerBase {
private final TreeMap<String, DocValues> docValues;
/**
* Creates a new {@link SepDocValuesProducer} instance and loads all
* {@link DocValues} instances for this segment and codec.
*/
public SepDocValuesProducer(SegmentReadState state) throws IOException {
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.getDocCount(), state.directory, state.context);
}
@Override
protected Map<String,DocValues> docValues() {
return docValues;
}
@Override
protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
IOUtils.close(closeables);
}
@Override
protected DocValues loadDocValues(int docCount, Directory dir, String id,
Type type, IOContext context) throws IOException {
switch (type) {
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
return Ints.getValues(dir, id, docCount, type, context);
case FLOAT_32:
return Floats.getValues(dir, id, docCount, context, type);
case FLOAT_64:
return Floats.getValues(dir, id, docCount, context, type);
case BYTES_FIXED_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
case BYTES_FIXED_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
case BYTES_FIXED_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
case BYTES_VAR_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
case BYTES_VAR_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
case BYTES_VAR_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
default:
throw new IllegalStateException("unrecognized index values mode " + type);
}
}
}

View File

@ -18,10 +18,8 @@ package org.apache.lucene.codecs.simpletext;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
@ -41,10 +39,6 @@ public final class SimpleTextCodec extends Codec {
private final SegmentInfoFormat segmentInfos = new SimpleTextSegmentInfoFormat();
private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
// TODO: need a plain-text impl
private final DocValuesFormat docValues = new SimpleTextDocValuesFormat();
// TODO: need a plain-text impl (using the above)
private final NormsFormat normsFormat = new SimpleTextNormsFormat();
private final SimpleNormsFormat simpleNormsFormat = new SimpleTextSimpleNormsFormat();
private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
@ -60,11 +54,6 @@ public final class SimpleTextCodec extends Codec {
return postings;
}
@Override
public DocValuesFormat docValuesFormat() {
return docValues;
}
@Override
public StoredFieldsFormat storedFieldsFormat() {
return storedFields;
@ -85,11 +74,6 @@ public final class SimpleTextCodec extends Codec {
return segmentInfos;
}
@Override
public NormsFormat normsFormat() {
return normsFormat;
}
@Override
public SimpleNormsFormat simpleNormsFormat() {
return simpleNormsFormat;

View File

@ -1,295 +0,0 @@
package org.apache.lucene.codecs.simpletext;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to You under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesArraySource;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.IOUtils;
/**
* Writes plain-text DocValues.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
*/
public class SimpleTextDocValuesConsumer extends DocValuesConsumer {
static final BytesRef ZERO_DOUBLE = new BytesRef(Double.toString(0d));
static final BytesRef ZERO_INT = new BytesRef(Integer.toString(0));
static final BytesRef HEADER = new BytesRef("SimpleTextDocValues");
static final BytesRef END = new BytesRef("END");
static final BytesRef VALUE_SIZE = new BytesRef("valuesize ");
static final BytesRef DOC = new BytesRef(" doc ");
static final BytesRef VALUE = new BytesRef(" value ");
protected BytesRef scratch = new BytesRef();
protected int maxDocId = -1;
protected final String segment;
protected final Directory dir;
protected final IOContext ctx;
protected final Type type;
protected final BytesRefHash hash;
private int[] ords;
private int valueSize = Integer.MIN_VALUE;
private BytesRef zeroBytes;
private final String segmentSuffix;
public SimpleTextDocValuesConsumer(String segment, Directory dir,
IOContext ctx, Type type, String segmentSuffix) {
this.ctx = ctx;
this.dir = dir;
this.segment = segment;
this.type = type;
hash = new BytesRefHash();
ords = new int[0];
this.segmentSuffix = segmentSuffix;
}
@Override
public void add(int docID, StorableField value) throws IOException {
assert docID >= 0;
final int ord, vSize;
switch (type) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
vSize = value.binaryValue().length;
ord = hash.add(value.binaryValue());
break;
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
vSize = -1;
ord = hash.add(value.binaryValue());
break;
case FIXED_INTS_16:
vSize = 2;
scratch.grow(2);
DocValuesArraySource.copyShort(scratch, value.numericValue().shortValue());
ord = hash.add(scratch);
break;
case FIXED_INTS_32:
vSize = 4;
scratch.grow(4);
DocValuesArraySource.copyInt(scratch, value.numericValue().intValue());
ord = hash.add(scratch);
break;
case FIXED_INTS_8:
vSize = 1;
scratch.grow(1);
scratch.bytes[scratch.offset] = value.numericValue().byteValue();
scratch.length = 1;
ord = hash.add(scratch);
break;
case FIXED_INTS_64:
vSize = 8;
scratch.grow(8);
DocValuesArraySource.copyLong(scratch, value.numericValue().longValue());
ord = hash.add(scratch);
break;
case VAR_INTS:
vSize = -1;
scratch.grow(8);
DocValuesArraySource.copyLong(scratch, value.numericValue().longValue());
ord = hash.add(scratch);
break;
case FLOAT_32:
vSize = 4;
scratch.grow(4);
DocValuesArraySource.copyInt(scratch,
Float.floatToRawIntBits(value.numericValue().floatValue()));
ord = hash.add(scratch);
break;
case FLOAT_64:
vSize = 8;
scratch.grow(8);
DocValuesArraySource.copyLong(scratch,
Double.doubleToRawLongBits(value.numericValue().doubleValue()));
ord = hash.add(scratch);
break;
default:
throw new RuntimeException("should not reach this line");
}
if (valueSize == Integer.MIN_VALUE) {
assert maxDocId == -1;
valueSize = vSize;
} else {
if (valueSize != vSize) {
throw new IllegalArgumentException("value size must be " + valueSize + " but was: " + vSize);
}
}
maxDocId = Math.max(docID, maxDocId);
ords = grow(ords, docID);
ords[docID] = (ord < 0 ? (-ord)-1 : ord) + 1;
}
protected BytesRef getHeader() {
return HEADER;
}
private int[] grow(int[] array, int upto) {
if (array.length <= upto) {
return ArrayUtil.grow(array, 1 + upto);
}
return array;
}
private void prepareFlush(int docCount) {
assert ords != null;
ords = grow(ords, docCount);
}
@Override
public void finish(int docCount) throws IOException {
final String fileName = IndexFileNames.segmentFileName(segment, "",
segmentSuffix);
IndexOutput output = dir.createOutput(fileName, ctx);
boolean success = false;
BytesRef spare = new BytesRef();
try {
SimpleTextUtil.write(output, getHeader());
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, VALUE_SIZE);
SimpleTextUtil.write(output, Integer.toString(this.valueSize), scratch);
SimpleTextUtil.writeNewline(output);
prepareFlush(docCount);
for (int i = 0; i < docCount; i++) {
SimpleTextUtil.write(output, DOC);
SimpleTextUtil.write(output, Integer.toString(i), scratch);
SimpleTextUtil.writeNewline(output);
SimpleTextUtil.write(output, VALUE);
writeDoc(output, i, spare);
SimpleTextUtil.writeNewline(output);
}
SimpleTextUtil.write(output, END);
SimpleTextUtil.writeNewline(output);
success = true;
} finally {
hash.close();
if (success) {
IOUtils.close(output);
} else {
IOUtils.closeWhileHandlingException(output);
dir.deleteFile(fileName);
}
}
}
protected void writeDoc(IndexOutput output, int docId, BytesRef spare) throws IOException {
int ord = ords[docId] - 1;
if (ord != -1) {
assert ord >= 0;
hash.get(ord, spare);
switch (type) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
SimpleTextUtil.write(output, spare);
break;
case FIXED_INTS_16:
SimpleTextUtil.write(output,
Short.toString(DocValuesArraySource.asShort(spare)), scratch);
break;
case FIXED_INTS_32:
SimpleTextUtil.write(output,
Integer.toString(DocValuesArraySource.asInt(spare)), scratch);
break;
case VAR_INTS:
case FIXED_INTS_64:
SimpleTextUtil.write(output,
Long.toString(DocValuesArraySource.asLong(spare)), scratch);
break;
case FIXED_INTS_8:
assert spare.length == 1 : spare.length;
SimpleTextUtil.write(output,
Integer.toString(spare.bytes[spare.offset]), scratch);
break;
case FLOAT_32:
float valueFloat = Float.intBitsToFloat(DocValuesArraySource.asInt(spare));
SimpleTextUtil.write(output, Float.toString(valueFloat), scratch);
break;
case FLOAT_64:
double valueDouble = Double.longBitsToDouble(DocValuesArraySource
.asLong(spare));
SimpleTextUtil.write(output, Double.toString(valueDouble), scratch);
break;
default:
throw new IllegalArgumentException("unsupported type: " + type);
}
} else {
switch (type) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
if(zeroBytes == null) {
assert valueSize > 0;
zeroBytes = new BytesRef(new byte[valueSize]);
}
SimpleTextUtil.write(output, zeroBytes);
break;
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
scratch.length = 0;
SimpleTextUtil.write(output, scratch);
break;
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
SimpleTextUtil.write(output, ZERO_INT);
break;
case FLOAT_32:
case FLOAT_64:
SimpleTextUtil.write(output, ZERO_DOUBLE);
break;
default:
throw new IllegalArgumentException("unsupported type: " + type);
}
}
}
@Override
protected Type getType() {
return type;
}
@Override
public int getValueSize() {
return valueSize;
}
}

View File

@ -1,51 +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.DocValuesFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.BytesRef;
/**
* Plain-text DocValues format.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
*/
public class SimpleTextDocValuesFormat extends DocValuesFormat {
private static final String DOC_VALUES_SEG_SUFFIX = "dv";
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new SimpleTextPerDocConsumer(state, DOC_VALUES_SEG_SUFFIX);
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new SimpleTextPerDocProducer(state, BytesRef.getUTF8SortedAsUnicodeComparator(), DOC_VALUES_SEG_SUFFIX);
}
static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId;
}
}

View File

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

View File

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

View File

@ -1,124 +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 java.util.Comparator;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.util.BytesRef;
/**
* plain-text norms format.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
*/
public class SimpleTextNormsFormat extends NormsFormat {
private static final String NORMS_SEG_SUFFIX = "len";
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new SimpleTextNormsPerDocConsumer(state);
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new SimpleTextNormsPerDocProducer(state,
BytesRef.getUTF8SortedAsUnicodeComparator());
}
/**
* Reads plain-text norms.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
*/
public static class SimpleTextNormsPerDocProducer extends
SimpleTextPerDocProducer {
public SimpleTextNormsPerDocProducer(SegmentReadState state,
Comparator<BytesRef> comp) throws IOException {
super(state, comp, NORMS_SEG_SUFFIX);
}
@Override
protected boolean canLoad(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
@Override
protected boolean anyDocValuesFields(FieldInfos infos) {
return infos.hasNorms();
}
}
/**
* Writes plain-text norms.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
*/
public static class SimpleTextNormsPerDocConsumer extends
SimpleTextPerDocConsumer {
public SimpleTextNormsPerDocConsumer(PerDocWriteState state) {
super(state, NORMS_SEG_SUFFIX);
}
@Override
protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
throws IOException {
return reader.normValues(info.name);
}
@Override
protected boolean canMerge(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
@Override
public void abort() {
// We don't have to remove files here: IndexFileDeleter
// will do so
}
}
}

View File

@ -1,61 +0,0 @@
package org.apache.lucene.codecs.simpletext;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to You under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.DocValues.Type;
/**
* @lucene.experimental
*/
class SimpleTextPerDocConsumer extends PerDocConsumer {
protected final PerDocWriteState state;
protected final String segmentSuffix;
public SimpleTextPerDocConsumer(PerDocWriteState state, String segmentSuffix) {
this.state = state;
this.segmentSuffix = segmentSuffix;
}
@Override
public void close() throws IOException {
}
@Override
public DocValuesConsumer addValuesField(Type type, FieldInfo field)
throws IOException {
return new SimpleTextDocValuesConsumer(SimpleTextDocValuesFormat.docValuesId(state.segmentInfo.name,
field.number), state.directory, state.context, type, segmentSuffix);
}
@Override
public void abort() {
// We don't have to remove files here: IndexFileDeleter
// will do so
}
static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId;
}
}

View File

@ -1,447 +0,0 @@
package org.apache.lucene.codecs.simpletext;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to You under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.DOC;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.END;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.HEADER;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.VALUE;
import static org.apache.lucene.codecs.simpletext.SimpleTextDocValuesConsumer.VALUE_SIZE;
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Comparator;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.codecs.DocValuesArraySource;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefHash;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.packed.PackedInts.Reader;
/**
* Reads plain-text DocValues.
* <p>
* <b><font color="red">FOR RECREATIONAL USE ONLY</font></B>
*
* @lucene.experimental
*/
public class SimpleTextPerDocProducer extends PerDocProducerBase {
protected final TreeMap<String, DocValues> docValues;
private Comparator<BytesRef> comp;
private final String segmentSuffix;
/**
* Creates a new {@link SimpleTextPerDocProducer} instance and loads all
* {@link DocValues} instances for this segment and codec.
*/
public SimpleTextPerDocProducer(SegmentReadState state,
Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
this.comp = comp;
this.segmentSuffix = segmentSuffix;
if (anyDocValuesFields(state.fieldInfos)) {
docValues = load(state.fieldInfos, state.segmentInfo.name,
state.segmentInfo.getDocCount(), state.directory, state.context);
} else {
docValues = new TreeMap<String, DocValues>();
}
}
@Override
protected Map<String, DocValues> docValues() {
return docValues;
}
@Override
protected DocValues loadDocValues(int docCount, Directory dir, String id,
DocValues.Type type, IOContext context) throws IOException {
return new SimpleTextDocValues(dir, context, type, id, docCount, comp, segmentSuffix);
}
@Override
protected void closeInternal(Collection<? extends Closeable> closeables)
throws IOException {
IOUtils.close(closeables);
}
private static class SimpleTextDocValues extends DocValues {
private int docCount;
@Override
public void close() throws IOException {
boolean success = false;
try {
super.close();
success = true;
} finally {
if (success) {
IOUtils.close(input);
} else {
IOUtils.closeWhileHandlingException(input);
}
}
}
private Type type;
private Comparator<BytesRef> comp;
private int valueSize;
private final IndexInput input;
public SimpleTextDocValues(Directory dir, IOContext ctx, Type type,
String id, int docCount, Comparator<BytesRef> comp, String segmentSuffix) throws IOException {
this.type = type;
this.docCount = docCount;
this.comp = comp;
final String fileName = IndexFileNames.segmentFileName(id, "", segmentSuffix);
boolean success = false;
IndexInput in = null;
try {
in = dir.openInput(fileName, ctx);
valueSize = readHeader(in);
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(in);
}
}
input = in;
}
@Override
protected Source loadSource() throws IOException {
boolean success = false;
IndexInput in = input.clone();
try {
Source source = null;
switch (type) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
source = read(in, new ValueReader(type, docCount, comp));
break;
case FIXED_INTS_16:
case FIXED_INTS_32:
case VAR_INTS:
case FIXED_INTS_64:
case FIXED_INTS_8:
case FLOAT_32:
case FLOAT_64:
source = read(in, new ValueReader(type, docCount, null));
break;
default:
throw new IllegalArgumentException("unknown type: " + type);
}
assert source != null;
success = true;
return source;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(in);
} else {
IOUtils.close(in);
}
}
}
private int readHeader(IndexInput in) throws IOException {
BytesRef scratch = new BytesRef();
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, HEADER);
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, VALUE_SIZE);
return Integer.parseInt(readString(scratch.offset + VALUE_SIZE.length,
scratch));
}
private Source read(IndexInput in, ValueReader reader) throws IOException {
BytesRef scratch = new BytesRef();
for (int i = 0; i < docCount; i++) {
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, DOC) : scratch.utf8ToString();
SimpleTextUtil.readLine(in, scratch);
assert StringHelper.startsWith(scratch, VALUE);
reader.fromString(i, scratch, scratch.offset + VALUE.length);
}
SimpleTextUtil.readLine(in, scratch);
assert scratch.equals(END);
return reader.getSource();
}
@Override
public Source getDirectSource() throws IOException {
return this.getSource(); // don't cache twice
}
@Override
protected Source loadDirectSource() throws IOException {
return this.getSource();
}
@Override
public int getValueSize() {
return valueSize;
}
@Override
public Type getType() {
return type;
}
}
public static String readString(int offset, BytesRef scratch) {
return new String(scratch.bytes, scratch.offset + offset, scratch.length
- offset, IOUtils.CHARSET_UTF_8);
}
private static final class ValueReader {
private final Type type;
private byte[] bytes;
private short[] shorts;
private int[] ints;
private long[] longs;
private float[] floats;
private double[] doubles;
private Source source;
private BytesRefHash hash;
private BytesRef scratch;
public ValueReader(Type type, int maxDocs, Comparator<BytesRef> comp) {
super();
this.type = type;
Source docValuesArray = null;
switch (type) {
case FIXED_INTS_16:
shorts = new short[maxDocs];
docValuesArray = DocValuesArraySource.forType(type)
.newFromArray(shorts);
break;
case FIXED_INTS_32:
ints = new int[maxDocs];
docValuesArray = DocValuesArraySource.forType(type).newFromArray(ints);
break;
case FIXED_INTS_64:
longs = new long[maxDocs];
docValuesArray = DocValuesArraySource.forType(type)
.newFromArray(longs);
break;
case VAR_INTS:
longs = new long[maxDocs];
docValuesArray = new VarIntsArraySource(type, longs);
break;
case FIXED_INTS_8:
bytes = new byte[maxDocs];
docValuesArray = DocValuesArraySource.forType(type).newFromArray(bytes);
break;
case FLOAT_32:
floats = new float[maxDocs];
docValuesArray = DocValuesArraySource.forType(type)
.newFromArray(floats);
break;
case FLOAT_64:
doubles = new double[maxDocs];
docValuesArray = DocValuesArraySource.forType(type).newFromArray(
doubles);
break;
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
assert comp != null;
hash = new BytesRefHash();
BytesSource bytesSource = new BytesSource(type, comp, maxDocs, hash);
ints = bytesSource.docIdToEntry;
source = bytesSource;
scratch = new BytesRef();
break;
}
if (docValuesArray != null) {
assert source == null;
this.source = docValuesArray;
}
}
public void fromString(int ord, BytesRef ref, int offset) {
switch (type) {
case FIXED_INTS_16:
assert shorts != null;
shorts[ord] = Short.parseShort(readString(offset, ref));
break;
case FIXED_INTS_32:
assert ints != null;
ints[ord] = Integer.parseInt(readString(offset, ref));
break;
case FIXED_INTS_64:
case VAR_INTS:
assert longs != null;
longs[ord] = Long.parseLong(readString(offset, ref));
break;
case FIXED_INTS_8:
assert bytes != null;
bytes[ord] = (byte) Integer.parseInt(readString(offset, ref));
break;
case FLOAT_32:
assert floats != null;
floats[ord] = Float.parseFloat(readString(offset, ref));
break;
case FLOAT_64:
assert doubles != null;
doubles[ord] = Double.parseDouble(readString(offset, ref));
break;
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
scratch.bytes = ref.bytes;
scratch.length = ref.length - offset;
scratch.offset = ref.offset + offset;
int key = hash.add(scratch);
ints[ord] = key < 0 ? (-key) - 1 : key;
break;
}
}
public Source getSource() {
if (source instanceof BytesSource) {
((BytesSource) source).maybeSort();
}
return source;
}
}
private static final class BytesSource extends SortedSource {
private final BytesRefHash hash;
int[] docIdToEntry;
int[] sortedEntries;
int[] adresses;
private final boolean isSorted;
protected BytesSource(Type type, Comparator<BytesRef> comp, int maxDoc,
BytesRefHash hash) {
super(type, comp);
docIdToEntry = new int[maxDoc];
this.hash = hash;
isSorted = type == Type.BYTES_FIXED_SORTED
|| type == Type.BYTES_VAR_SORTED;
}
void maybeSort() {
if (isSorted) {
adresses = new int[hash.size()];
sortedEntries = hash.sort(getComparator());
for (int i = 0; i < adresses.length; i++) {
int entry = sortedEntries[i];
adresses[entry] = i;
}
}
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
if (isSorted) {
return hash.get(sortedEntries[ord(docID)], ref);
} else {
return hash.get(docIdToEntry[docID], ref);
}
}
@Override
public SortedSource asSortedSource() {
if (isSorted) {
return this;
}
return null;
}
@Override
public int ord(int docID) {
assert isSorted;
try {
return adresses[docIdToEntry[docID]];
} catch (Exception e) {
return 0;
}
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
assert isSorted;
return hash.get(sortedEntries[ord], bytesRef);
}
@Override
public Reader getDocToOrd() {
return null;
}
@Override
public int getValueCount() {
return hash.size();
}
}
private static class VarIntsArraySource extends Source {
private final long[] array;
protected VarIntsArraySource(Type type, long[] array) {
super(type);
this.array = array;
}
@Override
public long getInt(int docID) {
return array[docID];
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
DocValuesArraySource.copyLong(ref, getInt(docID));
return ref;
}
}
}

View File

@ -34,8 +34,8 @@ import org.apache.lucene.codecs.SimpleDVProducer;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.SegmentReadState;
@ -161,8 +161,9 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
@Override
public void addNumericField(FieldInfo field, Iterable<Number> values) throws IOException {
assert fieldSeen(field.name);
assert (field.getDocValuesType() != null && (DocValues.isNumber(field.getDocValuesType()) || DocValues.isFloat(field.getDocValuesType()))) ||
(field.getNormType() != null && (DocValues.isNumber(field.getNormType()) || DocValues.isFloat(field.getNormType()))): "field=" + field.name;
// nocommit: this must be multiple asserts
//assert (field.getDocValuesType() != null && (DocValues.isNumber(field.getDocValuesType()) || DocValues.isFloat(field.getDocValuesType()))) ||
// (field.getNormType() != null && (DocValues.isNumber(field.getNormType()) || DocValues.isFloat(field.getNormType()))): "field=" + field.name;
writeFieldEntry(field);
// first pass to find min/max
@ -220,7 +221,7 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
@Override
public void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
assert fieldSeen(field.name);
assert DocValues.isBytes(field.getDocValuesType());
assert field.getDocValuesType() == DocValuesType.BINARY;
assert !isNorms;
int maxLength = 0;
for(BytesRef value : values) {
@ -269,7 +270,7 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
@Override
public void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> docToOrd) throws IOException {
assert fieldSeen(field.name);
assert DocValues.isSortedBytes(field.getDocValuesType());
assert field.getDocValuesType() == DocValuesType.SORTED;
assert !isNorms;
writeFieldEntry(field);
@ -421,9 +422,9 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
//System.out.println(" field=" + fieldName);
// nocommit hack hack hack!!:
DocValues.Type dvType = ext.equals("slen") ? DocValues.Type.FIXED_INTS_8 : fieldInfo.getDocValuesType();
DocValuesType dvType = ext.equals("slen") ? DocValuesType.NUMERIC : fieldInfo.getDocValuesType();
assert dvType != null;
if (DocValues.isNumber(dvType) || DocValues.isFloat(dvType)) {
if (dvType == DocValuesType.NUMERIC) {
readLine();
assert startsWith(MINVALUE): "got " + scratch.utf8ToString() + " field=" + fieldName + " ext=" + ext;
field.minValue = Long.parseLong(stripPrefix(MINVALUE));
@ -432,7 +433,7 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
field.pattern = stripPrefix(PATTERN);
field.dataStartFilePointer = data.getFilePointer();
data.seek(data.getFilePointer() + (1+field.pattern.length()) * maxDoc);
} else if (DocValues.isBytes(dvType)) {
} else if (dvType == DocValuesType.BINARY) {
readLine();
assert startsWith(MAXLENGTH);
field.maxLength = Integer.parseInt(stripPrefix(MAXLENGTH));
@ -441,7 +442,7 @@ public class SimpleTextSimpleDocValuesFormat extends SimpleDocValuesFormat {
field.pattern = stripPrefix(PATTERN);
field.dataStartFilePointer = data.getFilePointer();
data.seek(data.getFilePointer() + (9+field.pattern.length()+field.maxLength) * maxDoc);
} else if (DocValues.isSortedBytes(dvType)) {
} else if (dvType == DocValuesType.SORTED) {
readLine();
assert startsWith(NUMVALUES);
field.numValues = Integer.parseInt(stripPrefix(NUMVALUES));

View File

@ -64,11 +64,8 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
/** Encodes/decodes postings */
public abstract PostingsFormat postingsFormat();
/** Encodes/decodes docvalues */
public abstract DocValuesFormat docValuesFormat();
/** Encodes/decodes streaming docvalues */
/** Encodes/decodes docvalues */
public abstract SimpleDocValuesFormat simpleDocValuesFormat();
/** Encodes/decodes stored fields */
@ -84,12 +81,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
public abstract SegmentInfoFormat segmentInfoFormat();
/** Encodes/decodes document normalization values */
public abstract NormsFormat normsFormat();
public SimpleNormsFormat simpleNormsFormat() {
// nocommit make this abstract
return null;
}
public abstract SimpleNormsFormat simpleNormsFormat();
/** Encodes/decodes live docs */
public abstract LiveDocsFormat liveDocsFormat();

View File

@ -1,545 +0,0 @@
package org.apache.lucene.codecs;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to You under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
import java.io.IOException;
import java.util.Collections;
import java.util.EnumMap;
import java.util.Map;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
/**
* DocValues {@link Source} implementation backed by
* simple arrays.
*
* @lucene.experimental
* @lucene.internal
*/
public abstract class DocValuesArraySource extends Source {
private static final Map<Type, DocValuesArraySource> TEMPLATES;
static {
EnumMap<Type, DocValuesArraySource> templates = new EnumMap<Type, DocValuesArraySource>(
Type.class);
templates.put(Type.FIXED_INTS_16, new ShortValues());
templates.put(Type.FIXED_INTS_32, new IntValues());
templates.put(Type.FIXED_INTS_64, new LongValues());
templates.put(Type.FIXED_INTS_8, new ByteValues());
templates.put(Type.FLOAT_32, new FloatValues());
templates.put(Type.FLOAT_64, new DoubleValues());
TEMPLATES = Collections.unmodifiableMap(templates);
}
/** Returns the {@link DocValuesArraySource} for the given
* {@link Type}. */
public static DocValuesArraySource forType(Type type) {
return TEMPLATES.get(type);
}
/** Number of bytes to encode each doc value. */
protected final int bytesPerValue;
DocValuesArraySource(int bytesPerValue, Type type) {
super(type);
this.bytesPerValue = bytesPerValue;
}
@Override
public abstract BytesRef getBytes(int docID, BytesRef ref);
/** Creates a {@link DocValuesArraySource} by loading a
* previously saved one from an {@link IndexInput}. */
public abstract DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException;
/** Creates {@link DocValuesArraySource} from a native
* array. */
public abstract DocValuesArraySource newFromArray(Object array);
@Override
public final boolean hasArray() {
return true;
}
/** Encode a long value into the provided {@link
* BytesRef}. */
public void toBytes(long value, BytesRef bytesRef) {
copyLong(bytesRef, value);
}
/** Encode a double value into the provided {@link
* BytesRef}. */
public void toBytes(double value, BytesRef bytesRef) {
copyLong(bytesRef, Double.doubleToRawLongBits(value));
}
final static class ByteValues extends DocValuesArraySource {
private final byte[] values;
ByteValues() {
super(1, Type.FIXED_INTS_8);
values = new byte[0];
}
private ByteValues(byte[] array) {
super(1, Type.FIXED_INTS_8);
values = array;
}
private ByteValues(IndexInput input, int numDocs) throws IOException {
super(1, Type.FIXED_INTS_8);
values = new byte[numDocs];
input.readBytes(values, 0, values.length, false);
}
@Override
public byte[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
return getInt(docID);
}
@Override
public long getInt(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new ByteValues(input, numDocs);
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof byte[];
return new ByteValues((byte[]) array);
}
@Override
public void toBytes(long value, BytesRef bytesRef) {
if (bytesRef.bytes.length == 0) {
bytesRef.bytes = new byte[1];
}
bytesRef.bytes[0] = (byte) (0xFFL & value);
bytesRef.offset = 0;
bytesRef.length = 1;
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getInt(docID), ref);
return ref;
}
};
final static class ShortValues extends DocValuesArraySource {
private final short[] values;
ShortValues() {
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
values = new short[0];
}
private ShortValues(short[] array) {
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
values = array;
}
private ShortValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_SHORT, Type.FIXED_INTS_16);
values = new short[numDocs];
for (int i = 0; i < values.length; i++) {
values[i] = input.readShort();
}
}
@Override
public short[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
return getInt(docID);
}
@Override
public long getInt(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new ShortValues(input, numDocs);
}
@Override
public void toBytes(long value, BytesRef bytesRef) {
copyShort(bytesRef, (short) (0xFFFFL & value));
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof short[];
return new ShortValues((short[]) array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getInt(docID), ref);
return ref;
}
};
final static class IntValues extends DocValuesArraySource {
private final int[] values;
IntValues() {
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
values = new int[0];
}
private IntValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
values = new int[numDocs];
for (int i = 0; i < values.length; i++) {
values[i] = input.readInt();
}
}
private IntValues(int[] array) {
super(RamUsageEstimator.NUM_BYTES_INT, Type.FIXED_INTS_32);
values = array;
}
@Override
public int[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
return getInt(docID);
}
@Override
public long getInt(int docID) {
assert docID >= 0 && docID < values.length;
return 0xFFFFFFFF & values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new IntValues(input, numDocs);
}
@Override
public void toBytes(long value, BytesRef bytesRef) {
copyInt(bytesRef, (int) (0xFFFFFFFF & value));
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof int[];
return new IntValues((int[]) array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getInt(docID), ref);
return ref;
}
};
final static class LongValues extends DocValuesArraySource {
private final long[] values;
LongValues() {
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
values = new long[0];
}
private LongValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
values = new long[numDocs];
for (int i = 0; i < values.length; i++) {
values[i] = input.readLong();
}
}
private LongValues(long[] array) {
super(RamUsageEstimator.NUM_BYTES_LONG, Type.FIXED_INTS_64);
values = array;
}
@Override
public long[] getArray() {
return values;
}
@Override
public long getInt(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new LongValues(input, numDocs);
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof long[];
return new LongValues((long[])array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getInt(docID), ref);
return ref;
}
};
final static class FloatValues extends DocValuesArraySource {
private final float[] values;
FloatValues() {
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
values = new float[0];
}
private FloatValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
values = new float[numDocs];
/*
* we always read BIG_ENDIAN here since the writer serialized plain bytes
* we can simply read the ints / longs back in using readInt / readLong
*/
for (int i = 0; i < values.length; i++) {
values[i] = Float.intBitsToFloat(input.readInt());
}
}
private FloatValues(float[] array) {
super(RamUsageEstimator.NUM_BYTES_FLOAT, Type.FLOAT_32);
values = array;
}
@Override
public float[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public void toBytes(double value, BytesRef bytesRef) {
copyInt(bytesRef, Float.floatToRawIntBits((float)value));
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new FloatValues(input, numDocs);
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof float[];
return new FloatValues((float[]) array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getFloat(docID), ref);
return ref;
}
};
final static class DoubleValues extends DocValuesArraySource {
private final double[] values;
DoubleValues() {
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
values = new double[0];
}
private DoubleValues(IndexInput input, int numDocs) throws IOException {
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
values = new double[numDocs];
/*
* we always read BIG_ENDIAN here since the writer serialized plain bytes
* we can simply read the ints / longs back in using readInt / readLong
*/
for (int i = 0; i < values.length; i++) {
values[i] = Double.longBitsToDouble(input.readLong());
}
}
private DoubleValues(double[] array) {
super(RamUsageEstimator.NUM_BYTES_DOUBLE, Type.FLOAT_64);
values = array;
}
@Override
public double[] getArray() {
return values;
}
@Override
public double getFloat(int docID) {
assert docID >= 0 && docID < values.length;
return values[docID];
}
@Override
public DocValuesArraySource newFromInput(IndexInput input, int numDocs)
throws IOException {
return new DoubleValues(input, numDocs);
}
@Override
public DocValuesArraySource newFromArray(Object array) {
assert array instanceof double[];
return new DoubleValues((double[]) array);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
toBytes(getFloat(docID), ref);
return ref;
}
};
/**
* Copies the given long value and encodes it as 8 byte Big-Endian.
* <p>
* NOTE: this method resets the offset to 0, length to 8 and resizes the
* reference array if needed.
*/
public static void copyLong(BytesRef ref, long value) {
if (ref.bytes.length < 8) {
ref.bytes = new byte[8];
}
copyInternal(ref, (int) (value >> 32), ref.offset = 0);
copyInternal(ref, (int) value, 4);
ref.length = 8;
}
/**
* Copies the given int value and encodes it as 4 byte Big-Endian.
* <p>
* NOTE: this method resets the offset to 0, length to 4 and resizes the
* reference array if needed.
*/
public static void copyInt(BytesRef ref, int value) {
if (ref.bytes.length < 4) {
ref.bytes = new byte[4];
}
copyInternal(ref, value, ref.offset = 0);
ref.length = 4;
}
/**
* Copies the given short value and encodes it as a 2 byte Big-Endian.
* <p>
* NOTE: this method resets the offset to 0, length to 2 and resizes the
* reference array if needed.
*/
public static void copyShort(BytesRef ref, short value) {
if (ref.bytes.length < 2) {
ref.bytes = new byte[2];
}
ref.offset = 0;
ref.bytes[ref.offset] = (byte) (value >> 8);
ref.bytes[ref.offset + 1] = (byte) (value);
ref.length = 2;
}
private static void copyInternal(BytesRef ref, int value, int startOffset) {
ref.bytes[startOffset] = (byte) (value >> 24);
ref.bytes[startOffset + 1] = (byte) (value >> 16);
ref.bytes[startOffset + 2] = (byte) (value >> 8);
ref.bytes[startOffset + 3] = (byte) (value);
}
/**
* Converts 2 consecutive bytes from the current offset to a short. Bytes are
* interpreted as Big-Endian (most significant bit first)
* <p>
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
*/
public static short asShort(BytesRef b) {
return (short) (0xFFFF & ((b.bytes[b.offset] & 0xFF) << 8) | (b.bytes[b.offset + 1] & 0xFF));
}
/**
* Converts 4 consecutive bytes from the current offset to an int. Bytes are
* interpreted as Big-Endian (most significant bit first)
* <p>
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
*/
public static int asInt(BytesRef b) {
return asIntInternal(b, b.offset);
}
/**
* Converts 8 consecutive bytes from the current offset to a long. Bytes are
* interpreted as Big-Endian (most significant bit first)
* <p>
* NOTE: this method does <b>NOT</b> check the bounds of the referenced array.
*/
public static long asLong(BytesRef b) {
return (((long) asIntInternal(b, b.offset) << 32) | asIntInternal(b,
b.offset + 4) & 0xFFFFFFFFL);
}
private static int asIntInternal(BytesRef b, int pos) {
return ((b.bytes[pos++] & 0xFF) << 24) | ((b.bytes[pos++] & 0xFF) << 16)
| ((b.bytes[pos++] & 0xFF) << 8) | (b.bytes[pos] & 0xFF);
}
}

View File

@ -1,250 +0,0 @@
package org.apache.lucene.codecs;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.document.ByteDocValuesField;
import org.apache.lucene.document.DerefBytesDocValuesField;
import org.apache.lucene.document.DoubleDocValuesField;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FloatDocValuesField;
import org.apache.lucene.document.IntDocValuesField;
import org.apache.lucene.document.LongDocValuesField;
import org.apache.lucene.document.PackedLongDocValuesField;
import org.apache.lucene.document.ShortDocValuesField;
import org.apache.lucene.document.SortedBytesDocValuesField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StraightBytesDocValuesField;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
/**
* Abstract API that consumes {@link StorableField}s.
* {@link DocValuesConsumer} are always associated with a specific field and
* segments. Concrete implementations of this API write the given
* {@link StorableField} into a implementation specific format depending on
* the fields meta-data.
*
* @lucene.experimental
*/
public abstract class DocValuesConsumer {
/** Spare {@link BytesRef} that subclasses can reuse. */
protected final BytesRef spare = new BytesRef();
/** Returns the {@link Type} of this consumer. */
protected abstract Type getType();
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected DocValuesConsumer() {
}
/**
* Adds the given {@link StorableField} instance to this
* {@link DocValuesConsumer}
*
* @param docID
* the document ID to add the value for. The docID must always
* increase or be <tt>0</tt> if it is the first call to this method.
* @param value
* the value to add
* @throws IOException
* if an {@link IOException} occurs
*/
public abstract void add(int docID, StorableField value)
throws IOException;
/**
* Called when the consumer of this API is done adding values.
*
* @param docCount
* the total number of documents in this {@link DocValuesConsumer}.
* Must be greater than or equal the last given docID to
* {@link #add(int, StorableField)}.
* @throws IOException If an I/O error occurs
*/
public abstract void finish(int docCount) throws IOException;
/**
* Returns the value size this consumer accepts or <tt>-1</tt> iff this
* consumer is value size agnostic ie. accepts variable length values.
* <p>
* NOTE: the return value is undefined until the consumer has successfully
* consumed at least one value.
*
* @return the value size this consumer accepts or <tt>-1</tt> iff this
* consumer is value size agnostic ie. accepts variable length values.
*/
public abstract int getValueSize();
/**
* Merges the given {@link org.apache.lucene.index.MergeState} into
* this {@link DocValuesConsumer}.
*
* @param mergeState
* the state to merge
* @param docValues docValues array containing one instance per reader (
* {@link org.apache.lucene.index.MergeState#readers}) or <code>null</code> if the reader has
* no {@link DocValues} instance.
* @throws IOException
* if an {@link IOException} occurs
*/
public void merge(MergeState mergeState, DocValues[] docValues) throws IOException {
assert mergeState != null;
boolean hasMerged = false;
for(int readerIDX=0;readerIDX<mergeState.readers.size();readerIDX++) {
final AtomicReader reader = mergeState.readers.get(readerIDX);
if (docValues[readerIDX] != null) {
hasMerged = true;
merge(docValues[readerIDX], mergeState.docBase[readerIDX],
reader.maxDoc(), reader.getLiveDocs());
mergeState.checkAbort.work(reader.maxDoc());
}
}
// only finish if no exception is thrown!
if (hasMerged) {
finish(mergeState.segmentInfo.getDocCount());
}
}
/**
* Merges the given {@link DocValues} into this {@link DocValuesConsumer}.
*
* @throws IOException
* if an {@link IOException} occurs
*/
protected void merge(DocValues reader, int docBase, int docCount, Bits liveDocs) throws IOException {
// This enables bulk copies in subclasses per MergeState, subclasses can
// simply override this and decide if they want to merge
// segments using this generic implementation or if a bulk merge is possible
// / feasible.
final Source source = reader.getDirectSource();
assert source != null;
int docID = docBase;
final Type type = getType();
final StoredField scratchField;
switch(type) {
case VAR_INTS:
scratchField = new PackedLongDocValuesField("", (long) 0);
break;
case FIXED_INTS_8:
scratchField = new ByteDocValuesField("", (byte) 0);
break;
case FIXED_INTS_16:
scratchField = new ShortDocValuesField("", (short) 0);
break;
case FIXED_INTS_32:
scratchField = new IntDocValuesField("", 0);
break;
case FIXED_INTS_64:
scratchField = new LongDocValuesField("", (long) 0);
break;
case FLOAT_32:
scratchField = new FloatDocValuesField("", 0f);
break;
case FLOAT_64:
scratchField = new DoubleDocValuesField("", 0d);
break;
case BYTES_FIXED_STRAIGHT:
scratchField = new StraightBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_STRAIGHT:
scratchField = new StraightBytesDocValuesField("", new BytesRef(), false);
break;
case BYTES_FIXED_DEREF:
scratchField = new DerefBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_DEREF:
scratchField = new DerefBytesDocValuesField("", new BytesRef(), false);
break;
case BYTES_FIXED_SORTED:
scratchField = new SortedBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_SORTED:
scratchField = new SortedBytesDocValuesField("", new BytesRef(), false);
break;
default:
throw new IllegalStateException("unknown Type: " + type);
}
for (int i = 0; i < docCount; i++) {
if (liveDocs == null || liveDocs.get(i)) {
mergeDoc(scratchField, source, docID++, i);
}
}
}
/**
* 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}.
* <p>
* This method is used during merging to provide implementation agnostic
* default merge implementation.
* </p>
* <p>
* All documents IDs between the given ID and the previously given ID or
* <tt>0</tt> if the method is call the first time are filled with default
* values depending on the implementation. The given document
* ID must always be greater than the previous ID or <tt>0</tt> if called the
* first time.
*/
protected void mergeDoc(StoredField scratchField, Source source, int docID, int sourceDoc)
throws IOException {
switch(getType()) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
scratchField.setBytesValue(source.getBytes(sourceDoc, spare));
break;
case FIXED_INTS_8:
scratchField.setByteValue((byte) source.getInt(sourceDoc));
break;
case FIXED_INTS_16:
scratchField.setShortValue((short) source.getInt(sourceDoc));
break;
case FIXED_INTS_32:
scratchField.setIntValue((int) source.getInt(sourceDoc));
break;
case FIXED_INTS_64:
scratchField.setLongValue(source.getInt(sourceDoc));
break;
case VAR_INTS:
scratchField.setLongValue(source.getInt(sourceDoc));
break;
case FLOAT_32:
scratchField.setFloatValue((float) source.getFloat(sourceDoc));
break;
case FLOAT_64:
scratchField.setDoubleValue(source.getFloat(sourceDoc));
break;
}
add(docID, scratchField);
}
}

View File

@ -1,41 +0,0 @@
package org.apache.lucene.codecs;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.index.DocValues; // javadocs
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
/**
* Encodes/decodes {@link DocValues}
* @lucene.experimental
*/
public abstract class DocValuesFormat {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected DocValuesFormat() {
}
/** Consumes (writes) doc values during indexing. */
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
/** Produces (reads) doc values during reading/searching. */
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
}

View File

@ -59,11 +59,6 @@ public abstract class FilterCodec extends Codec {
this.delegate = delegate;
}
@Override
public DocValuesFormat docValuesFormat() {
return delegate.docValuesFormat();
}
@Override
public SimpleDocValuesFormat simpleDocValuesFormat() {
return delegate.simpleDocValuesFormat();
@ -79,11 +74,6 @@ public abstract class FilterCodec extends Codec {
return delegate.liveDocsFormat();
}
@Override
public NormsFormat normsFormat() {
return delegate.normsFormat();
}
@Override
public SimpleNormsFormat simpleNormsFormat() {
return delegate.simpleNormsFormat();

View File

@ -1,41 +0,0 @@
package org.apache.lucene.codecs;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
/**
* format for normalization factors
*/
public abstract class NormsFormat {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected NormsFormat() {
}
/** Returns a {@link PerDocConsumer} to write norms to the
* index. */
public abstract PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException;
/** Returns a {@link PerDocProducer} to read norms from the
* index. */
public abstract PerDocProducer docsProducer(SegmentReadState state) throws IOException;
}

View File

@ -1,115 +0,0 @@
package org.apache.lucene.codecs;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to You under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.DocValues.Type;
/**
* Abstract API that consumes per document values. Concrete implementations of
* this convert field values into a Codec specific format during indexing.
* <p>
* The {@link PerDocConsumer} API is accessible through the
* {@link PostingsFormat} - API providing per field consumers and producers for inverted
* data (terms, postings) as well as per-document data.
*
* @lucene.experimental
*/
public abstract class PerDocConsumer implements Closeable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected PerDocConsumer() {
}
/** Adds a new DocValuesField */
public abstract DocValuesConsumer addValuesField(DocValues.Type type, FieldInfo field)
throws IOException;
/**
* Consumes and merges the given {@link PerDocProducer} producer
* into this consumers format.
*/
public void merge(MergeState mergeState) throws IOException {
final DocValues[] docValues = new DocValues[mergeState.readers.size()];
for (FieldInfo fieldInfo : mergeState.fieldInfos) {
mergeState.fieldInfo = fieldInfo; // set the field we are merging
if (canMerge(fieldInfo)) {
for (int i = 0; i < docValues.length; i++) {
docValues[i] = getDocValuesForMerge(mergeState.readers.get(i), fieldInfo);
}
Type docValuesType = getDocValuesType(fieldInfo);
assert docValuesType != null;
final DocValuesConsumer docValuesConsumer = addValuesField(docValuesType, fieldInfo);
assert docValuesConsumer != null;
docValuesConsumer.merge(mergeState, docValues);
}
}
}
/**
* Returns a {@link DocValues} instance for merging from the given reader for the given
* {@link FieldInfo}. This method is used for merging and uses
* {@link AtomicReader#docValues(String)} by default.
* <p>
* To enable {@link DocValues} merging for different {@link DocValues} than
* the default override this method accordingly.
* <p>
*/
protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info) throws IOException {
return reader.docValues(info.name);
}
/**
* Returns <code>true</code> iff the given field can be merged ie. has {@link DocValues}.
* By default this method uses {@link FieldInfo#hasDocValues()}.
* <p>
* To enable {@link DocValues} merging for different {@link DocValues} than
* the default override this method accordingly.
* <p>
*/
protected boolean canMerge(FieldInfo info) {
return info.hasDocValues();
}
/**
* Returns the {@link DocValues} {@link Type} for the given {@link FieldInfo}.
* By default this method uses {@link FieldInfo#getDocValuesType()}.
* <p>
* To enable {@link DocValues} merging for different {@link DocValues} than
* the default override this method accordingly.
* <p>
*/
protected Type getDocValuesType(FieldInfo info) {
return info.getDocValuesType();
}
/**
* Called during indexing if the indexing session is aborted due to a unrecoverable exception.
* This method should cleanup all resources.
*/
public abstract void abort();
@Override
public abstract void close() throws IOException;
}

View File

@ -1,54 +0,0 @@
package org.apache.lucene.codecs;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.index.DocValues;
/**
* Abstract API that provides access to one or more per-document storage
* features. The concrete implementations provide access to the underlying
* storage on a per-document basis corresponding to their actual
* {@link PerDocConsumer} counterpart.
* <p>
* The {@link PerDocProducer} API is accessible through the
* {@link PostingsFormat} - API providing per field consumers and producers for inverted
* data (terms, postings) as well as per-document data.
*
* @lucene.experimental
*/
public abstract class PerDocProducer implements Closeable {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected PerDocProducer() {
}
/**
* Returns {@link DocValues} for the current field.
*
* @param field
* the field name
* @return the {@link DocValues} for this field or <code>null</code> if not
* applicable.
* @throws IOException If an I/O error occurs
*/
public abstract DocValues docValues(String field) throws IOException;
@Override
public abstract void close() throws IOException;
}

View File

@ -1,138 +0,0 @@
package org.apache.lucene.codecs;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.Closeable;
import java.io.IOException;
import java.util.Collection;
import java.util.Comparator;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type; // javadocs
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
/**
* Abstract base class for PerDocProducer implementations
* @lucene.experimental
*/
public abstract class PerDocProducerBase extends PerDocProducer {
/** Closes provided Closables. */
protected abstract void closeInternal(Collection<? extends Closeable> closeables) throws IOException;
/** Returns a map, mapping field names to doc values. */
protected abstract Map<String, DocValues> docValues();
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected PerDocProducerBase() {
}
@Override
public void close() throws IOException {
closeInternal(docValues().values());
}
@Override
public DocValues docValues(String field) throws IOException {
return docValues().get(field);
}
/** Returns the comparator used to sort {@link BytesRef} values. */
public Comparator<BytesRef> getComparator() throws IOException {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
/** Only opens files... doesn't actually load any values. */
protected TreeMap<String, DocValues> load(FieldInfos fieldInfos,
String segment, int docCount, Directory dir, IOContext context)
throws IOException {
TreeMap<String, DocValues> values = new TreeMap<String, DocValues>();
boolean success = false;
try {
for (FieldInfo fieldInfo : fieldInfos) {
if (canLoad(fieldInfo)) {
final String field = fieldInfo.name;
final String id = docValuesId(segment,
fieldInfo.number);
values.put(field,
loadDocValues(docCount, dir, id, getDocValuesType(fieldInfo), context));
}
}
success = true;
} finally {
if (!success) {
// if we fail we must close all opened resources if there are any
try {
closeInternal(values.values());
} catch (Throwable t) {} // keep our original exception
}
}
return values;
}
/** Returns true if this field indexed doc values. */
protected boolean canLoad(FieldInfo info) {
return info.hasDocValues();
}
/** Returns the doc values type for this field. */
protected Type getDocValuesType(FieldInfo info) {
return info.getDocValuesType();
}
/** Returns true if any fields indexed doc values. */
protected boolean anyDocValuesFields(FieldInfos infos) {
return infos.hasDocValues();
}
/** Returns the unique segment and field id for any
* per-field files this implementation needs to write. */
public static String docValuesId(String segmentsName, int fieldId) {
return segmentsName + "_" + fieldId;
}
/**
* Loads a {@link DocValues} instance depending on the given {@link Type}.
* Codecs that use different implementations for a certain {@link Type} can
* simply override this method and return their custom implementations.
*
* @param docCount
* number of documents in the segment
* @param dir
* the {@link Directory} to load the {@link DocValues} from
* @param id
* the unique file ID within the segment
* @param type
* the type to load
* @return a {@link DocValues} instance for the given type
* @throws IOException
* if an {@link IOException} occurs
* @throws IllegalArgumentException
* if the given {@link Type} is not supported
*/
protected abstract DocValues loadDocValues(int docCount, Directory dir, String id,
DocValues.Type type, IOContext context) throws IOException;
}

View File

@ -18,11 +18,9 @@ package org.apache.lucene.codecs.lucene40;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
@ -49,9 +47,7 @@ public final class Lucene40Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@ -76,11 +72,6 @@ public final class Lucene40Codec extends Codec {
return vectorsFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
@ -95,11 +86,6 @@ public final class Lucene40Codec extends Codec {
public final SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
// nocommit need a read-only Lucene40SimpleDVFormat
private final SimpleDocValuesFormat defaultDVFormat = SimpleDocValuesFormat.forName("Disk");

View File

@ -1,86 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.0 PerDocConsumer implementation that uses compound file.
*
* @see Lucene40DocValuesFormat
* @lucene.experimental
*/
public class Lucene40DocValuesConsumer extends DocValuesWriterBase {
private final Directory mainDirectory;
private Directory directory;
private final String segmentSuffix;
/** Segment suffix used when writing doc values index files. */
public final static String DOC_VALUES_SEGMENT_SUFFIX = "dv";
/** Sole constructor. */
public Lucene40DocValuesConsumer(PerDocWriteState state, String segmentSuffix) {
super(state);
this.segmentSuffix = segmentSuffix;
mainDirectory = state.directory;
//TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
}
@Override
protected Directory getDirectory() throws IOException {
// lazy init
if (directory == null) {
directory = new CompoundFileDirectory(mainDirectory,
IndexFileNames.segmentFileName(segmentName, segmentSuffix,
IndexFileNames.COMPOUND_FILE_EXTENSION), context, true);
}
return directory;
}
@Override
public void close() throws IOException {
if (directory != null) {
directory.close();
}
}
@Override
public void abort() {
try {
close();
} catch (Throwable t) {
// ignore
} finally {
// TODO: why the inconsistency here? we do this, but not SimpleText (which says IFD
// will do it).
// TODO: check that IFD really does this always, even if codec abort() throws a
// RuntimeException (e.g. ThreadInterruptedException)
IOUtils.deleteFilesIgnoringExceptions(mainDirectory, IndexFileNames.segmentFileName(
segmentName, segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
IndexFileNames.segmentFileName(segmentName, segmentSuffix,
IndexFileNames.COMPOUND_FILE_ENTRIES_EXTENSION));
}
}
}

View File

@ -1,146 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.DocValues; // javadocs
import org.apache.lucene.index.DocValues.Type; // javadocs
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.CompoundFileDirectory; // javadocs
import org.apache.lucene.store.DataOutput; // javadocs
import org.apache.lucene.util.packed.PackedInts; // javadocs
/**
* Lucene 4.0 DocValues format.
* <p>
* Files:
* <ul>
* <li><tt>.dv.cfs</tt>: {@link CompoundFileDirectory compound container}</li>
* <li><tt>.dv.cfe</tt>: {@link CompoundFileDirectory compound entries}</li>
* </ul>
* Entries within the compound file:
* <ul>
* <li><tt>&lt;segment&gt;_&lt;fieldNumber&gt;.dat</tt>: data values</li>
* <li><tt>&lt;segment&gt;_&lt;fieldNumber&gt;.idx</tt>: index into the .dat for DEREF types</li>
* </ul>
* <p>
* There are several many types of {@link DocValues} with different encodings.
* From the perspective of filenames, all types store their values in <tt>.dat</tt>
* entries within the compound file. In the case of dereferenced/sorted types, the <tt>.dat</tt>
* actually contains only the unique values, and an additional <tt>.idx</tt> file contains
* pointers to these unique values.
* </p>
* Formats:
* <ul>
* <li>{@link Type#VAR_INTS VAR_INTS} .dat --&gt; Header, PackedType, MinValue,
* DefaultValue, PackedStream</li>
* <li>{@link Type#FIXED_INTS_8 FIXED_INTS_8} .dat --&gt; Header, ValueSize,
* {@link DataOutput#writeByte Byte}<sup>maxdoc</sup></li>
* <li>{@link Type#FIXED_INTS_16 FIXED_INTS_16} .dat --&gt; Header, ValueSize,
* {@link DataOutput#writeShort Short}<sup>maxdoc</sup></li>
* <li>{@link Type#FIXED_INTS_32 FIXED_INTS_32} .dat --&gt; Header, ValueSize,
* {@link DataOutput#writeInt Int32}<sup>maxdoc</sup></li>
* <li>{@link Type#FIXED_INTS_64 FIXED_INTS_64} .dat --&gt; Header, ValueSize,
* {@link DataOutput#writeLong Int64}<sup>maxdoc</sup></li>
* <li>{@link Type#FLOAT_32 FLOAT_32} .dat --&gt; Header, ValueSize,
* Float32<sup>maxdoc</sup></li>
* <li>{@link Type#FLOAT_64 FLOAT_64} .dat --&gt; Header, ValueSize,
* Float64<sup>maxdoc</sup></li>
* <li>{@link Type#BYTES_FIXED_STRAIGHT BYTES_FIXED_STRAIGHT} .dat --&gt; Header, ValueSize,
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>maxdoc</sup></li>
* <li>{@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} .idx --&gt; Header, MaxAddress,
* Addresses</li>
* <li>{@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} .dat --&gt; Header, TotalBytes,
* Addresses, ({@link DataOutput#writeByte Byte} *
* <i>variable ValueSize</i>)<sup>maxdoc</sup></li>
* <li>{@link Type#BYTES_FIXED_DEREF BYTES_FIXED_DEREF} .idx --&gt; Header, NumValues,
* Addresses</li>
* <li>{@link Type#BYTES_FIXED_DEREF BYTES_FIXED_DEREF} .dat --&gt; Header, ValueSize,
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>NumValues</sup></li>
* <li>{@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF} .idx --&gt; Header, TotalVarBytes,
* Addresses</li>
* <li>{@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF} .dat --&gt; Header,
* (LengthPrefix + {@link DataOutput#writeByte Byte} * <i>variable ValueSize</i>)<sup>NumValues</sup></li>
* <li>{@link Type#BYTES_FIXED_SORTED BYTES_FIXED_SORTED} .idx --&gt; Header, NumValues,
* Ordinals</li>
* <li>{@link Type#BYTES_FIXED_SORTED BYTES_FIXED_SORTED} .dat --&gt; Header, ValueSize,
* ({@link DataOutput#writeByte Byte} * ValueSize)<sup>NumValues</sup></li>
* <li>{@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED} .idx --&gt; Header, TotalVarBytes,
* Addresses, Ordinals</li>
* <li>{@link Type#BYTES_VAR_SORTED BYTES_VAR_SORTED} .dat --&gt; Header,
* ({@link DataOutput#writeByte Byte} * <i>variable ValueSize</i>)<sup>NumValues</sup></li>
* </ul>
* Data Types:
* <ul>
* <li>Header --&gt; {@link CodecUtil#writeHeader CodecHeader}</li>
* <li>PackedType --&gt; {@link DataOutput#writeByte Byte}</li>
* <li>MaxAddress, MinValue, DefaultValue --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>PackedStream, Addresses, Ordinals --&gt; {@link PackedInts}</li>
* <li>ValueSize, NumValues --&gt; {@link DataOutput#writeInt Int32}</li>
* <li>Float32 --&gt; 32-bit float encoded with {@link Float#floatToRawIntBits(float)}
* then written as {@link DataOutput#writeInt Int32}</li>
* <li>Float64 --&gt; 64-bit float encoded with {@link Double#doubleToRawLongBits(double)}
* then written as {@link DataOutput#writeLong Int64}</li>
* <li>TotalBytes --&gt; {@link DataOutput#writeVLong VLong}</li>
* <li>TotalVarBytes --&gt; {@link DataOutput#writeLong Int64}</li>
* <li>LengthPrefix --&gt; Length of the data value as {@link DataOutput#writeVInt VInt} (maximum
* of 2 bytes)</li>
* </ul>
* Notes:
* <ul>
* <li>PackedType is a 0 when compressed, 1 when the stream is written as 64-bit integers.</li>
* <li>Addresses stores pointers to the actual byte location (indexed by docid). In the VAR_STRAIGHT
* case, each entry can have a different length, so to determine the length, docid+1 is
* retrieved. A sentinel address is written at the end for the VAR_STRAIGHT case, so the Addresses
* stream contains maxdoc+1 indices. For the deduplicated VAR_DEREF case, each length
* is encoded as a prefix to the data itself as a {@link DataOutput#writeVInt VInt}
* (maximum of 2 bytes).</li>
* <li>Ordinals stores the term ID in sorted order (indexed by docid). In the FIXED_SORTED case,
* the address into the .dat can be computed from the ordinal as
* <code>Header+ValueSize+(ordinal*ValueSize)</code> because the byte length is fixed.
* In the VAR_SORTED case, there is double indirection (docid -> ordinal -> address), but
* an additional sentinel ordinal+address is always written (so there are NumValues+1 ordinals). To
* determine the length, ord+1's address is looked up as well.</li>
* <li>{@link Type#BYTES_VAR_STRAIGHT BYTES_VAR_STRAIGHT} in contrast to other straight
* variants uses a <tt>.idx</tt> file to improve lookup perfromance. In contrast to
* {@link Type#BYTES_VAR_DEREF BYTES_VAR_DEREF} it doesn't apply deduplication of the document values.
* </li>
* </ul>
*/
public class Lucene40DocValuesFormat extends DocValuesFormat {
/** Sole constructor. */
public Lucene40DocValuesFormat() {
}
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new Lucene40DocValuesConsumer(state, Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX);
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new Lucene40DocValuesProducer(state, Lucene40DocValuesConsumer.DOC_VALUES_SEGMENT_SUFFIX);
}
}

View File

@ -1,113 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Map;
import java.util.TreeMap;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.codecs.lucene40.values.Bytes;
import org.apache.lucene.codecs.lucene40.values.Floats;
import org.apache.lucene.codecs.lucene40.values.Ints;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
/**
* Lucene 4.0 PerDocProducer implementation that uses compound file.
*
* @see Lucene40DocValuesFormat
* @lucene.experimental
*/
public class Lucene40DocValuesProducer extends PerDocProducerBase {
/** Maps field name to {@link DocValues} instance. */
protected final TreeMap<String,DocValues> docValues;
private final Directory cfs;
/**
* Creates a new {@link Lucene40DocValuesProducer} instance and loads all
* {@link DocValues} instances for this segment and codec.
*/
public Lucene40DocValuesProducer(SegmentReadState state, String segmentSuffix) throws IOException {
if (anyDocValuesFields(state.fieldInfos)) {
cfs = new CompoundFileDirectory(state.directory,
IndexFileNames.segmentFileName(state.segmentInfo.name,
segmentSuffix, IndexFileNames.COMPOUND_FILE_EXTENSION),
state.context, false);
docValues = load(state.fieldInfos, state.segmentInfo.name, state.segmentInfo.getDocCount(), cfs, state.context);
} else {
cfs = null;
docValues = new TreeMap<String,DocValues>();
}
}
@Override
protected Map<String,DocValues> docValues() {
return docValues;
}
@Override
protected void closeInternal(Collection<? extends Closeable> closeables) throws IOException {
if (cfs != null) {
final ArrayList<Closeable> list = new ArrayList<Closeable>(closeables);
list.add(cfs);
IOUtils.close(list);
} else {
IOUtils.close(closeables);
}
}
@Override
protected DocValues loadDocValues(int docCount, Directory dir, String id,
Type type, IOContext context) throws IOException {
switch (type) {
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
return Ints.getValues(dir, id, docCount, type, context);
case FLOAT_32:
return Floats.getValues(dir, id, docCount, context, type);
case FLOAT_64:
return Floats.getValues(dir, id, docCount, context, type);
case BYTES_FIXED_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, getComparator(), context);
case BYTES_FIXED_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, getComparator(), context);
case BYTES_FIXED_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, getComparator(), context);
case BYTES_VAR_STRAIGHT:
return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, getComparator(), context);
case BYTES_VAR_DEREF:
return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, getComparator(), context);
case BYTES_VAR_SORTED:
return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, getComparator(), context);
default:
throw new IllegalStateException("unrecognized index values mode " + type);
}
}
}

View File

@ -1,125 +0,0 @@
package org.apache.lucene.codecs.lucene40;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.AtomicReader;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.CompoundFileDirectory; // javadocs
/**
* Lucene 4.0 Norms Format.
* <p>
* Files:
* <ul>
* <li><tt>.nrm.cfs</tt>: {@link CompoundFileDirectory compound container}</li>
* <li><tt>.nrm.cfe</tt>: {@link CompoundFileDirectory compound entries}</li>
* </ul>
* Norms are implemented as DocValues, so other than file extension, norms are
* written exactly the same way as {@link Lucene40DocValuesFormat DocValues}.
*
* @see Lucene40DocValuesFormat
* @lucene.experimental
*/
public class Lucene40NormsFormat extends NormsFormat {
private final static String NORMS_SEGMENT_SUFFIX = "nrm";
/** Sole constructor. */
public Lucene40NormsFormat() {
}
@Override
public PerDocConsumer docsConsumer(PerDocWriteState state) throws IOException {
return new Lucene40NormsDocValuesConsumer(state, NORMS_SEGMENT_SUFFIX);
}
@Override
public PerDocProducer docsProducer(SegmentReadState state) throws IOException {
return new Lucene40NormsDocValuesProducer(state, NORMS_SEGMENT_SUFFIX);
}
/**
* Lucene 4.0 PerDocProducer implementation that uses compound file.
*
* @see Lucene40DocValuesFormat
*/
public static class Lucene40NormsDocValuesProducer extends Lucene40DocValuesProducer {
/** Sole constructor. */
public Lucene40NormsDocValuesProducer(SegmentReadState state,
String segmentSuffix) throws IOException {
super(state, segmentSuffix);
}
@Override
protected boolean canLoad(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
@Override
protected boolean anyDocValuesFields(FieldInfos infos) {
return infos.hasNorms();
}
}
/**
* Lucene 4.0 PerDocConsumer implementation that uses compound file.
*
* @see Lucene40DocValuesFormat
* @lucene.experimental
*/
public static class Lucene40NormsDocValuesConsumer extends Lucene40DocValuesConsumer {
/** Sole constructor. */
public Lucene40NormsDocValuesConsumer(PerDocWriteState state,
String segmentSuffix) {
super(state, segmentSuffix);
}
@Override
protected DocValues getDocValuesForMerge(AtomicReader reader, FieldInfo info)
throws IOException {
return reader.normValues(info.name);
}
@Override
protected boolean canMerge(FieldInfo info) {
return info.hasNorms();
}
@Override
protected Type getDocValuesType(FieldInfo info) {
return info.getNormType();
}
}
}

View File

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

View File

@ -1,164 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
/**
* Base class for disk resident source implementations
* @lucene.internal
*/
abstract class DirectSource extends Source {
protected final IndexInput data;
private final ToNumeric toNumeric;
protected final long baseOffset;
public DirectSource(IndexInput input, Type type) {
super(type);
this.data = input;
baseOffset = input.getFilePointer();
switch (type) {
case FIXED_INTS_16:
toNumeric = new ShortToLong();
break;
case FLOAT_32:
toNumeric = new BytesToFloat();
break;
case FLOAT_64:
toNumeric = new BytesToDouble();
break;
case FIXED_INTS_32:
toNumeric = new IntToLong();
break;
case FIXED_INTS_8:
toNumeric = new ByteToLong();
break;
default:
toNumeric = new LongToLong();
}
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
try {
final int sizeToRead = position(docID);
ref.offset = 0;
ref.grow(sizeToRead);
data.readBytes(ref.bytes, 0, sizeToRead);
ref.length = sizeToRead;
return ref;
} catch (IOException ex) {
throw new IllegalStateException("failed to get value for docID: " + docID, ex);
}
}
@Override
public long getInt(int docID) {
try {
position(docID);
return toNumeric.toLong(data);
} catch (IOException ex) {
throw new IllegalStateException("failed to get value for docID: " + docID, ex);
}
}
@Override
public double getFloat(int docID) {
try {
position(docID);
return toNumeric.toDouble(data);
} catch (IOException ex) {
throw new IllegalStateException("failed to get value for docID: " + docID, ex);
}
}
protected abstract int position(int docID) throws IOException;
private abstract static class ToNumeric {
abstract long toLong(IndexInput input) throws IOException;
double toDouble(IndexInput input) throws IOException {
return toLong(input);
}
}
private static final class ByteToLong extends ToNumeric {
@Override
long toLong(IndexInput input) throws IOException {
return input.readByte();
}
}
private static final class ShortToLong extends ToNumeric {
@Override
long toLong(IndexInput input) throws IOException {
return input.readShort();
}
}
private static final class IntToLong extends ToNumeric {
@Override
long toLong(IndexInput input) throws IOException {
return input.readInt();
}
}
private static final class BytesToFloat extends ToNumeric {
@Override
long toLong(IndexInput input) {
throw new UnsupportedOperationException("ints are not supported");
}
@Override
double toDouble(IndexInput input) throws IOException {
return Float.intBitsToFloat(input.readInt());
}
}
private static final class BytesToDouble extends ToNumeric {
@Override
long toLong(IndexInput input) {
throw new UnsupportedOperationException("ints are not supported");
}
@Override
double toDouble(IndexInput input) throws IOException {
return Double.longBitsToDouble(input.readLong());
}
}
private static final class LongToLong extends ToNumeric {
@Override
long toLong(IndexInput input) throws IOException {
return input.readLong();
}
@Override
double toDouble(IndexInput input) {
throw new UnsupportedOperationException("doubles are not supported");
}
}
}

View File

@ -1,107 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.PerDocProducerBase;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.lucene40.values.Writer;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.DocValues.Type; // javadoc
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.packed.PackedInts;
/**
* Abstract base class for PerDocConsumer implementations
*
* @lucene.experimental
*/
public abstract class DocValuesWriterBase extends PerDocConsumer {
/** Segment name to use when writing files. */
protected final String segmentName;
private final Counter bytesUsed;
/** {@link IOContext} to use when writing files. */
protected final IOContext context;
private final float acceptableOverheadRatio;
/**
* Filename extension for index files
*/
public static final String INDEX_EXTENSION = "idx";
/**
* Filename extension for data files.
*/
public static final String DATA_EXTENSION = "dat";
/**
* Creates {@code DocValuesWriterBase}, using {@link
* PackedInts#FAST}.
* @param state The state to initiate a {@link PerDocConsumer} instance
*/
protected DocValuesWriterBase(PerDocWriteState state) {
this(state, PackedInts.FAST);
}
/**
* Creates {@code DocValuesWriterBase}.
* @param state The state to initiate a {@link PerDocConsumer} instance
* @param acceptableOverheadRatio
* how to trade space for speed. This option is only applicable for
* docvalues of type {@link Type#BYTES_FIXED_SORTED} and
* {@link Type#BYTES_VAR_SORTED}.
* @see PackedInts#getReader(org.apache.lucene.store.DataInput)
*/
protected DocValuesWriterBase(PerDocWriteState state, float acceptableOverheadRatio) {
this.segmentName = state.segmentInfo.name;
this.bytesUsed = state.bytesUsed;
this.context = state.context;
this.acceptableOverheadRatio = acceptableOverheadRatio;
}
/** Returns the {@link Directory} that files should be
* written to. */
protected abstract Directory getDirectory() throws IOException;
@Override
public void close() throws IOException {
}
@Override
public DocValuesConsumer addValuesField(Type valueType, FieldInfo field) throws IOException {
return Writer.create(valueType,
PerDocProducerBase.docValuesId(segmentName, field.number),
getDirectory(), getComparator(), bytesUsed, context, acceptableOverheadRatio);
}
/** Returns the comparator used to sort {@link BytesRef}
* values. */
public Comparator<BytesRef> getComparator() throws IOException {
return BytesRef.getUTF8SortedAsUnicodeComparator();
}
}

View File

@ -1,135 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts;
// Stores fixed-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[]
/**
* @lucene.experimental
*/
class FixedDerefBytesImpl {
static final String CODEC_NAME_IDX = "FixedDerefBytesIdx";
static final String CODEC_NAME_DAT = "FixedDerefBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
public static class Writer extends DerefBytesWriterBase {
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, Type.BYTES_FIXED_DEREF);
}
@Override
protected void finishInternal(int docCount) throws IOException {
final int numValues = hash.size();
final IndexOutput datOut = getOrCreateDataOut();
datOut.writeInt(size);
if (size != -1) {
final BytesRef bytesRef = new BytesRef(size);
for (int i = 0; i < numValues; i++) {
hash.get(i, bytesRef);
datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
}
}
final IndexOutput idxOut = getOrCreateIndexOut();
idxOut.writeInt(numValues);
writeIndex(idxOut, docCount, numValues, docToEntry);
}
}
public static class FixedDerefReader extends BytesReaderBase {
private final int size;
private final int numValuesStored;
FixedDerefReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, Type.BYTES_FIXED_DEREF);
size = datIn.readInt();
numValuesStored = idxIn.readInt();
}
@Override
protected Source loadSource() throws IOException {
return new FixedDerefSource(cloneData(), cloneIndex(), size, numValuesStored);
}
@Override
protected Source loadDirectSource()
throws IOException {
return new DirectFixedDerefSource(cloneData(), cloneIndex(), size, getType());
}
@Override
public int getValueSize() {
return size;
}
}
static final class FixedDerefSource extends BytesSourceBase {
private final int size;
private final PackedInts.Reader addresses;
protected FixedDerefSource(IndexInput datIn, IndexInput idxIn, int size, long numValues) throws IOException {
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), size * numValues,
Type.BYTES_FIXED_DEREF);
this.size = size;
addresses = PackedInts.getReader(idxIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
return data.fillSlice(bytesRef, addresses.get(docID) * size, size);
}
}
final static class DirectFixedDerefSource extends DirectSource {
private final PackedInts.Reader index;
private final int size;
DirectFixedDerefSource(IndexInput data, IndexInput index, int size, Type type)
throws IOException {
super(data, type);
this.size = size;
this.index = PackedInts.getDirectReader(index);
}
@Override
protected int position(int docID) throws IOException {
data.seek(baseOffset + index.get(docID) * size);
return size;
}
}
}

View File

@ -1,232 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSortedSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.SortedBytesMergeUtils;
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.SortedBytesMergeUtils.IndexOutputBytesRefConsumer;
import org.apache.lucene.index.SortedBytesMergeUtils.MergeContext;
import org.apache.lucene.index.SortedBytesMergeUtils.SortedSourceSlice;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
// Stores fixed-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[]
/**
* @lucene.experimental
*/
class FixedSortedBytesImpl {
static final String CODEC_NAME_IDX = "FixedSortedBytesIdx";
static final String CODEC_NAME_DAT = "FixedSortedBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static final class Writer extends DerefBytesWriterBase {
private final Comparator<BytesRef> comp;
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Counter bytesUsed, IOContext context, float acceptableOverheadRatio) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, acceptableOverheadRatio, Type.BYTES_FIXED_SORTED);
this.comp = comp;
}
@Override
public void merge(MergeState mergeState, DocValues[] docValues)
throws IOException {
boolean success = false;
try {
final MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_FIXED_SORTED, docValues, comp, mergeState.segmentInfo.getDocCount());
List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState.docBase, mergeState.docMaps, docValues, ctx);
final IndexOutput datOut = getOrCreateDataOut();
datOut.writeInt(ctx.sizePerValues);
final int maxOrd = SortedBytesMergeUtils.mergeRecords(ctx, new IndexOutputBytesRefConsumer(datOut), slices);
final IndexOutput idxOut = getOrCreateIndexOut();
idxOut.writeInt(maxOrd);
final PackedInts.Writer ordsWriter = PackedInts.getWriter(idxOut, ctx.docToEntry.length,
PackedInts.bitsRequired(maxOrd), PackedInts.DEFAULT);
for (SortedSourceSlice slice : slices) {
slice.writeOrds(ordsWriter);
}
ordsWriter.finish();
success = true;
} finally {
releaseResources();
if (success) {
IOUtils.close(getIndexOut(), getDataOut());
} else {
IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
}
}
}
// Important that we get docCount, in case there were
// some last docs that we didn't see
@Override
public void finishInternal(int docCount) throws IOException {
fillDefault(docCount);
final IndexOutput datOut = getOrCreateDataOut();
final int count = hash.size();
final int[] address = new int[count];
datOut.writeInt(size);
if (size != -1) {
final int[] sortedEntries = hash.sort(comp);
// first dump bytes data, recording address as we go
final BytesRef spare = new BytesRef(size);
for (int i = 0; i < count; i++) {
final int e = sortedEntries[i];
final BytesRef bytes = hash.get(e, spare);
assert bytes.length == size;
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
address[e] = i;
}
}
final IndexOutput idxOut = getOrCreateIndexOut();
idxOut.writeInt(count);
writeIndex(idxOut, docCount, count, address, docToEntry);
}
}
static final class Reader extends BytesReaderBase {
private final int size;
private final int valueCount;
private final Comparator<BytesRef> comparator;
public Reader(Directory dir, String id, int maxDoc, IOContext context,
Type type, Comparator<BytesRef> comparator) throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, type);
size = datIn.readInt();
valueCount = idxIn.readInt();
this.comparator = comparator;
}
@Override
protected Source loadSource() throws IOException {
return new FixedSortedSource(cloneData(), cloneIndex(), size, valueCount,
comparator);
}
@Override
protected Source loadDirectSource() throws IOException {
return new DirectFixedSortedSource(cloneData(), cloneIndex(), size,
valueCount, comparator, type);
}
@Override
public int getValueSize() {
return size;
}
}
static final class FixedSortedSource extends BytesSortedSourceBase {
private final int valueCount;
private final int size;
FixedSortedSource(IndexInput datIn, IndexInput idxIn, int size,
int numValues, Comparator<BytesRef> comp) throws IOException {
super(datIn, idxIn, comp, size * numValues, Type.BYTES_FIXED_SORTED,
false);
this.size = size;
this.valueCount = numValues;
closeIndexInput();
}
@Override
public int getValueCount() {
return valueCount;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
return data.fillSlice(bytesRef, (ord * size), size);
}
}
static final class DirectFixedSortedSource extends SortedSource {
final PackedInts.Reader docToOrdIndex;
private final IndexInput datIn;
private final long basePointer;
private final int size;
private final int valueCount;
DirectFixedSortedSource(IndexInput datIn, IndexInput idxIn, int size,
int valueCount, Comparator<BytesRef> comp, Type type)
throws IOException {
super(type, comp);
docToOrdIndex = PackedInts.getDirectReader(idxIn);
basePointer = datIn.getFilePointer();
this.datIn = datIn;
this.size = size;
this.valueCount = valueCount;
}
@Override
public int ord(int docID) {
return (int) docToOrdIndex.get(docID);
}
@Override
public boolean hasPackedDocToOrd() {
return true;
}
@Override
public PackedInts.Reader getDocToOrd() {
return docToOrdIndex;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
try {
datIn.seek(basePointer + size * ord);
bytesRef.offset = 0;
bytesRef.grow(size);
datIn.readBytes(bytesRef.bytes, 0, size);
bytesRef.length = size;
return bytesRef;
} catch (IOException ex) {
throw new IllegalStateException("failed to getByOrd", ex);
}
}
@Override
public int getValueCount() {
return valueCount;
}
}
}

View File

@ -1,369 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesWriterBase;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StraightBytesDocValuesField;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
// Simplest storage: stores fixed length byte[] per
// document, with no dedup and no sorting.
/**
* @lucene.experimental
*/
class FixedStraightBytesImpl {
static final String CODEC_NAME = "FixedStraightBytes";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static abstract class FixedBytesWriterBase extends BytesWriterBase {
protected final StraightBytesDocValuesField bytesSpareField = new StraightBytesDocValuesField("", new BytesRef(), true);
protected int lastDocID = -1;
// start at -1 if the first added value is > 0
protected int size = -1;
private final int byteBlockSize = BYTE_BLOCK_SIZE;
private final ByteBlockPool pool;
protected FixedBytesWriterBase(Directory dir, String id, String codecNameDat,
int version, Counter bytesUsed, IOContext context) {
this(dir, id, codecNameDat, version, bytesUsed, context, Type.BYTES_FIXED_STRAIGHT);
}
protected FixedBytesWriterBase(Directory dir, String id, String codecNameDat,
int version, Counter bytesUsed, IOContext context, Type type) {
super(dir, id, null, codecNameDat, version, bytesUsed, context, type);
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
pool.nextBuffer();
}
@Override
public void add(int docID, StorableField value) throws IOException {
final BytesRef bytes = value.binaryValue();
assert bytes != null;
assert lastDocID < docID;
if (size == -1) {
if (bytes.length > BYTE_BLOCK_SIZE) {
throw new IllegalArgumentException("bytes arrays > " + BYTE_BLOCK_SIZE + " are not supported");
}
size = bytes.length;
} else if (bytes.length != size) {
throw new IllegalArgumentException("byte[] length changed for BYTES_FIXED_STRAIGHT type (before=" + size + " now=" + bytes.length);
}
if (lastDocID+1 < docID) {
advancePool(docID);
}
pool.copy(bytes);
lastDocID = docID;
}
private final void advancePool(int docID) {
long numBytes = (docID - (lastDocID+1))*size;
while(numBytes > 0) {
if (numBytes + pool.byteUpto < byteBlockSize) {
pool.byteUpto += numBytes;
numBytes = 0;
} else {
numBytes -= byteBlockSize - pool.byteUpto;
pool.nextBuffer();
}
}
assert numBytes == 0;
}
protected void set(BytesRef ref, int docId) {
assert BYTE_BLOCK_SIZE % size == 0 : "BYTE_BLOCK_SIZE ("+ BYTE_BLOCK_SIZE + ") must be a multiple of the size: " + size;
ref.offset = docId*size;
ref.length = size;
pool.deref(ref);
}
protected void resetPool() {
pool.reset(false, false);
}
protected void writeData(IndexOutput out) throws IOException {
pool.writePool(out);
}
protected void writeZeros(int num, IndexOutput out) throws IOException {
final byte[] zeros = new byte[size];
for (int i = 0; i < num; i++) {
out.writeBytes(zeros, zeros.length);
}
}
@Override
public int getValueSize() {
return size;
}
}
static class Writer extends FixedBytesWriterBase {
private boolean hasMerged;
private IndexOutput datOut;
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
}
public Writer(Directory dir, String id, String codecNameDat, int version, Counter bytesUsed, IOContext context) {
super(dir, id, codecNameDat, version, bytesUsed, context);
}
@Override
protected void merge(DocValues readerIn, int docBase, int docCount, Bits liveDocs) throws IOException {
datOut = getOrCreateDataOut();
boolean success = false;
try {
if (!hasMerged && size != -1) {
datOut.writeInt(size);
}
if (liveDocs == null && tryBulkMerge(readerIn)) {
FixedStraightReader reader = (FixedStraightReader) readerIn;
final int maxDocs = reader.maxDoc;
if (maxDocs == 0) {
return;
}
if (size == -1) {
size = reader.size;
datOut.writeInt(size);
} else if (size != reader.size) {
throw new IllegalArgumentException("expected bytes size=" + size
+ " but got " + reader.size);
}
if (lastDocID+1 < docBase) {
fill(datOut, docBase);
lastDocID = docBase-1;
}
// TODO should we add a transfer to API to each reader?
final IndexInput cloneData = reader.cloneData();
try {
datOut.copyBytes(cloneData, size * maxDocs);
} finally {
IOUtils.close(cloneData);
}
lastDocID += maxDocs;
} else {
super.merge(readerIn, docBase, docCount, liveDocs);
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(datOut);
}
hasMerged = true;
}
}
protected boolean tryBulkMerge(DocValues docValues) {
return docValues instanceof FixedStraightReader;
}
@Override
protected void mergeDoc(StoredField scratchField, Source source, int docID, int sourceDoc) throws IOException {
assert lastDocID < docID;
setMergeBytes(source, sourceDoc);
if (size == -1) {
size = bytesRef.length;
datOut.writeInt(size);
}
assert size == bytesRef.length : "size: " + size + " ref: " + bytesRef.length;
if (lastDocID+1 < docID) {
fill(datOut, docID);
}
datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
lastDocID = docID;
}
protected void setMergeBytes(Source source, int sourceDoc) {
source.getBytes(sourceDoc, bytesRef);
}
// Fills up to but not including this docID
private void fill(IndexOutput datOut, int docID) throws IOException {
assert size >= 0;
writeZeros((docID - (lastDocID+1)), datOut);
}
@Override
public void finish(int docCount) throws IOException {
boolean success = false;
try {
if (!hasMerged) {
// indexing path - no disk IO until here
assert datOut == null;
datOut = getOrCreateDataOut();
if (size == -1) {
datOut.writeInt(0);
} else {
datOut.writeInt(size);
writeData(datOut);
}
if (lastDocID + 1 < docCount) {
fill(datOut, docCount);
}
} else {
// merge path - datOut should be initialized
assert datOut != null;
if (size == -1) {// no data added
datOut.writeInt(0);
} else {
fill(datOut, docCount);
}
}
success = true;
} finally {
resetPool();
if (success) {
IOUtils.close(datOut);
} else {
IOUtils.closeWhileHandlingException(datOut);
}
}
}
}
public static class FixedStraightReader extends BytesReaderBase {
protected final int size;
protected final int maxDoc;
FixedStraightReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
this(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, Type.BYTES_FIXED_STRAIGHT);
}
protected FixedStraightReader(Directory dir, String id, String codecNameDat, int version, int maxDoc, IOContext context, Type type) throws IOException {
super(dir, id, null, codecNameDat, version, false, context, type);
size = datIn.readInt();
this.maxDoc = maxDoc;
}
@Override
protected Source loadSource() throws IOException {
return size == 1 ? new SingleByteSource(cloneData(), maxDoc) :
new FixedStraightSource(cloneData(), size, maxDoc, type);
}
@Override
public void close() throws IOException {
datIn.close();
}
@Override
protected Source loadDirectSource() throws IOException {
return new DirectFixedStraightSource(cloneData(), size, getType());
}
@Override
public int getValueSize() {
return size;
}
}
// specialized version for single bytes
private static final class SingleByteSource extends Source {
private final byte[] data;
public SingleByteSource(IndexInput datIn, int maxDoc) throws IOException {
super(Type.BYTES_FIXED_STRAIGHT);
try {
data = new byte[maxDoc];
datIn.readBytes(data, 0, data.length, false);
} finally {
IOUtils.close(datIn);
}
}
@Override
public boolean hasArray() {
return true;
}
@Override
public Object getArray() {
return data;
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
bytesRef.length = 1;
bytesRef.bytes = data;
bytesRef.offset = docID;
return bytesRef;
}
}
private final static class FixedStraightSource extends BytesSourceBase {
private final int size;
public FixedStraightSource(IndexInput datIn, int size, int maxDoc, Type type)
throws IOException {
super(datIn, null, new PagedBytes(PAGED_BYTES_BITS), size * maxDoc,
type);
this.size = size;
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
return data.fillSlice(bytesRef, size * ((long) docID), size);
}
}
public final static class DirectFixedStraightSource extends DirectSource {
private final int size;
DirectFixedStraightSource(IndexInput input, int size, Type type) {
super(input, type);
this.size = size;
}
@Override
protected int position(int docID) throws IOException {
data.seek(baseOffset + size * ((long) docID));
return size;
}
}
}

View File

@ -1,139 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesArraySource;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
/**
* Exposes {@link Writer} and reader ({@link Source}) for 32 bit and 64 bit
* floating point values.
* <p>
* Current implementations store either 4 byte or 8 byte floating points with
* full precision without any compression.
*
* @lucene.experimental
*/
public class Floats {
/** Codec name, written in the header. */
protected static final String CODEC_NAME = "Floats";
/** Initial version. */
protected static final int VERSION_START = 0;
/** Current version. */
protected static final int VERSION_CURRENT = VERSION_START;
/** Sole constructor. */
private Floats() {
}
/** Creates and returns a {@link DocValuesConsumer} to
* write float values. */
public static DocValuesConsumer getWriter(Directory dir, String id, Counter bytesUsed,
IOContext context, Type type) {
return new FloatsWriter(dir, id, bytesUsed, context, type);
}
/** Creates and returns a {@link DocValues} to
* read previously written float values. */
public static DocValues getValues(Directory dir, String id, int maxDoc, IOContext context, Type type)
throws IOException {
return new FloatsReader(dir, id, maxDoc, context, type);
}
private static int typeToSize(Type type) {
switch (type) {
case FLOAT_32:
return 4;
case FLOAT_64:
return 8;
default:
throw new IllegalStateException("illegal type " + type);
}
}
final static class FloatsWriter extends FixedStraightBytesImpl.Writer {
private final int size;
private final DocValuesArraySource template;
public FloatsWriter(Directory dir, String id, Counter bytesUsed,
IOContext context, Type type) {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
size = typeToSize(type);
this.bytesRef = new BytesRef(size);
bytesRef.length = size;
template = DocValuesArraySource.forType(type);
assert template != null;
}
@Override
protected boolean tryBulkMerge(DocValues docValues) {
// only bulk merge if value type is the same otherwise size differs
return super.tryBulkMerge(docValues) && docValues.getType() == template.getType();
}
@Override
public void add(int docID, StorableField value) throws IOException {
template.toBytes(value.numericValue().doubleValue(), bytesRef);
bytesSpareField.setBytesValue(bytesRef);
super.add(docID, bytesSpareField);
}
@Override
protected void setMergeBytes(Source source, int sourceDoc) {
final double value = source.getFloat(sourceDoc);
template.toBytes(value, bytesRef);
}
}
final static class FloatsReader extends FixedStraightBytesImpl.FixedStraightReader {
final DocValuesArraySource arrayTemplate;
FloatsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, type);
arrayTemplate = DocValuesArraySource.forType(type);
assert size == 4 || size == 8: "wrong size=" + size + " type=" + type + " id=" + id;
}
@Override
protected Source loadSource() throws IOException {
final IndexInput indexInput = cloneData();
try {
return arrayTemplate.newFromInput(indexInput, maxDoc);
} finally {
IOUtils.close(indexInput);
}
}
}
}

View File

@ -1,161 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesArraySource;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
/**
* Stores ints packed and fixed with fixed-bit precision.
*
* @lucene.experimental
*/
public final class Ints {
/** Codec name, written in the header. */
protected static final String CODEC_NAME = "Ints";
/** Initial version. */
protected static final int VERSION_START = 0;
/** Current version. */
protected static final int VERSION_CURRENT = VERSION_START;
/** Sole constructor. */
private Ints() {
}
/** Creates and returns a {@link DocValuesConsumer} to
* write int values. */
public static DocValuesConsumer getWriter(Directory dir, String id, Counter bytesUsed,
Type type, IOContext context) {
return type == Type.VAR_INTS ? new PackedIntValues.PackedIntsWriter(dir, id,
bytesUsed, context) : new IntsWriter(dir, id, bytesUsed, context, type);
}
/** Creates and returns a {@link DocValues} to
* read previously written int values. */
public static DocValues getValues(Directory dir, String id, int numDocs,
Type type, IOContext context) throws IOException {
return type == Type.VAR_INTS ? new PackedIntValues.PackedIntsReader(dir, id,
numDocs, context) : new IntsReader(dir, id, numDocs, context, type);
}
private static Type sizeToType(int size) {
switch (size) {
case 1:
return Type.FIXED_INTS_8;
case 2:
return Type.FIXED_INTS_16;
case 4:
return Type.FIXED_INTS_32;
case 8:
return Type.FIXED_INTS_64;
default:
throw new IllegalStateException("illegal size " + size);
}
}
private static int typeToSize(Type type) {
switch (type) {
case FIXED_INTS_16:
return 2;
case FIXED_INTS_32:
return 4;
case FIXED_INTS_64:
return 8;
case FIXED_INTS_8:
return 1;
default:
throw new IllegalStateException("illegal type " + type);
}
}
static class IntsWriter extends FixedStraightBytesImpl.Writer {
private final DocValuesArraySource template;
public IntsWriter(Directory dir, String id, Counter bytesUsed,
IOContext context, Type valueType) {
this(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, valueType);
}
protected IntsWriter(Directory dir, String id, String codecName,
int version, Counter bytesUsed, IOContext context, Type valueType) {
super(dir, id, codecName, version, bytesUsed, context);
size = typeToSize(valueType);
this.bytesRef = new BytesRef(size);
bytesRef.length = size;
template = DocValuesArraySource.forType(valueType);
}
@Override
protected void setMergeBytes(Source source, int sourceDoc) {
final long value = source.getInt(sourceDoc);
template.toBytes(value, bytesRef);
}
@Override
public void add(int docID, StorableField value) throws IOException {
template.toBytes(value.numericValue().longValue(), bytesRef);
bytesSpareField.setBytesValue(bytesRef);
super.add(docID, bytesSpareField);
}
@Override
protected boolean tryBulkMerge(DocValues docValues) {
// only bulk merge if value type is the same otherwise size differs
return super.tryBulkMerge(docValues) && docValues.getType() == template.getType();
}
}
final static class IntsReader extends FixedStraightBytesImpl.FixedStraightReader {
private final DocValuesArraySource arrayTemplate;
IntsReader(Directory dir, String id, int maxDoc, IOContext context, Type type)
throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc,
context, type);
arrayTemplate = DocValuesArraySource.forType(type);
assert arrayTemplate != null;
assert type == sizeToType(size);
}
@Override
protected Source loadSource() throws IOException {
final IndexInput indexInput = cloneData();
try {
return arrayTemplate.newFromInput(indexInput, maxDoc);
} finally {
IOUtils.close(indexInput);
}
}
}
}

View File

@ -1,256 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesArraySource;
import org.apache.lucene.codecs.lucene40.values.FixedStraightBytesImpl.FixedBytesWriterBase;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
/**
* Stores integers using {@link PackedInts}
*
* @lucene.experimental
* */
class PackedIntValues {
private static final String CODEC_NAME = "PackedInts";
private static final byte PACKED = 0x00;
private static final byte FIXED_64 = 0x01;
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static class PackedIntsWriter extends FixedBytesWriterBase {
private long minValue;
private long maxValue;
private boolean started;
private int lastDocId = -1;
protected PackedIntsWriter(Directory dir, String id, Counter bytesUsed,
IOContext context) {
super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context, Type.VAR_INTS);
bytesRef = new BytesRef(8);
}
@Override
public void finish(int docCount) throws IOException {
boolean success = false;
final IndexOutput dataOut = getOrCreateDataOut();
try {
if (!started) {
minValue = maxValue = 0;
}
final long delta = maxValue - minValue;
// if we exceed the range of positive longs we must switch to fixed
// ints
if (delta <= (maxValue >= 0 && minValue <= 0 ? Long.MAX_VALUE
: Long.MAX_VALUE - 1) && delta >= 0) {
dataOut.writeByte(PACKED);
writePackedInts(dataOut, docCount);
return; // done
} else {
dataOut.writeByte(FIXED_64);
}
writeData(dataOut);
writeZeros(docCount - (lastDocID + 1), dataOut);
success = true;
} finally {
resetPool();
if (success) {
IOUtils.close(dataOut);
} else {
IOUtils.closeWhileHandlingException(dataOut);
}
}
}
private void writePackedInts(IndexOutput datOut, int docCount) throws IOException {
datOut.writeLong(minValue);
// write a default value to recognize docs without a value for that
// field
final long defaultValue = maxValue >= 0 && minValue <= 0 ? 0 - minValue
: ++maxValue - minValue;
datOut.writeLong(defaultValue);
PackedInts.Writer w = PackedInts.getWriter(datOut, docCount,
PackedInts.bitsRequired(maxValue - minValue), PackedInts.DEFAULT);
for (int i = 0; i < lastDocID + 1; i++) {
set(bytesRef, i);
byte[] bytes = bytesRef.bytes;
int offset = bytesRef.offset;
long asLong =
(((long)(bytes[offset+0] & 0xff) << 56) |
((long)(bytes[offset+1] & 0xff) << 48) |
((long)(bytes[offset+2] & 0xff) << 40) |
((long)(bytes[offset+3] & 0xff) << 32) |
((long)(bytes[offset+4] & 0xff) << 24) |
((long)(bytes[offset+5] & 0xff) << 16) |
((long)(bytes[offset+6] & 0xff) << 8) |
((long)(bytes[offset+7] & 0xff)));
w.add(asLong == 0 ? defaultValue : asLong - minValue);
}
for (int i = lastDocID + 1; i < docCount; i++) {
w.add(defaultValue);
}
w.finish();
}
@Override
public void add(int docID, StorableField docValue) throws IOException {
final long v = docValue.numericValue().longValue();
assert lastDocId < docID;
if (!started) {
started = true;
minValue = maxValue = v;
} else {
if (v < minValue) {
minValue = v;
} else if (v > maxValue) {
maxValue = v;
}
}
lastDocId = docID;
DocValuesArraySource.copyLong(bytesRef, v);
bytesSpareField.setBytesValue(bytesRef);
super.add(docID, bytesSpareField);
}
}
/**
* Opens all necessary files, but does not read any data in until you call
* {@link #loadSource}.
*/
static class PackedIntsReader extends DocValues {
private final IndexInput datIn;
private final byte type;
private final int numDocs;
private final DocValuesArraySource values;
protected PackedIntsReader(Directory dir, String id, int numDocs,
IOContext context) throws IOException {
datIn = dir.openInput(
IndexFileNames.segmentFileName(id, Bytes.DV_SEGMENT_SUFFIX, DocValuesWriterBase.DATA_EXTENSION),
context);
this.numDocs = numDocs;
boolean success = false;
try {
CodecUtil.checkHeader(datIn, CODEC_NAME, VERSION_START, VERSION_START);
type = datIn.readByte();
values = type == FIXED_64 ? DocValuesArraySource.forType(Type.FIXED_INTS_64) : null;
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(datIn);
}
}
}
/**
* Loads the actual values. You may call this more than once, eg if you
* already previously loaded but then discarded the Source.
*/
@Override
protected Source loadSource() throws IOException {
boolean success = false;
final Source source;
IndexInput input = null;
try {
input = datIn.clone();
if (values == null) {
source = new PackedIntsSource(input, false);
} else {
source = values.newFromInput(input, numDocs);
}
success = true;
return source;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(input, datIn);
}
}
}
@Override
public void close() throws IOException {
super.close();
datIn.close();
}
@Override
public Type getType() {
return Type.VAR_INTS;
}
@Override
protected Source loadDirectSource() throws IOException {
return values != null ? new FixedStraightBytesImpl.DirectFixedStraightSource(datIn.clone(), 8, Type.FIXED_INTS_64) : new PackedIntsSource(datIn.clone(), true);
}
}
static class PackedIntsSource extends Source {
private final long minValue;
private final long defaultValue;
private final PackedInts.Reader values;
public PackedIntsSource(IndexInput dataIn, boolean direct) throws IOException {
super(Type.VAR_INTS);
minValue = dataIn.readLong();
defaultValue = dataIn.readLong();
values = direct ? PackedInts.getDirectReader(dataIn) : PackedInts.getReader(dataIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
ref.grow(8);
DocValuesArraySource.copyLong(ref, getInt(docID));
return ref;
}
@Override
public long getInt(int docID) {
// TODO -- can we somehow avoid 2X method calls
// on each get? must push minValue down, and make
// PackedInts implement Ints.Source
assert docID >= 0;
final long value = values.get(docID);
return value == defaultValue ? 0 : minValue + value;
}
}
}

View File

@ -1,152 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts;
// Stores variable-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[] and both
// docs reference that single source
/**
* @lucene.experimental
*/
class VarDerefBytesImpl {
static final String CODEC_NAME_IDX = "VarDerefBytesIdx";
static final String CODEC_NAME_DAT = "VarDerefBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
/*
* TODO: if impls like this are merged we are bound to the amount of memory we
* can store into a BytesRefHash and therefore how much memory a ByteBlockPool
* can address. This is currently limited to 2GB. While we could extend that
* and use 64bit for addressing this still limits us to the existing main
* memory as all distinct bytes will be loaded up into main memory. We could
* move the byte[] writing to #finish(int) and store the bytes in sorted
* order and merge them in a streamed fashion.
*/
static class Writer extends DerefBytesWriterBase {
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, Type.BYTES_VAR_DEREF);
size = 0;
}
@Override
protected void checkSize(BytesRef bytes) {
// allow var bytes sizes
}
// Important that we get docCount, in case there were
// some last docs that we didn't see
@Override
public void finishInternal(int docCount) throws IOException {
fillDefault(docCount);
final int size = hash.size();
final long[] addresses = new long[size];
final IndexOutput datOut = getOrCreateDataOut();
int addr = 0;
final BytesRef bytesRef = new BytesRef();
for (int i = 0; i < size; i++) {
hash.get(i, bytesRef);
addresses[i] = addr;
addr += writePrefixLength(datOut, bytesRef) + bytesRef.length;
datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
}
final IndexOutput idxOut = getOrCreateIndexOut();
// write the max address to read directly on source load
idxOut.writeLong(addr);
writeIndex(idxOut, docCount, addresses[addresses.length-1], addresses, docToEntry);
}
}
public static class VarDerefReader extends BytesReaderBase {
private final long totalBytes;
VarDerefReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, Type.BYTES_VAR_DEREF);
totalBytes = idxIn.readLong();
}
@Override
protected Source loadSource() throws IOException {
return new VarDerefSource(cloneData(), cloneIndex(), totalBytes);
}
@Override
protected Source loadDirectSource()
throws IOException {
return new DirectVarDerefSource(cloneData(), cloneIndex(), getType());
}
}
final static class VarDerefSource extends BytesSourceBase {
private final PackedInts.Reader addresses;
public VarDerefSource(IndexInput datIn, IndexInput idxIn, long totalBytes)
throws IOException {
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), totalBytes,
Type.BYTES_VAR_DEREF);
addresses = PackedInts.getReader(idxIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
return data.fillSliceWithPrefix(bytesRef,
addresses.get(docID));
}
}
final static class DirectVarDerefSource extends DirectSource {
private final PackedInts.Reader index;
DirectVarDerefSource(IndexInput data, IndexInput index, Type type)
throws IOException {
super(data, type);
this.index = PackedInts.getDirectReader(index);
}
@Override
protected int position(int docID) throws IOException {
data.seek(baseOffset + index.get(docID));
final byte sizeByte = data.readByte();
if ((sizeByte & 128) == 0) {
// length is 1 byte
return sizeByte;
} else {
return ((sizeByte & 0x7f) << 8) | ((data.readByte() & 0xff));
}
}
}
}

View File

@ -1,258 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSortedSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.DerefBytesWriterBase;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.SortedBytesMergeUtils;
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.SortedBytesMergeUtils.IndexOutputBytesRefConsumer;
import org.apache.lucene.index.SortedBytesMergeUtils.MergeContext;
import org.apache.lucene.index.SortedBytesMergeUtils.SortedSourceSlice;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
// Stores variable-length byte[] by deref, ie when two docs
// have the same value, they store only 1 byte[] and both
// docs reference that single source
/**
* @lucene.experimental
*/
final class VarSortedBytesImpl {
static final String CODEC_NAME_IDX = "VarDerefBytesIdx";
static final String CODEC_NAME_DAT = "VarDerefBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
final static class Writer extends DerefBytesWriterBase {
private final Comparator<BytesRef> comp;
public Writer(Directory dir, String id, Comparator<BytesRef> comp,
Counter bytesUsed, IOContext context, float acceptableOverheadRatio) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, acceptableOverheadRatio, Type.BYTES_VAR_SORTED);
this.comp = comp;
size = 0;
}
@Override
public void merge(MergeState mergeState, DocValues[] docValues)
throws IOException {
boolean success = false;
try {
MergeContext ctx = SortedBytesMergeUtils.init(Type.BYTES_VAR_SORTED, docValues, comp, mergeState.segmentInfo.getDocCount());
final List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(mergeState.docBase, mergeState.docMaps, docValues, ctx);
IndexOutput datOut = getOrCreateDataOut();
ctx.offsets = new long[1];
final int maxOrd = SortedBytesMergeUtils.mergeRecords(ctx, new IndexOutputBytesRefConsumer(datOut), slices);
final long[] offsets = ctx.offsets;
maxBytes = offsets[maxOrd-1];
final IndexOutput idxOut = getOrCreateIndexOut();
idxOut.writeLong(maxBytes);
final PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, maxOrd+1,
PackedInts.bitsRequired(maxBytes), PackedInts.DEFAULT);
offsetWriter.add(0);
for (int i = 0; i < maxOrd; i++) {
offsetWriter.add(offsets[i]);
}
offsetWriter.finish();
final PackedInts.Writer ordsWriter = PackedInts.getWriter(idxOut, ctx.docToEntry.length,
PackedInts.bitsRequired(maxOrd-1), PackedInts.DEFAULT);
for (SortedSourceSlice slice : slices) {
slice.writeOrds(ordsWriter);
}
ordsWriter.finish();
success = true;
} finally {
releaseResources();
if (success) {
IOUtils.close(getIndexOut(), getDataOut());
} else {
IOUtils.closeWhileHandlingException(getIndexOut(), getDataOut());
}
}
}
@Override
protected void checkSize(BytesRef bytes) {
// allow var bytes sizes
}
// Important that we get docCount, in case there were
// some last docs that we didn't see
@Override
public void finishInternal(int docCount) throws IOException {
fillDefault(docCount);
final int count = hash.size();
final IndexOutput datOut = getOrCreateDataOut();
final IndexOutput idxOut = getOrCreateIndexOut();
long offset = 0;
final int[] index = new int[count];
final int[] sortedEntries = hash.sort(comp);
// total bytes of data
idxOut.writeLong(maxBytes);
PackedInts.Writer offsetWriter = PackedInts.getWriter(idxOut, count+1,
PackedInts.bitsRequired(maxBytes), PackedInts.DEFAULT);
// first dump bytes data, recording index & write offset as
// we go
final BytesRef spare = new BytesRef();
for (int i = 0; i < count; i++) {
final int e = sortedEntries[i];
offsetWriter.add(offset);
index[e] = i;
final BytesRef bytes = hash.get(e, spare);
// TODO: we could prefix code...
datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
offset += bytes.length;
}
// write sentinel
offsetWriter.add(offset);
offsetWriter.finish();
// write index
writeIndex(idxOut, docCount, count, index, docToEntry);
}
}
public static class Reader extends BytesReaderBase {
private final Comparator<BytesRef> comparator;
Reader(Directory dir, String id, int maxDoc,
IOContext context, Type type, Comparator<BytesRef> comparator)
throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, type);
this.comparator = comparator;
}
@Override
public org.apache.lucene.index.DocValues.Source loadSource()
throws IOException {
return new VarSortedSource(cloneData(), cloneIndex(), comparator);
}
@Override
protected Source loadDirectSource() throws IOException {
return new DirectSortedSource(cloneData(), cloneIndex(), comparator, getType());
}
}
private static final class VarSortedSource extends BytesSortedSourceBase {
private final int valueCount;
VarSortedSource(IndexInput datIn, IndexInput idxIn,
Comparator<BytesRef> comp) throws IOException {
super(datIn, idxIn, comp, idxIn.readLong(), Type.BYTES_VAR_SORTED, true);
valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
closeIndexInput();
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
final long offset = ordToOffsetIndex.get(ord);
final long nextOffset = ordToOffsetIndex.get(1 + ord);
data.fillSlice(bytesRef, offset, (int) (nextOffset - offset));
return bytesRef;
}
@Override
public int getValueCount() {
return valueCount;
}
}
private static final class DirectSortedSource extends SortedSource {
private final PackedInts.Reader docToOrdIndex;
private final PackedInts.Reader ordToOffsetIndex;
private final IndexInput datIn;
private final long basePointer;
private final int valueCount;
DirectSortedSource(IndexInput datIn, IndexInput idxIn,
Comparator<BytesRef> comparator, Type type) throws IOException {
super(type, comparator);
idxIn.readLong();
ordToOffsetIndex = PackedInts.getDirectReader(idxIn);
valueCount = ordToOffsetIndex.size()-1; // the last value here is just a dummy value to get the length of the last value
// advance this iterator to the end and clone the stream once it points to the docToOrdIndex header
ordToOffsetIndex.get(valueCount);
docToOrdIndex = PackedInts.getDirectReader(idxIn.clone()); // read the ords in to prevent too many random disk seeks
basePointer = datIn.getFilePointer();
this.datIn = datIn;
}
@Override
public int ord(int docID) {
return (int) docToOrdIndex.get(docID);
}
@Override
public boolean hasPackedDocToOrd() {
return true;
}
@Override
public PackedInts.Reader getDocToOrd() {
return docToOrdIndex;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
try {
final long offset = ordToOffsetIndex.get(ord);
// 1+ord is safe because we write a sentinel at the end
final long nextOffset = ordToOffsetIndex.get(1+ord);
datIn.seek(basePointer + offset);
final int length = (int) (nextOffset - offset);
bytesRef.offset = 0;
bytesRef.grow(length);
datIn.readBytes(bytesRef.bytes, 0, length);
bytesRef.length = length;
return bytesRef;
} catch (IOException ex) {
throw new IllegalStateException("failed", ex);
}
}
@Override
public int getValueCount() {
return valueCount;
}
}
}

View File

@ -1,297 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesReaderBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesSourceBase;
import org.apache.lucene.codecs.lucene40.values.Bytes.BytesWriterBase;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.StorableField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.ByteBlockPool;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.packed.PackedInts.ReaderIterator;
import org.apache.lucene.util.packed.PackedInts;
// Variable length byte[] per document, no sharing
/**
* @lucene.experimental
*/
class VarStraightBytesImpl {
static final String CODEC_NAME_IDX = "VarStraightBytesIdx";
static final String CODEC_NAME_DAT = "VarStraightBytesDat";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
static class Writer extends BytesWriterBase {
private long address;
// start at -1 if the first added value is > 0
private int lastDocID = -1;
private long[] docToAddress;
private final ByteBlockPool pool;
private IndexOutput datOut;
private boolean merge = false;
public Writer(Directory dir, String id, Counter bytesUsed, IOContext context) {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_CURRENT, bytesUsed, context, Type.BYTES_VAR_STRAIGHT);
pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
docToAddress = new long[1];
pool.nextBuffer(); // init
bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
}
// Fills up to but not including this docID
private void fill(final int docID, final long nextAddress) {
if (docID >= docToAddress.length) {
int oldSize = docToAddress.length;
docToAddress = ArrayUtil.grow(docToAddress, 1 + docID);
bytesUsed.addAndGet((docToAddress.length - oldSize)
* RamUsageEstimator.NUM_BYTES_INT);
}
for (int i = lastDocID + 1; i < docID; i++) {
docToAddress[i] = nextAddress;
}
}
@Override
public void add(int docID, StorableField value) throws IOException {
final BytesRef bytes = value.binaryValue();
assert bytes != null;
assert !merge;
if (bytes.length == 0) {
return; // default
}
fill(docID, address);
docToAddress[docID] = address;
pool.copy(bytes);
address += bytes.length;
lastDocID = docID;
}
@Override
protected void merge(DocValues readerIn, int docBase, int docCount, Bits liveDocs) throws IOException {
merge = true;
datOut = getOrCreateDataOut();
boolean success = false;
try {
if (liveDocs == null && readerIn instanceof VarStraightReader) {
// bulk merge since we don't have any deletes
VarStraightReader reader = (VarStraightReader) readerIn;
final int maxDocs = reader.maxDoc;
if (maxDocs == 0) {
return;
}
if (lastDocID+1 < docBase) {
fill(docBase, address);
lastDocID = docBase-1;
}
final long numDataBytes;
final IndexInput cloneIdx = reader.cloneIndex();
try {
numDataBytes = cloneIdx.readVLong();
final ReaderIterator iter = PackedInts.getReaderIterator(cloneIdx, PackedInts.DEFAULT_BUFFER_SIZE);
for (int i = 0; i < maxDocs; i++) {
long offset = iter.next();
++lastDocID;
if (lastDocID >= docToAddress.length) {
int oldSize = docToAddress.length;
docToAddress = ArrayUtil.grow(docToAddress, 1 + lastDocID);
bytesUsed.addAndGet((docToAddress.length - oldSize)
* RamUsageEstimator.NUM_BYTES_INT);
}
docToAddress[lastDocID] = address + offset;
}
address += numDataBytes; // this is the address after all addr pointers are updated
} finally {
IOUtils.close(cloneIdx);
}
final IndexInput cloneData = reader.cloneData();
try {
datOut.copyBytes(cloneData, numDataBytes);
} finally {
IOUtils.close(cloneData);
}
} else {
super.merge(readerIn, docBase, docCount, liveDocs);
}
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(datOut);
}
}
}
@Override
protected void mergeDoc(StoredField scratchField, Source source, int docID, int sourceDoc) throws IOException {
assert merge;
assert lastDocID < docID;
source.getBytes(sourceDoc, bytesRef);
if (bytesRef.length == 0) {
return; // default
}
fill(docID, address);
datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
docToAddress[docID] = address;
address += bytesRef.length;
lastDocID = docID;
}
@Override
public void finish(int docCount) throws IOException {
boolean success = false;
assert (!merge && datOut == null) || (merge && datOut != null);
final IndexOutput datOut = getOrCreateDataOut();
try {
if (!merge) {
// header is already written in getDataOut()
pool.writePool(datOut);
}
success = true;
} finally {
if (success) {
IOUtils.close(datOut);
} else {
IOUtils.closeWhileHandlingException(datOut);
}
pool.reset(false, false);
}
success = false;
final IndexOutput idxOut = getOrCreateIndexOut();
try {
if (lastDocID == -1) {
idxOut.writeVLong(0);
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount+1,
PackedInts.bitsRequired(0), PackedInts.DEFAULT);
// docCount+1 so we write sentinel
for (int i = 0; i < docCount+1; i++) {
w.add(0);
}
w.finish();
} else {
fill(docCount, address);
idxOut.writeVLong(address);
final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount+1,
PackedInts.bitsRequired(address), PackedInts.DEFAULT);
for (int i = 0; i < docCount; i++) {
w.add(docToAddress[i]);
}
// write sentinel
w.add(address);
w.finish();
}
success = true;
} finally {
bytesUsed.addAndGet(-(docToAddress.length)
* RamUsageEstimator.NUM_BYTES_INT);
docToAddress = null;
if (success) {
IOUtils.close(idxOut);
} else {
IOUtils.closeWhileHandlingException(idxOut);
}
}
}
public long ramBytesUsed() {
return bytesUsed.get();
}
@Override
public int getValueSize() {
return -1;
}
}
public static class VarStraightReader extends BytesReaderBase {
final int maxDoc;
VarStraightReader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
super(dir, id, CODEC_NAME_IDX, CODEC_NAME_DAT, VERSION_START, true, context, Type.BYTES_VAR_STRAIGHT);
this.maxDoc = maxDoc;
}
@Override
protected Source loadSource() throws IOException {
return new VarStraightSource(cloneData(), cloneIndex());
}
@Override
protected Source loadDirectSource()
throws IOException {
return new DirectVarStraightSource(cloneData(), cloneIndex(), getType());
}
}
private static final class VarStraightSource extends BytesSourceBase {
private final PackedInts.Reader addresses;
public VarStraightSource(IndexInput datIn, IndexInput idxIn) throws IOException {
super(datIn, idxIn, new PagedBytes(PAGED_BYTES_BITS), idxIn.readVLong(),
Type.BYTES_VAR_STRAIGHT);
addresses = PackedInts.getReader(idxIn);
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
final long address = addresses.get(docID);
return data.fillSlice(bytesRef, address,
(int) (addresses.get(docID + 1) - address));
}
}
public final static class DirectVarStraightSource extends DirectSource {
private final PackedInts.Reader index;
DirectVarStraightSource(IndexInput data, IndexInput index, Type type)
throws IOException {
super(data, type);
index.readVLong();
this.index = PackedInts.getDirectReader(index);
}
@Override
protected int position(int docID) throws IOException {
final long offset = index.get(docID);
data.seek(baseOffset + offset);
// Safe to do 1+docID because we write sentinel at the end:
final long nextOffset = index.get(1+docID);
return (int) (nextOffset - offset);
}
}
}

View File

@ -1,126 +0,0 @@
package org.apache.lucene.codecs.lucene40.values;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.Comparator;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.packed.PackedInts;
/**
* Abstract API for per-document stored primitive values of type <tt>byte[]</tt>
* , <tt>long</tt> or <tt>double</tt>. The API accepts a single value for each
* document. The underlying storage mechanism, file formats, data-structures and
* representations depend on the actual implementation.
* <p>
* Document IDs passed to this API must always be increasing unless stated
* otherwise.
* </p>
*
* @lucene.experimental
*/
abstract class Writer extends DocValuesConsumer {
protected final Counter bytesUsed;
protected Type type;
/**
* Creates a new {@link Writer}.
*
* @param bytesUsed
* bytes-usage tracking reference used by implementation to track
* internally allocated memory. All tracked bytes must be released
* once {@link #finish(int)} has been called.
*/
protected Writer(Counter bytesUsed, Type type) {
this.bytesUsed = bytesUsed;
this.type = type;
}
@Override
protected Type getType() {
return type;
}
/**
* Factory method to create a {@link Writer} instance for a given type. This
* method returns default implementations for each of the different types
* defined in the {@link Type} enumeration.
*
* @param type
* the {@link Type} to create the {@link Writer} for
* @param id
* the file name id used to create files within the writer.
* @param directory
* the {@link Directory} to create the files from.
* @param bytesUsed
* a byte-usage tracking reference
* @param acceptableOverheadRatio
* how to trade space for speed. This option is only applicable for
* docvalues of type {@link Type#BYTES_FIXED_SORTED} and
* {@link Type#BYTES_VAR_SORTED}.
* @return a new {@link Writer} instance for the given {@link Type}
* @see PackedInts#getReader(org.apache.lucene.store.DataInput)
*/
public static DocValuesConsumer create(Type type, String id, Directory directory,
Comparator<BytesRef> comp, Counter bytesUsed, IOContext context, float acceptableOverheadRatio) {
if (comp == null) {
comp = BytesRef.getUTF8SortedAsUnicodeComparator();
}
switch (type) {
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case VAR_INTS:
return Ints.getWriter(directory, id, bytesUsed, type, context);
case FLOAT_32:
return Floats.getWriter(directory, id, bytesUsed, context, type);
case FLOAT_64:
return Floats.getWriter(directory, id, bytesUsed, context, type);
case BYTES_FIXED_STRAIGHT:
return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, true, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_FIXED_DEREF:
return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, true, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_FIXED_SORTED:
return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, true, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_VAR_STRAIGHT:
return Bytes.getWriter(directory, id, Bytes.Mode.STRAIGHT, false, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_VAR_DEREF:
return Bytes.getWriter(directory, id, Bytes.Mode.DEREF, false, comp,
bytesUsed, context, acceptableOverheadRatio);
case BYTES_VAR_SORTED:
return Bytes.getWriter(directory, id, Bytes.Mode.SORTED, false, comp,
bytesUsed, context, acceptableOverheadRatio);
default:
throw new IllegalArgumentException("Unknown Values: " + type);
}
}
}

View File

@ -1,25 +0,0 @@
<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
<!--
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.
-->
<html>
<head>
<meta http-equiv="Content-Type" content="text/html; charset=iso-8859-1">
</head>
<body>
Default DocValues implementation for Lucene 4.0 indexes.
</body>
</html>

View File

@ -18,21 +18,17 @@ package org.apache.lucene.codecs.lucene41;
*/
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
import org.apache.lucene.codecs.SimpleNormsFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfoFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
@ -54,9 +50,7 @@ public class Lucene41Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new Lucene41StoredFieldsFormat();
private final TermVectorsFormat vectorsFormat = new Lucene40TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene40FieldInfosFormat();
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfoFormat infosFormat = new Lucene40SegmentInfoFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@ -89,11 +83,6 @@ public class Lucene41Codec extends Codec {
return vectorsFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
@ -108,11 +97,6 @@ public class Lucene41Codec extends Codec {
public final SegmentInfoFormat segmentInfoFormat() {
return infosFormat;
}
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {

View File

@ -1,4 +1,7 @@
package org.apache.lucene.codecs.sep;
package org.apache.lucene.document;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.BytesRef;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
@ -17,31 +20,16 @@ package org.apache.lucene.codecs.sep;
* limitations under the License.
*/
import org.apache.lucene.codecs.lucene40.values.DocValuesWriterBase;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.store.Directory;
/**
* Implementation of PerDocConsumer that uses separate files.
* @lucene.experimental
*/
public class SepDocValuesConsumer extends DocValuesWriterBase {
private final Directory directory;
public SepDocValuesConsumer(PerDocWriteState state) {
super(state);
this.directory = state.directory;
public class BinaryDocValuesField extends StoredField {
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(FieldInfo.DocValuesType.BINARY);
TYPE.freeze();
}
@Override
protected Directory getDirectory() {
return directory;
}
@Override
public void abort() {
// We don't have to remove files here: IndexFileDeleter
// will do so
public BinaryDocValuesField(String name, BytesRef value) {
super(name, TYPE);
fieldsData = value;
}
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
/**
* <p>
@ -32,7 +32,6 @@ import org.apache.lucene.index.DocValues;
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class ByteDocValuesField extends StoredField {
@ -42,7 +41,7 @@ public class ByteDocValuesField extends StoredField {
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FIXED_INTS_8);
TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
TYPE.freeze();
}

View File

@ -17,7 +17,6 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.util.BytesRef;
/**
@ -38,29 +37,9 @@ import org.apache.lucene.util.BytesRef;
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class DerefBytesDocValuesField extends StoredField {
// TODO: ideally indexer figures out var vs fixed on its own!?
/**
* Type for indirect bytes DocValues: all with the same length
*/
public static final FieldType TYPE_FIXED_LEN = new FieldType();
static {
TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_DEREF);
TYPE_FIXED_LEN.freeze();
}
/**
* Type for indirect bytes DocValues: can have variable lengths
*/
public static final FieldType TYPE_VAR_LEN = new FieldType();
static {
TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_DEREF);
TYPE_VAR_LEN.freeze();
}
@Deprecated
public class DerefBytesDocValuesField extends SortedBytesDocValuesField {
/**
* Create a new variable-length indirect DocValues field.
@ -75,8 +54,7 @@ public class DerefBytesDocValuesField extends StoredField {
* @throws IllegalArgumentException if the field name is null
*/
public DerefBytesDocValuesField(String name, BytesRef bytes) {
super(name, TYPE_VAR_LEN);
fieldsData = bytes;
super(name, bytes);
}
/**
@ -88,7 +66,6 @@ public class DerefBytesDocValuesField extends StoredField {
* @throws IllegalArgumentException if the field name is null
*/
public DerefBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
fieldsData = bytes;
super(name, bytes);
}
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
/**
* <p>
@ -32,7 +32,6 @@ import org.apache.lucene.index.DocValues;
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class DoubleDocValuesField extends StoredField {
@ -42,7 +41,7 @@ public class DoubleDocValuesField extends StoredField {
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FLOAT_64);
TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
TYPE.freeze();
}

View File

@ -29,7 +29,6 @@ import org.apache.lucene.document.FieldType.NumericType;
import org.apache.lucene.index.IndexWriter; // javadocs
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.IndexableFieldType;
import org.apache.lucene.index.Norm; // javadocs
import org.apache.lucene.index.StorableField;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.index.FieldInvertState; // javadocs

View File

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

View File

@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
/**
* <p>
@ -31,7 +31,6 @@ import org.apache.lucene.index.DocValues;
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
* @see DocValues
* */
public class FloatDocValuesField extends StoredField {
@ -41,7 +40,7 @@ public class FloatDocValuesField extends StoredField {
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FLOAT_32);
TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
TYPE.freeze();
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
/**
* <p>
@ -31,7 +31,6 @@ import org.apache.lucene.index.DocValues;
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
* @see DocValues
* */
public class IntDocValuesField extends StoredField {
@ -41,7 +40,7 @@ public class IntDocValuesField extends StoredField {
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FIXED_INTS_32);
TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
TYPE.freeze();
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
/**
* <p>
@ -31,7 +31,6 @@ import org.apache.lucene.index.DocValues;
* <p>
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
* @see DocValues
* */
public class LongDocValuesField extends StoredField {
@ -41,7 +40,7 @@ public class LongDocValuesField extends StoredField {
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FIXED_INTS_64);
TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
TYPE.freeze();
}

View File

@ -17,7 +17,6 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.AtomicReader; // javadocs
/**
@ -25,7 +24,7 @@ import org.apache.lucene.index.AtomicReader; // javadocs
* Field that stores a per-document <code>long</code> value
* for scoring, sorting or value retrieval. The values are
* encoded in the index an in RAM (when loaded via
* {@link AtomicReader#docValues})
* {@link AtomicReader#getNumericDocValues(String)})
* using packed ints. Here's an example usage:
*
* <pre class="prettyprint">
@ -36,19 +35,10 @@ import org.apache.lucene.index.AtomicReader; // javadocs
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class PackedLongDocValuesField extends StoredField {
/**
* Type for packed long DocValues.
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.VAR_INTS);
TYPE.freeze();
}
@Deprecated
public class PackedLongDocValuesField extends LongDocValuesField {
/**
* Creates a new DocValues field with the specified long value
@ -57,7 +47,6 @@ public class PackedLongDocValuesField extends StoredField {
* @throws IllegalArgumentException if the field name is null
*/
public PackedLongDocValuesField(String name, long value) {
super(name, TYPE);
fieldsData = Long.valueOf(value);
super(name, value);
}
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
/**
* <p>
@ -32,7 +32,6 @@ import org.apache.lucene.index.DocValues;
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class ShortDocValuesField extends StoredField {
@ -42,7 +41,7 @@ public class ShortDocValuesField extends StoredField {
*/
public static final FieldType TYPE = new FieldType();
static {
TYPE.setDocValueType(DocValues.Type.FIXED_INTS_16);
TYPE.setDocValueType(FieldInfo.DocValuesType.NUMERIC);
TYPE.freeze();
}

View File

@ -17,7 +17,7 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.util.BytesRef;
/**
@ -34,44 +34,28 @@ import org.apache.lucene.util.BytesRef;
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class SortedBytesDocValuesField extends StoredField {
// TODO: ideally indexer figures out var vs fixed on its own!?
/**
* Type for sorted bytes DocValues: all with the same length
*/
public static final FieldType TYPE_FIXED_LEN = new FieldType();
public static final FieldType TYPE = new FieldType();
static {
TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_SORTED);
TYPE_FIXED_LEN.freeze();
TYPE.setDocValueType(FieldInfo.DocValuesType.SORTED);
TYPE.freeze();
}
/**
* Type for sorted bytes DocValues: can have variable lengths
*/
public static final FieldType TYPE_VAR_LEN = new FieldType();
static {
TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_SORTED);
TYPE_VAR_LEN.freeze();
}
/**
* Create a new variable-length sorted DocValues field.
* <p>
* This calls
* {@link SortedBytesDocValuesField#SortedBytesDocValuesField(String, BytesRef, boolean)
* SortedBytesDocValuesField(name, bytes, false}, meaning by default
* it allows for values of different lengths. If your values are all
* the same length, use that constructor instead.
* Create a new sorted DocValues field.
* @param name field name
* @param bytes binary content
* @throws IllegalArgumentException if the field name is null
*/
public SortedBytesDocValuesField(String name, BytesRef bytes) {
this(name, bytes, false);
super(name, TYPE);
fieldsData = bytes;
}
/**
@ -81,8 +65,9 @@ public class SortedBytesDocValuesField extends StoredField {
* @param isFixedLength true if all values have the same length.
* @throws IllegalArgumentException if the field name is null
*/
@Deprecated
public SortedBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
super(name, TYPE);
fieldsData = bytes;
}
}

View File

@ -17,7 +17,6 @@ package org.apache.lucene.document;
* limitations under the License.
*/
import org.apache.lucene.index.DocValues;
import org.apache.lucene.util.BytesRef;
/**
@ -37,29 +36,11 @@ import org.apache.lucene.util.BytesRef;
* If you also need to store the value, you should add a
* separate {@link StoredField} instance.
*
* @see DocValues
* */
public class StraightBytesDocValuesField extends StoredField {
@Deprecated
public class StraightBytesDocValuesField extends BinaryDocValuesField {
// TODO: ideally indexer figures out var vs fixed on its own!?
/**
* Type for direct bytes DocValues: all with the same length
*/
public static final FieldType TYPE_FIXED_LEN = new FieldType();
static {
TYPE_FIXED_LEN.setDocValueType(DocValues.Type.BYTES_FIXED_STRAIGHT);
TYPE_FIXED_LEN.freeze();
}
/**
* Type for direct bytes DocValues: can have variable lengths
*/
public static final FieldType TYPE_VAR_LEN = new FieldType();
static {
TYPE_VAR_LEN.setDocValueType(DocValues.Type.BYTES_VAR_STRAIGHT);
TYPE_VAR_LEN.freeze();
}
/**
* Create a new variable-length direct DocValues field.
@ -74,8 +55,7 @@ public class StraightBytesDocValuesField extends StoredField {
* @throws IllegalArgumentException if the field name is null
*/
public StraightBytesDocValuesField(String name, BytesRef bytes) {
super(name, TYPE_VAR_LEN);
fieldsData = bytes;
super(name, bytes);
}
/**
@ -87,7 +67,6 @@ public class StraightBytesDocValuesField extends StoredField {
* @throws IllegalArgumentException if the field name is null
*/
public StraightBytesDocValuesField(String name, BytesRef bytes, boolean isFixedLength) {
super(name, isFixedLength ? TYPE_FIXED_LEN : TYPE_VAR_LEN);
fieldsData = bytes;
super(name, bytes);
}
}

View File

@ -156,13 +156,6 @@ public abstract class AtomicReader extends IndexReader {
}
return null;
}
/**
* Returns {@link DocValues} for this field.
* This method may return null if the reader has no per-document
* values stored.
*/
public abstract DocValues docValues(String field) throws IOException;
/** Returns {@link NumericDocValues} for this field, or
* null if no {@link NumericDocValues} were indexed for
@ -182,12 +175,6 @@ public abstract class AtomicReader extends IndexReader {
* used by a single thread. */
public abstract SortedDocValues getSortedDocValues(String field) throws IOException;
/**
* Returns {@link DocValues} for this field's normalization values.
* This method may return null if the field has no norms.
*/
public abstract DocValues normValues(String field) throws IOException;
// nocommit document that these are thread-private:
/** Returns {@link NumericDocValues} representing norms
* for this field, or null if no {@link NumericDocValues}

View File

@ -32,8 +32,6 @@ import org.apache.lucene.codecs.BlockTreeTermsReader;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat; // javadocs
import org.apache.lucene.document.FieldType; // for javadocs
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.store.Directory;
@ -678,12 +676,10 @@ public class CheckIndex {
}
for (FieldInfo info : reader.getFieldInfos()) {
if (info.hasNorms()) {
DocValues dv = reader.normValues(info.name);
checkDocValues(dv, info.name, info.getNormType(), reader.maxDoc());
checkSimpleNorms(info, reader, infoStream);
++status.totFields;
} else {
if (reader.normValues(info.name) != null) {
if (reader.simpleNormValues(info.name) != null) {
throw new RuntimeException("field: " + info.name + " should omit norms but has them!");
}
}
@ -1256,92 +1252,6 @@ public class CheckIndex {
return status;
}
/** Helper method to verify values (either docvalues or norms), also checking
* type and size against fieldinfos/segmentinfo
*/
private static void checkDocValues(DocValues docValues, String fieldName, DocValues.Type expectedType, int expectedDocs) throws IOException {
if (docValues == null) {
throw new RuntimeException("field: " + fieldName + " omits docvalues but should have them!");
}
DocValues.Type type = docValues.getType();
if (type != expectedType) {
throw new RuntimeException("field: " + fieldName + " has type: " + type + " but fieldInfos says:" + expectedType);
}
final Source values = docValues.getDirectSource();
int size = docValues.getValueSize();
for (int i = 0; i < expectedDocs; i++) {
switch (type) {
case BYTES_FIXED_SORTED:
case BYTES_VAR_SORTED:
case BYTES_FIXED_DEREF:
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_DEREF:
case BYTES_VAR_STRAIGHT:
BytesRef bytes = new BytesRef();
values.getBytes(i, bytes);
if (size != -1 && size != bytes.length) {
throw new RuntimeException("field: " + fieldName + " returned wrongly sized bytes, was: " + bytes.length + " should be: " + size);
}
break;
case FLOAT_32:
assert size == 4;
values.getFloat(i);
break;
case FLOAT_64:
assert size == 8;
values.getFloat(i);
break;
case VAR_INTS:
assert size == -1;
values.getInt(i);
break;
case FIXED_INTS_16:
assert size == 2;
values.getInt(i);
break;
case FIXED_INTS_32:
assert size == 4;
values.getInt(i);
break;
case FIXED_INTS_64:
assert size == 8;
values.getInt(i);
break;
case FIXED_INTS_8:
assert size == 1;
values.getInt(i);
break;
default:
throw new IllegalArgumentException("Field: " + fieldName
+ " - no such DocValues type: " + type);
}
}
if (type == DocValues.Type.BYTES_FIXED_SORTED || type == DocValues.Type.BYTES_VAR_SORTED) {
// check sorted bytes
SortedSource sortedValues = values.asSortedSource();
Comparator<BytesRef> comparator = sortedValues.getComparator();
int lastOrd = -1;
BytesRef lastBytes = new BytesRef();
for (int i = 0; i < expectedDocs; i++) {
int ord = sortedValues.ord(i);
if (ord < 0 || ord > expectedDocs) {
throw new RuntimeException("field: " + fieldName + " ord is out of bounds: " + ord);
}
BytesRef bytes = new BytesRef();
sortedValues.getByOrd(ord, bytes);
if (lastOrd != -1) {
int ordComp = Integer.signum(new Integer(ord).compareTo(new Integer(lastOrd)));
int bytesComp = Integer.signum(comparator.compare(bytes, lastBytes));
if (ordComp != bytesComp) {
throw new RuntimeException("field: " + fieldName + " ord comparison is wrong: " + ordComp + " comparator claims: " + bytesComp);
}
}
lastOrd = ord;
lastBytes = bytes;
}
}
}
public static Status.DocValuesStatus testDocValues(AtomicReader reader,
PrintStream infoStream) {
final Status.DocValuesStatus status = new Status.DocValuesStatus();
@ -1352,11 +1262,11 @@ public class CheckIndex {
for (FieldInfo fieldInfo : reader.getFieldInfos()) {
if (fieldInfo.hasDocValues()) {
status.totalValueFields++;
final DocValues docValues = reader.docValues(fieldInfo.name);
checkDocValues(docValues, fieldInfo.name, fieldInfo.getDocValuesType(), reader.maxDoc());
checkSimpleDocValues(fieldInfo, reader, infoStream);
} else {
if (reader.docValues(fieldInfo.name) != null) {
if (reader.getBinaryDocValues(fieldInfo.name) != null ||
reader.getNumericDocValues(fieldInfo.name) != null ||
reader.getSortedDocValues(fieldInfo.name) != null) {
throw new RuntimeException("field: " + fieldInfo.name + " has docvalues but should omit them!");
}
}
@ -1440,23 +1350,13 @@ public class CheckIndex {
msg(infoStream, " field: " + fi.name + ": " + atts);
}
switch(fi.getDocValuesType()) {
case BYTES_FIXED_SORTED:
case BYTES_VAR_SORTED:
case BYTES_FIXED_DEREF:
case BYTES_VAR_DEREF:
case SORTED:
checkSortedDocValues(fi.name, reader, reader.getSortedDocValues(fi.name));
break;
case BYTES_FIXED_STRAIGHT:
case BYTES_VAR_STRAIGHT:
case BINARY:
checkBinaryDocValues(fi.name, reader, reader.getBinaryDocValues(fi.name));
break;
case FLOAT_32:
case FLOAT_64:
case VAR_INTS:
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case NUMERIC:
checkNumericDocValues(fi.name, reader, reader.getNumericDocValues(fi.name));
break;
default:
@ -1467,13 +1367,7 @@ public class CheckIndex {
// nocommit
public static void checkSimpleNorms(FieldInfo fi, AtomicReader reader, PrintStream infoStream) throws IOException {
switch(fi.getNormType()) {
case FLOAT_32:
case FLOAT_64:
case VAR_INTS:
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
case FIXED_INTS_8:
case NUMERIC:
checkNumericDocValues(fi.name, reader, reader.simpleNormValues(fi.name));
break;
default:

View File

@ -25,16 +25,10 @@ import java.util.HashSet;
import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.FieldInfosWriter;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.index.DocumentsWriterPerThread.DocState;
import org.apache.lucene.index.TypePromoter.TypeCompatibility;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.IOUtils;
/**
* This is a DocConsumer that gathers all fields under the
@ -86,14 +80,6 @@ final class DocFieldProcessor extends DocConsumer {
storedConsumer.flush(state);
consumer.flush(childFields, state);
for (DocValuesConsumerHolder consumer : docValues.values()) {
consumer.docValuesConsumer.finish(state.segmentInfo.getDocCount());
}
// close perDocConsumer during flush to ensure all files are flushed due to PerCodec CFS
// nocommit
IOUtils.close(perDocConsumer);
// Important to save after asking consumer to flush so
// consumer can alter the FieldInfo* if necessary. EG,
// FreqProxTermsWriter does this with
@ -119,8 +105,6 @@ final class DocFieldProcessor extends DocConsumer {
field = next;
}
}
IOUtils.closeWhileHandlingException(perDocConsumer);
// TODO add abort to PerDocConsumer!
try {
storedConsumer.abort();
@ -138,16 +122,6 @@ final class DocFieldProcessor extends DocConsumer {
}
}
try {
if (perDocConsumer != null) {
perDocConsumer.abort();
}
} catch (Throwable t) {
if (th == null) {
th = t;
}
}
// If any errors occured, throw it.
if (th != null) {
if (th instanceof RuntimeException) throw (RuntimeException) th;
@ -177,8 +151,6 @@ final class DocFieldProcessor extends DocConsumer {
fieldHash = new DocFieldProcessorPerField[2];
hashMask = 1;
totalFieldCount = 0;
perDocConsumer = null;
docValues.clear();
}
private void rehash() {
@ -233,30 +205,6 @@ final class DocFieldProcessor extends DocConsumer {
IndexableFieldType ft = field.fieldType();
FieldInfo fieldInfo = fieldInfos.addOrUpdate(fieldName, ft);
storedConsumer.addField(docState.docID, field, fieldInfo);
final DocValues.Type dvType = ft.docValueType();
if (dvType != null) {
DocValuesConsumerHolder docValuesConsumer = docValuesConsumer(dvType,
docState, fieldInfo);
DocValuesConsumer consumer = docValuesConsumer.docValuesConsumer;
if (docValuesConsumer.compatibility == null) {
consumer.add(docState.docID, field);
docValuesConsumer.compatibility = new TypeCompatibility(dvType,
consumer.getValueSize());
} else if (docValuesConsumer.compatibility.isCompatible(dvType,
TypePromoter.getValueSize(dvType, field.binaryValue()))) {
consumer.add(docState.docID, field);
} else {
docValuesConsumer.compatibility.isCompatible(dvType,
TypePromoter.getValueSize(dvType, field.binaryValue()));
TypeCompatibility compatibility = docValuesConsumer.compatibility;
throw new IllegalArgumentException("Incompatible DocValues type: "
+ dvType.name() + " size: "
+ TypePromoter.getValueSize(dvType, field.binaryValue())
+ " expected: " + " type: " + compatibility.getBaseType()
+ " size: " + compatibility.getBaseSize());
}
}
}
// If we are writing vectors then we must visit
@ -344,45 +292,4 @@ final class DocFieldProcessor extends DocConsumer {
consumer.finishDocument();
}
}
private static class DocValuesConsumerHolder {
// Only used to enforce that same DV field name is never
// added more than once per doc:
int docID;
final DocValuesConsumer docValuesConsumer;
TypeCompatibility compatibility;
public DocValuesConsumerHolder(DocValuesConsumer docValuesConsumer) {
this.docValuesConsumer = docValuesConsumer;
}
}
final private Map<String, DocValuesConsumerHolder> docValues = new HashMap<String, DocValuesConsumerHolder>();
private PerDocConsumer perDocConsumer;
DocValuesConsumerHolder docValuesConsumer(DocValues.Type valueType, DocState docState, FieldInfo fieldInfo)
throws IOException {
DocValuesConsumerHolder docValuesConsumerAndDocID = docValues.get(fieldInfo.name);
if (docValuesConsumerAndDocID != null) {
if (docState.docID == docValuesConsumerAndDocID.docID) {
throw new IllegalArgumentException("DocValuesField \"" + fieldInfo.name + "\" appears more than once in this document (only one value is allowed, per field)");
}
assert docValuesConsumerAndDocID.docID < docState.docID;
docValuesConsumerAndDocID.docID = docState.docID;
return docValuesConsumerAndDocID;
}
if (perDocConsumer == null) {
PerDocWriteState perDocWriteState = docState.docWriter.newPerDocWriteState("");
perDocConsumer = docState.docWriter.codec.docValuesFormat().docsConsumer(perDocWriteState);
if (perDocConsumer == null) {
throw new IllegalStateException("codec=" + docState.docWriter.codec + " does not support docValues: from docValuesFormat().docsConsumer(...) returned null; field=" + fieldInfo.name);
}
}
DocValuesConsumer docValuesConsumer = perDocConsumer.addValuesField(valueType, fieldInfo);
docValuesConsumerAndDocID = new DocValuesConsumerHolder(docValuesConsumer);
docValuesConsumerAndDocID.docID = docState.docID;
docValues.put(fieldInfo.name, docValuesConsumerAndDocID);
return docValuesConsumerAndDocID;
}
}

View File

@ -1,864 +0,0 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.Comparator;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.document.ByteDocValuesField; // javadocs
import org.apache.lucene.document.DerefBytesDocValuesField; // javadocs
import org.apache.lucene.document.DoubleDocValuesField; // javadocs
import org.apache.lucene.document.Field; // javadocs
import org.apache.lucene.document.FloatDocValuesField; // javadocs
import org.apache.lucene.document.IntDocValuesField; // javadocs
import org.apache.lucene.document.LongDocValuesField; // javadocs
import org.apache.lucene.document.PackedLongDocValuesField; // javadocs
import org.apache.lucene.document.ShortDocValuesField; // javadocs
import org.apache.lucene.document.SortedBytesDocValuesField; // javadocs
import org.apache.lucene.document.StraightBytesDocValuesField; // javadocs
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.packed.PackedInts;
/**
* {@link DocValues} provides a dense per-document typed storage for fast
* value access based on the lucene internal document id. {@link DocValues}
* exposes two distinct APIs:
* <ul>
* <li>via {@link #getSource()} providing RAM resident random access</li>
* <li>via {@link #getDirectSource()} providing on disk random access</li>
* </ul> {@link DocValues} are exposed via
* {@link AtomicReader#docValues(String)} on a per-segment basis. For best
* performance {@link DocValues} should be consumed per-segment just like
* IndexReader.
* <p>
* {@link DocValues} are fully integrated into the {@link DocValuesFormat} API.
* <p>
* NOTE: DocValues is a strongly typed per-field API. Type changes within an
* indexing session can result in exceptions if the type has changed in a way that
* the previously give type for a field can't promote the value without losing
* information. For instance a field initially indexed with {@link Type#FIXED_INTS_32}
* can promote a value with {@link Type#FIXED_INTS_8} but can't promote
* {@link Type#FIXED_INTS_64}. During segment merging type-promotion exceptions are suppressed.
* Fields will be promoted to their common denominator or automatically transformed
* into a 3rd type like {@link Type#BYTES_VAR_STRAIGHT} to prevent data loss and merge exceptions.
* This behavior is considered <i>best-effort</i> might change in future releases.
* </p>
* <p>
* DocValues are exposed via the {@link Field} API with type safe
* specializations for each type variant:
* <ul>
* <li> {@link ByteDocValuesField} - for adding byte values to the index</li>
* <li> {@link ShortDocValuesField} - for adding short values to the index</li>
* <li> {@link IntDocValuesField} - for adding int values to the index</li>
* <li> {@link LongDocValuesField} - for adding long values to the index</li>
* <li> {@link FloatDocValuesField} - for adding float values to the index</li>
* <li> {@link DoubleDocValuesField} - for adding double values to the index</li>
* <li> {@link PackedLongDocValuesField} - for adding packed long values to the
* index</li>
* <li> {@link SortedBytesDocValuesField} - for adding sorted {@link BytesRef}
* values to the index</li>
* <li> {@link StraightBytesDocValuesField} - for adding straight
* {@link BytesRef} values to the index</li>
* <li> {@link DerefBytesDocValuesField} - for adding deref {@link BytesRef}
* values to the index</li>
* </ul>
* See {@link Type} for limitations of each type variant.
* <p>
* <p>
*
* @see DocValuesFormat#docsConsumer(org.apache.lucene.index.PerDocWriteState)
*
* @lucene.experimental
*/
public abstract class DocValues implements Closeable {
/** Zero length DocValues array. */
public static final DocValues[] EMPTY_ARRAY = new DocValues[0];
private volatile SourceCache cache = new SourceCache.DirectSourceCache();
private final Object cacheLock = new Object();
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected DocValues() {
}
/**
* Loads a new {@link Source} instance for this {@link DocValues} field
* instance. Source instances returned from this method are not cached. It is
* the callers responsibility to maintain the instance and release its
* resources once the source is not needed anymore.
* <p>
* For managed {@link Source} instances see {@link #getSource()}.
*
* @see #getSource()
* @see #setCache(SourceCache)
*/
protected abstract Source loadSource() throws IOException;
/**
* Returns a {@link Source} instance through the current {@link SourceCache}.
* Iff no {@link Source} has been loaded into the cache so far the source will
* be loaded through {@link #loadSource()} and passed to the {@link SourceCache}.
* The caller of this method should not close the obtained {@link Source}
* instance unless it is not needed for the rest of its life time.
* <p>
* {@link Source} instances obtained from this method are closed / released
* from the cache once this {@link DocValues} instance is closed by the
* {@link IndexReader}, {@link Fields} or the
* {@link DocValues} was created from.
*/
public Source getSource() throws IOException {
return cache.load(this);
}
/**
* Returns a disk resident {@link Source} instance through the current
* {@link SourceCache}. Direct Sources are cached per thread in the
* {@link SourceCache}. The obtained instance should not be shared with other
* threads.
*/
public Source getDirectSource() throws IOException {
return this.cache.loadDirect(this);
}
/**
* Loads a new {@link Source direct source} instance from this {@link DocValues} field
* instance. Source instances returned from this method are not cached. It is
* the callers responsibility to maintain the instance and release its
* resources once the source is not needed anymore.
* <p>
* For managed {@link Source direct source} instances see {@link #getDirectSource()}.
*
* @see #getDirectSource()
* @see #setCache(SourceCache)
*/
protected abstract Source loadDirectSource() throws IOException;
/**
* Returns the {@link Type} of this {@link DocValues} instance
*/
public abstract Type getType();
/**
* Closes this {@link DocValues} instance. This method should only be called
* by the creator of this {@link DocValues} instance. API users should not
* close {@link DocValues} instances.
*/
@Override
public void close() throws IOException {
cache.close(this);
}
/**
* Returns the size per value in bytes or <code>-1</code> iff size per value
* is variable.
*
* @return the size per value in bytes or <code>-1</code> iff size per value
* is variable.
*/
public int getValueSize() {
return -1;
}
/**
* Sets the {@link SourceCache} used by this {@link DocValues} instance. This
* method should be called before {@link #loadSource()} is called. All {@link Source} instances in the currently used cache will be closed
* before the new cache is installed.
* <p>
* Note: All instances previously obtained from {@link #loadSource()} will be lost.
*
* @throws IllegalArgumentException
* if the given cache is <code>null</code>
*
*/
public void setCache(SourceCache cache) {
if (cache == null)
throw new IllegalArgumentException("cache must not be null");
synchronized (cacheLock) {
SourceCache toClose = this.cache;
this.cache = cache;
toClose.close(this);
}
}
/**
* Returns the currently used cache instance;
* @see #setCache(SourceCache)
*/
// for tests
SourceCache getCache() {
return cache;
}
/**
* Source of per document values like long, double or {@link BytesRef}
* depending on the {@link DocValues} fields {@link Type}. Source
* implementations provide random access semantics similar to array lookups
* <p>
* @see DocValues#getSource()
* @see DocValues#getDirectSource()
*/
public static abstract class Source {
/** {@link Type} of this {@code Source}. */
protected final Type type;
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected Source(Type type) {
this.type = type;
}
/**
* Returns a <tt>long</tt> for the given document id or throws an
* {@link UnsupportedOperationException} if this source doesn't support
* <tt>long</tt> values.
*
* @throws UnsupportedOperationException
* if this source doesn't support <tt>long</tt> values.
*/
public long getInt(int docID) {
throw new UnsupportedOperationException("ints are not supported");
}
/**
* Returns a <tt>double</tt> for the given document id or throws an
* {@link UnsupportedOperationException} if this source doesn't support
* <tt>double</tt> values.
*
* @throws UnsupportedOperationException
* if this source doesn't support <tt>double</tt> values.
*/
public double getFloat(int docID) {
throw new UnsupportedOperationException("floats are not supported");
}
/**
* Returns a {@link BytesRef} for the given document id or throws an
* {@link UnsupportedOperationException} if this source doesn't support
* <tt>byte[]</tt> values.
*
* @throws UnsupportedOperationException
* if this source doesn't support <tt>byte[]</tt> values.
*/
public BytesRef getBytes(int docID, BytesRef ref) {
throw new UnsupportedOperationException("bytes are not supported");
}
/**
* Returns the {@link Type} of this source.
*
* @return the {@link Type} of this source.
*/
public Type getType() {
return type;
}
/**
* Returns <code>true</code> iff this {@link Source} exposes an array via
* {@link #getArray()} otherwise <code>false</code>.
*
* @return <code>true</code> iff this {@link Source} exposes an array via
* {@link #getArray()} otherwise <code>false</code>.
*/
public boolean hasArray() {
return false;
}
/**
* Returns the internal array representation iff this {@link Source} uses an
* array as its inner representation, otherwise <code>UOE</code>.
*/
public Object getArray() {
throw new UnsupportedOperationException("getArray is not supported");
}
/**
* If this {@link Source} is sorted this method will return an instance of
* {@link SortedSource} otherwise <code>UOE</code>
*/
public SortedSource asSortedSource() {
throw new UnsupportedOperationException("asSortedSource is not supported");
}
}
/**
* A sorted variant of {@link Source} for <tt>byte[]</tt> values per document.
* <p>
*/
public static abstract class SortedSource extends Source {
private final Comparator<BytesRef> comparator;
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected SortedSource(Type type, Comparator<BytesRef> comparator) {
super(type);
this.comparator = comparator;
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
final int ord = ord(docID);
if (ord < 0) {
// Negative ord means doc was missing?
bytesRef.length = 0;
} else {
getByOrd(ord, bytesRef);
}
return bytesRef;
}
/**
* Returns ord for specified docID. Ord is dense, ie, starts at 0, then increments by 1
* for the next (as defined by {@link Comparator} value.
*/
public abstract int ord(int docID);
/** Returns value for specified ord. */
public abstract BytesRef getByOrd(int ord, BytesRef result);
/** Return true if it's safe to call {@link
* #getDocToOrd}. */
public boolean hasPackedDocToOrd() {
return false;
}
/**
* Returns the PackedInts.Reader impl that maps document to ord.
*/
// nocommit make non-abstract returning null?
public abstract PackedInts.Reader getDocToOrd();
/**
* Returns the comparator used to order the BytesRefs.
*/
public Comparator<BytesRef> getComparator() {
return comparator;
}
/**
* Lookup ord by value.
*
* @param value
* the value to look up
* @param spare
* a spare {@link BytesRef} instance used to compare internal
* values to the given value. Must not be <code>null</code>
* @return the given values ordinal if found or otherwise
* <code>(-(ord)-1)</code>, defined as the ordinal of the first
* element that is greater than the given value (the insertion
* point). This guarantees that the return value will always be
* &gt;= 0 if the given value is found.
*/
public int getOrdByValue(BytesRef value, BytesRef spare) {
return binarySearch(value, spare, 0, getValueCount() - 1);
}
private int binarySearch(BytesRef b, BytesRef bytesRef, int low,
int high) {
int mid = 0;
while (low <= high) {
mid = (low + high) >>> 1;
getByOrd(mid, bytesRef);
final int cmp = comparator.compare(bytesRef, b);
if (cmp < 0) {
low = mid + 1;
} else if (cmp > 0) {
high = mid - 1;
} else {
return mid;
}
}
assert comparator.compare(bytesRef, b) != 0;
return -(low + 1);
}
@Override
public SortedSource asSortedSource() {
return this;
}
/**
* Returns the number of unique values in this sorted source
*/
public abstract int getValueCount();
}
/** Returns a Source that always returns default (missing)
* values for all documents. */
public static Source getDefaultSource(final Type type) {
return new Source(type) {
@Override
public long getInt(int docID) {
return 0;
}
@Override
public double getFloat(int docID) {
return 0.0;
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
ref.length = 0;
return ref;
}
};
}
/** Returns a SortedSource that always returns default (missing)
* values for all documents. */
public static SortedSource getDefaultSortedSource(final Type type, final int size) {
final PackedInts.Reader docToOrd = new PackedInts.Reader() {
@Override
public long get(int index) {
return 0;
}
@Override
public int getBitsPerValue() {
return 0;
}
@Override
public int size() {
return size;
}
@Override
public boolean hasArray() {
return false;
}
@Override
public Object getArray() {
return null;
}
@Override
public int get(int index, long[] arr, int off, int len) {
len = Math.min(len, size() - index);
Arrays.fill(arr, off, off+len, 0);
return len;
}
@Override
public long ramBytesUsed() {
return 0;
}
};
return new SortedSource(type, BytesRef.getUTF8SortedAsUnicodeComparator()) {
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
ref.length = 0;
return ref;
}
@Override
public int ord(int docID) {
return 0;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
assert ord == 0;
bytesRef.length = 0;
return bytesRef;
}
@Override
public boolean hasPackedDocToOrd() {
return true;
}
@Override
public PackedInts.Reader getDocToOrd() {
return docToOrd;
}
@Override
public int getOrdByValue(BytesRef value, BytesRef spare) {
if (value.length == 0) {
return 0;
} else {
return -1;
}
}
@Override
public int getValueCount() {
return 1;
}
};
}
// nocommit nuke
public static boolean isSortedBytes(Type type) {
switch(type) {
case BYTES_VAR_SORTED:
case BYTES_FIXED_SORTED:
return true;
default:
return false;
}
}
// nocommit nuke
public static boolean isBytes(Type type) {
switch(type) {
case BYTES_VAR_DEREF:
case BYTES_FIXED_DEREF:
case BYTES_VAR_STRAIGHT:
case BYTES_FIXED_STRAIGHT:
return true;
default:
return false;
}
}
// nocommit nuke
public static boolean isNumber(Type type) {
switch(type) {
case VAR_INTS:
case FIXED_INTS_8:
case FIXED_INTS_16:
case FIXED_INTS_32:
case FIXED_INTS_64:
return true;
default:
return false;
}
}
public static boolean isFloat(Type type) {
switch(type) {
case FLOAT_64:
case FLOAT_32:
return true;
default:
return false;
}
}
/**
* <code>Type</code> specifies the {@link DocValues} type for a
* certain field. A <code>Type</code> only defines the data type for a field
* while the actual implementation used to encode and decode the values depends
* on the the {@link DocValuesFormat#docsConsumer} and {@link DocValuesFormat#docsProducer} methods.
*
* @lucene.experimental
*/
public static enum Type {
/**
* A variable bit signed integer value. By default this type uses
* {@link PackedInts} to compress the values, as an offset
* from the minimum value, as long as the value range
* fits into 2<sup>63</sup>-1. Otherwise,
* the default implementation falls back to fixed size 64bit
* integers ({@link #FIXED_INTS_64}).
* <p>
* NOTE: this type uses <tt>0</tt> as the default value without any
* distinction between provided <tt>0</tt> values during indexing. All
* documents without an explicit value will use <tt>0</tt> instead.
* Custom default values must be assigned explicitly.
* </p>
*/
VAR_INTS,
/**
* A 8 bit signed integer value. {@link Source} instances of
* this type return a <tt>byte</tt> array from {@link Source#getArray()}
* <p>
* NOTE: this type uses <tt>0</tt> as the default value without any
* distinction between provided <tt>0</tt> values during indexing. All
* documents without an explicit value will use <tt>0</tt> instead.
* Custom default values must be assigned explicitly.
* </p>
*/
FIXED_INTS_8,
/**
* A 16 bit signed integer value. {@link Source} instances of
* this type return a <tt>short</tt> array from {@link Source#getArray()}
* <p>
* NOTE: this type uses <tt>0</tt> as the default value without any
* distinction between provided <tt>0</tt> values during indexing. All
* documents without an explicit value will use <tt>0</tt> instead.
* Custom default values must be assigned explicitly.
* </p>
*/
FIXED_INTS_16,
/**
* A 32 bit signed integer value. {@link Source} instances of
* this type return a <tt>int</tt> array from {@link Source#getArray()}
* <p>
* NOTE: this type uses <tt>0</tt> as the default value without any
* distinction between provided <tt>0</tt> values during indexing. All
* documents without an explicit value will use <tt>0</tt> instead.
* Custom default values must be assigned explicitly.
* </p>
*/
FIXED_INTS_32,
/**
* A 64 bit signed integer value. {@link Source} instances of
* this type return a <tt>long</tt> array from {@link Source#getArray()}
* <p>
* NOTE: this type uses <tt>0</tt> as the default value without any
* distinction between provided <tt>0</tt> values during indexing. All
* documents without an explicit value will use <tt>0</tt> instead.
* Custom default values must be assigned explicitly.
* </p>
*/
FIXED_INTS_64,
/**
* A 32 bit floating point value. By default there is no compression
* applied. To fit custom float values into less than 32bit either a custom
* implementation is needed or values must be encoded into a
* {@link #BYTES_FIXED_STRAIGHT} type. {@link Source} instances of
* this type return a <tt>float</tt> array from {@link Source#getArray()}
* <p>
* NOTE: this type uses <tt>0.0f</tt> as the default value without any
* distinction between provided <tt>0.0f</tt> values during indexing. All
* documents without an explicit value will use <tt>0.0f</tt> instead.
* Custom default values must be assigned explicitly.
* </p>
*/
FLOAT_32,
/**
*
* A 64 bit floating point value. By default there is no compression
* applied. To fit custom float values into less than 64bit either a custom
* implementation is needed or values must be encoded into a
* {@link #BYTES_FIXED_STRAIGHT} type. {@link Source} instances of
* this type return a <tt>double</tt> array from {@link Source#getArray()}
* <p>
* NOTE: this type uses <tt>0.0d</tt> as the default value without any
* distinction between provided <tt>0.0d</tt> values during indexing. All
* documents without an explicit value will use <tt>0.0d</tt> instead.
* Custom default values must be assigned explicitly.
* </p>
*/
FLOAT_64,
// TODO(simonw): -- shouldn't lucene decide/detect straight vs
// deref, as well fixed vs var?
/**
* A fixed length straight byte[]. All values added to
* such a field must be of the same length. All bytes are stored sequentially
* for fast offset access.
* <p>
* NOTE: this type uses <tt>0 byte</tt> filled byte[] based on the length of the first seen
* value as the default value without any distinction between explicitly
* provided values during indexing. All documents without an explicit value
* will use the default instead.Custom default values must be assigned explicitly.
* </p>
*/
BYTES_FIXED_STRAIGHT,
/**
* A fixed length dereferenced byte[] variant. Fields with
* this type only store distinct byte values and store an additional offset
* pointer per document to dereference the shared byte[].
* Use this type if your documents may share the same byte[].
* <p>
* NOTE: Fields of this type will not store values for documents without an
* explicitly provided value. If a documents value is accessed while no
* explicit value is stored the returned {@link BytesRef} will be a 0-length
* reference. Custom default values must be assigned explicitly.
* </p>
*/
BYTES_FIXED_DEREF,
/**
* Variable length straight stored byte[] variant. All bytes are
* stored sequentially for compactness. Usage of this type via the
* disk-resident API might yield performance degradation since no additional
* index is used to advance by more than one document value at a time.
* <p>
* NOTE: Fields of this type will not store values for documents without an
* explicitly provided value. If a documents value is accessed while no
* explicit value is stored the returned {@link BytesRef} will be a 0-length
* byte[] reference. Custom default values must be assigned explicitly.
* </p>
*/
BYTES_VAR_STRAIGHT,
/**
* A variable length dereferenced byte[]. Just like
* {@link #BYTES_FIXED_DEREF}, but allowing each
* document's value to be a different length.
* <p>
* NOTE: Fields of this type will not store values for documents without an
* explicitly provided value. If a documents value is accessed while no
* explicit value is stored the returned {@link BytesRef} will be a 0-length
* reference. Custom default values must be assigned explicitly.
* </p>
*/
BYTES_VAR_DEREF,
/**
* A variable length pre-sorted byte[] variant. Just like
* {@link #BYTES_FIXED_SORTED}, but allowing each
* document's value to be a different length.
* <p>
* NOTE: Fields of this type will not store values for documents without an
* explicitly provided value. If a documents value is accessed while no
* explicit value is stored the returned {@link BytesRef} will be a 0-length
* reference.Custom default values must be assigned explicitly.
* </p>
*
* @see SortedSource
*/
BYTES_VAR_SORTED,
/**
* A fixed length pre-sorted byte[] variant. Fields with this type only
* store distinct byte values and store an additional offset pointer per
* document to dereference the shared byte[]. The stored
* byte[] is presorted, by default by unsigned byte order,
* and allows access via document id, ordinal and by-value.
* Use this type if your documents may share the same byte[].
* <p>
* NOTE: Fields of this type will not store values for documents without an
* explicitly provided value. If a documents value is accessed while no
* explicit value is stored the returned {@link BytesRef} will be a 0-length
* reference. Custom default values must be assigned
* explicitly.
* </p>
*
* @see SortedSource
*/
BYTES_FIXED_SORTED
}
/**
* Abstract base class for {@link DocValues} {@link Source} cache.
* <p>
* {@link Source} instances loaded via {@link DocValues#loadSource()} are entirely memory resident
* and need to be maintained by the caller. Each call to
* {@link DocValues#loadSource()} will cause an entire reload of
* the underlying data. Source instances obtained from
* {@link DocValues#getSource()} and {@link DocValues#getSource()}
* respectively are maintained by a {@link SourceCache} that is closed (
* {@link #close(DocValues)}) once the {@link IndexReader} that created the
* {@link DocValues} instance is closed.
* <p>
* Unless {@link Source} instances are managed by another entity it is
* recommended to use the cached variants to obtain a source instance.
* <p>
* Implementation of this API must be thread-safe.
*
* @see DocValues#setCache(SourceCache)
* @see DocValues#getSource()
*
* @lucene.experimental
*/
public static abstract class SourceCache {
/** Sole constructor. (For invocation by subclass
* constructors, typically implicit.) */
protected SourceCache() {
}
/**
* Atomically loads a {@link Source} into the cache from the given
* {@link DocValues} and returns it iff no other {@link Source} has already
* been cached. Otherwise the cached source is returned.
* <p>
* This method will not return <code>null</code>
*/
public abstract Source load(DocValues values) throws IOException;
/**
* Atomically loads a {@link Source direct source} into the per-thread cache from the given
* {@link DocValues} and returns it iff no other {@link Source direct source} has already
* been cached. Otherwise the cached source is returned.
* <p>
* This method will not return <code>null</code>
*/
public abstract Source loadDirect(DocValues values) throws IOException;
/**
* Atomically invalidates the cached {@link Source}
* instances if any and empties the cache.
*/
public abstract void invalidate(DocValues values);
/**
* Atomically closes the cache and frees all resources.
*/
public synchronized void close(DocValues values) {
invalidate(values);
}
/**
* Simple per {@link DocValues} instance cache implementation that holds a
* {@link Source} a member variable.
* <p>
* If a {@link DirectSourceCache} instance is closed or invalidated the cached
* reference are simply set to <code>null</code>
*/
public static final class DirectSourceCache extends SourceCache {
private Source ref;
private final CloseableThreadLocal<Source> directSourceCache = new CloseableThreadLocal<Source>();
/** Sole constructor. */
public DirectSourceCache() {
}
@Override
public synchronized Source load(DocValues values) throws IOException {
if (ref == null) {
ref = values.loadSource();
}
return ref;
}
@Override
public synchronized void invalidate(DocValues values) {
ref = null;
directSourceCache.close();
}
@Override
public synchronized Source loadDirect(DocValues values) throws IOException {
final Source source = directSourceCache.get();
if (source == null) {
final Source loadDirectSource = values.loadDirectSource();
directSourceCache.set(loadDirectSource);
return loadDirectSource;
} else {
return source;
}
}
}
}
}

View File

@ -23,6 +23,7 @@ import java.util.Map;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.codecs.SimpleDocValuesFormat;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.Counter;
@ -47,27 +48,25 @@ final class DocValuesProcessor extends StoredFieldsConsumer {
@Override
public void addField(int docID, StorableField field, FieldInfo fieldInfo) {
final DocValues.Type dvType = field.fieldType().docValueType();
// nocommit: these checks are duplicated everywhere
final DocValuesType dvType = field.fieldType().docValueType();
if (dvType != null) {
DocValues.Type currentDVType = fieldInfo.getDocValuesType();
DocValuesType currentDVType = fieldInfo.getDocValuesType();
if (currentDVType == null) {
fieldInfo.setDocValuesType(dvType);
} else if (currentDVType != dvType) {
throw new IllegalArgumentException("cannot change DocValues type from " + currentDVType + " to " + dvType + " for field \"" + fieldInfo.name + "\"");
}
if (DocValues.isBytes(dvType)) {
if (dvType == DocValuesType.BINARY) {
addBinaryField(fieldInfo, docID, field.binaryValue());
} else if (DocValues.isSortedBytes(dvType)) {
} else if (dvType == DocValuesType.SORTED) {
addSortedField(fieldInfo, docID, field.binaryValue());
} else if (DocValues.isFloat(dvType)) {
if (dvType == DocValues.Type.FLOAT_32) {
addNumericField(fieldInfo, docID, field.numericValue().floatValue());
} else if (dvType == DocValues.Type.FLOAT_64) {
addNumericField(fieldInfo, docID, field.numericValue().doubleValue());
} else {
assert false;
}
} else if (DocValues.isNumber(dvType)) {
// nocommit: hack
} else if (dvType == DocValuesType.NUMERIC && field.numericValue() instanceof Float) {
addNumericField(fieldInfo, docID, field.numericValue().floatValue());
} else if (dvType == DocValuesType.NUMERIC && field.numericValue() instanceof Double) {
addNumericField(fieldInfo, docID, field.numericValue().doubleValue());
} else if (dvType == DocValuesType.NUMERIC) {
addNumericField(fieldInfo, docID, field.numericValue().longValue());
} else {
assert false: "unrecognized DocValues.Type: " + dvType;

View File

@ -87,7 +87,7 @@ class DocumentsWriterPerThread {
final InvertedDocConsumer termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true,
new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null));
final NormsConsumer normsWriter = new NormsConsumer(documentsWriterPerThread);
final NormsConsumer normsWriter = new NormsConsumer();
final DocInverter docInverter = new DocInverter(documentsWriterPerThread.docState, termsHash, normsWriter);
final StoredFieldsConsumer storedFields = new TwoStoredFieldsConsumers(
new StoredFieldsProcessor(documentsWriterPerThread),
@ -657,10 +657,6 @@ class DocumentsWriterPerThread {
}
}
PerDocWriteState newPerDocWriteState(String segmentSuffix) {
assert segmentInfo != null;
return new PerDocWriteState(infoStream, directory, segmentInfo, bytesUsed, segmentSuffix, IOContext.DEFAULT);
}
@Override
public String toString() {

View File

@ -20,8 +20,6 @@ package org.apache.lucene.index;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.index.DocValues.Type;
/**
* Access to the Field Info file that describes document fields and whether or
* not they are indexed. Each segment has a separate Field Info file. Objects
@ -37,12 +35,12 @@ public final class FieldInfo {
public final int number;
private boolean indexed;
private DocValues.Type docValueType;
private DocValuesType docValueType;
// True if any document indexed term vectors
private boolean storeTermVector;
private DocValues.Type normType;
private DocValuesType normType;
private boolean omitNorms; // omit norms associated with indexed fields
private IndexOptions indexOptions;
private boolean storePayloads; // whether this field stores payloads together with term positions
@ -82,6 +80,12 @@ public final class FieldInfo {
*/
DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS,
};
public static enum DocValuesType {
NUMERIC,
BINARY,
SORTED
};
/**
* Sole Constructor.
@ -89,7 +93,7 @@ public final class FieldInfo {
* @lucene.experimental
*/
public FieldInfo(String name, boolean indexed, int number, boolean storeTermVector,
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normsType, Map<String,String> attributes) {
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normsType, Map<String,String> attributes) {
this.name = name;
this.indexed = indexed;
this.number = number;
@ -163,7 +167,7 @@ public final class FieldInfo {
assert checkConsistency();
}
void setDocValuesType(DocValues.Type type) {
void setDocValuesType(DocValuesType type) {
docValueType = type;
assert checkConsistency();
}
@ -181,16 +185,16 @@ public final class FieldInfo {
}
/**
* Returns {@link DocValues.Type} of the docValues. this may be null if the field has no docvalues.
* Returns {@link DocValuesType} of the docValues. this may be null if the field has no docvalues.
*/
public DocValues.Type getDocValuesType() {
public DocValuesType getDocValuesType() {
return docValueType;
}
/**
* Returns {@link DocValues.Type} of the norm. this may be null if the field has no norms.
* Returns {@link DocValuesType} of the norm. this may be null if the field has no norms.
*/
public DocValues.Type getNormType() {
public DocValuesType getNormType() {
return normType;
}
@ -206,7 +210,7 @@ public final class FieldInfo {
assert checkConsistency();
}
void setNormValueType(Type type) {
void setNormValueType(DocValuesType type) {
normType = type;
assert checkConsistency();
}

View File

@ -25,6 +25,7 @@ import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
/**
@ -162,13 +163,13 @@ public class FieldInfos implements Iterable<FieldInfo> {
private final Map<Integer,String> numberToName;
private final Map<String,Integer> nameToNumber;
private final Map<String,DocValues.Type> docValuesType;
private final Map<String,DocValuesType> docValuesType;
private int lowestUnassignedFieldNumber = -1;
FieldNumbers() {
this.nameToNumber = new HashMap<String, Integer>();
this.numberToName = new HashMap<Integer, String>();
this.docValuesType = new HashMap<String,DocValues.Type>();
this.docValuesType = new HashMap<String,DocValuesType>();
}
/**
@ -177,9 +178,9 @@ public class FieldInfos implements Iterable<FieldInfo> {
* number assigned if possible otherwise the first unassigned field number
* is used as the field number.
*/
synchronized int addOrGet(String fieldName, int preferredFieldNumber, DocValues.Type dvType) {
synchronized int addOrGet(String fieldName, int preferredFieldNumber, DocValuesType dvType) {
if (dvType != null) {
DocValues.Type currentDVType = docValuesType.get(fieldName);
DocValuesType currentDVType = docValuesType.get(fieldName);
if (currentDVType == null) {
docValuesType.put(fieldName, dvType);
} else if (currentDVType != null && currentDVType != dvType) {
@ -211,7 +212,8 @@ public class FieldInfos implements Iterable<FieldInfo> {
/**
* Sets the given field number and name if not yet set.
*/
synchronized void setIfNotSet(int fieldNumber, String fieldName, DocValues.Type dvType) {
// nocommit: why is docvalues involved with global field numbers?
synchronized void setIfNotSet(int fieldNumber, String fieldName, DocValuesType dvType) {
final Integer boxedFieldNumber = Integer.valueOf(fieldNumber);
if (!numberToName.containsKey(boxedFieldNumber)
&& !nameToNumber.containsKey(fieldName)
@ -225,7 +227,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
}
// used by assert
synchronized boolean containsConsistent(Integer number, String name, DocValues.Type dvType) {
synchronized boolean containsConsistent(Integer number, String name, DocValuesType dvType) {
return name.equals(numberToName.get(number))
&& number.equals(nameToNumber.get(name)) &&
(dvType == null || docValuesType.get(name) == null || dvType == docValuesType.get(name));
@ -285,7 +287,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
*/
// TODO: fix testCodecs to do this another way, its the only user of this
FieldInfo addOrUpdate(String name, boolean isIndexed, boolean storeTermVector,
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) {
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normType) {
return addOrUpdateInternal(name, -1, isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions, docValues, normType);
}
@ -307,7 +309,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
private FieldInfo addOrUpdateInternal(String name, int preferredFieldNumber, boolean isIndexed,
boolean storeTermVector,
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValues.Type docValues, DocValues.Type normType) {
boolean omitNorms, boolean storePayloads, IndexOptions indexOptions, DocValuesType docValues, DocValuesType normType) {
FieldInfo fi = fieldInfo(name);
if (fi == null) {
// get a global number for this field
@ -317,7 +319,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
fi.update(isIndexed, storeTermVector, omitNorms, storePayloads, indexOptions);
if (docValues != null) {
DocValues.Type currentDVType = fi.getDocValuesType();
DocValuesType currentDVType = fi.getDocValuesType();
if (currentDVType == null) {
fi.setDocValuesType(docValues);
} else if (currentDVType != docValues) {
@ -327,7 +329,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
}
if (!fi.omitsNorms() && normType != null) {
DocValues.Type currentDVType = fi.getNormType();
DocValuesType currentDVType = fi.getNormType();
if (currentDVType == null) {
fi.setNormValueType(docValues);
} else if (currentDVType != normType) {
@ -348,7 +350,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
private FieldInfo addInternal(String name, int fieldNumber, boolean isIndexed,
boolean storeTermVector, boolean omitNorms, boolean storePayloads,
IndexOptions indexOptions, DocValues.Type docValuesType, DocValues.Type normType) {
IndexOptions indexOptions, DocValuesType docValuesType, DocValuesType normType) {
globalFieldNumbers.setIfNotSet(fieldNumber, name, docValuesType);
final FieldInfo fi = new FieldInfo(name, isIndexed, fieldNumber, storeTermVector, omitNorms, storePayloads, indexOptions, docValuesType, normType, null);
putInternal(fi);

View File

@ -405,12 +405,6 @@ public class FilterAtomicReader extends AtomicReader {
return buffer.toString();
}
@Override
public DocValues docValues(String field) throws IOException {
ensureOpen();
return in.docValues(field);
}
@Override
public NumericDocValues getNumericDocValues(String field) throws IOException {
ensureOpen();
@ -428,12 +422,6 @@ public class FilterAtomicReader extends AtomicReader {
ensureOpen();
return in.getSortedDocValues(field);
}
@Override
public DocValues normValues(String field) throws IOException {
ensureOpen();
return in.normValues(field);
}
@Override
public NumericDocValues simpleNormValues(String field) throws IOException {

View File

@ -55,7 +55,7 @@ public interface IndexableField extends GeneralField {
* a "document boost", then you must pre-multiply it across all the
* relevant fields yourself.
* <p>The boost is used to compute the norm factor for the field. By
* default, in the {@link Similarity#computeNorm(FieldInvertState, Norm)} method,
* default, in the {@link Similarity#computeNorm(FieldInvertState)} method,
* the boost value is multiplied by the length normalization factor and then
* rounded by {@link DefaultSimilarity#encodeNormValue(float)} before it is stored in the
* index. One should attempt to ensure that this product does not overflow
@ -65,7 +65,7 @@ public interface IndexableField extends GeneralField {
* indexed ({@link IndexableFieldType#indexed()} is false) or omits normalization values
* ({@link IndexableFieldType#omitNorms()} returns true).
*
* @see Similarity#computeNorm(FieldInvertState, Norm)
* @see Similarity#computeNorm(FieldInvertState)
* @see DefaultSimilarity#encodeNormValue(float)
*/
public float boost();

View File

@ -18,6 +18,7 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.analysis.Analyzer; // javadocs
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.FieldInfo.IndexOptions;
/**
@ -92,8 +93,8 @@ public interface IndexableFieldType {
public IndexOptions indexOptions();
/**
* DocValues {@link DocValues.Type}: if non-null then the field's value
* DocValues {@link DocValuesType}: if non-null then the field's value
* will be indexed into docValues.
*/
public DocValues.Type docValueType();
public DocValuesType docValueType();
}

View File

@ -1,608 +0,0 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.lang.reflect.Array;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.index.SortedBytesMergeUtils.MergeContext;
import org.apache.lucene.index.SortedBytesMergeUtils.SortedSourceSlice;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.packed.PackedInts.Reader;
// nocommit if types are inconsistent, refuse to build
/**
* A wrapper for CompositeIndexReader providing access to per segment
* {@link DocValues}
*
* <p><b>NOTE</b>: for multi readers, you'll get better
* performance by gathering the sub readers using
* {@link IndexReader#getContext()} to get the
* atomic leaves and then operate per-AtomicReader,
* instead of using this class.
*
* @lucene.experimental
* @lucene.internal
*/
class MultiDocValues extends DocValues {
private static DocValuesPuller DEFAULT_PULLER = new DocValuesPuller();
private static final DocValuesPuller NORMS_PULLER = new DocValuesPuller() {
@Override
public DocValues pull(AtomicReader reader, String field) throws IOException {
return reader.normValues(field);
}
@Override
public boolean stopLoadingOnNull(AtomicReader reader, String field) {
// for norms we drop all norms if one leaf reader has no norms and the field is present
FieldInfos fieldInfos = reader.getFieldInfos();
FieldInfo fieldInfo = fieldInfos.fieldInfo(field);
return fieldInfo != null && fieldInfo.omitsNorms();
}
};
public static class DocValuesSlice {
public final static DocValuesSlice[] EMPTY_ARRAY = new DocValuesSlice[0];
final int start;
final int length;
DocValues docValues;
public DocValuesSlice(DocValues docValues, int start, int length) {
this.docValues = docValues;
this.start = start;
this.length = length;
}
}
private static class DocValuesPuller {
public DocValuesPuller() {}
public DocValues pull(AtomicReader reader, String field) throws IOException {
return reader.docValues(field);
}
public boolean stopLoadingOnNull(AtomicReader reader, String field) {
return false;
}
}
private DocValuesSlice[] slices;
private int[] starts;
private Type type;
private int valueSize;
private MultiDocValues(DocValuesSlice[] slices, int[] starts, TypePromoter promotedType) {
this.starts = starts;
this.slices = slices;
this.type = promotedType.type();
this.valueSize = promotedType.getValueSize();
}
/**
* Returns a single {@link DocValues} instance for this 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 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 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 reader, final String field, final DocValuesPuller puller) throws IOException {
if (reader instanceof AtomicReader) {
// already an atomic reader
return puller.pull((AtomicReader) reader, field);
}
assert reader instanceof CompositeReader;
final List<AtomicReaderContext> leaves = reader.leaves();
switch (leaves.size()) {
case 0:
// no fields
return null;
case 1:
// already an atomic reader / reader with one leave
return getDocValues(leaves.get(0).reader(), field, puller);
default:
final List<DocValuesSlice> slices = new ArrayList<DocValuesSlice>();
TypePromoter promotedType = TypePromoter.getIdentityPromoter();
// gather all docvalues fields, accumulating a promoted type across
// potentially incompatible types
for (final AtomicReaderContext ctx : leaves) {
final AtomicReader r = ctx.reader();
final DocValues d = puller.pull(r, field);
if (d != null) {
TypePromoter incoming = TypePromoter.create(d.getType(), d.getValueSize());
promotedType = promotedType.promote(incoming);
} else if (puller.stopLoadingOnNull(r, field)){
return null;
}
slices.add(new DocValuesSlice(d, ctx.docBase, r.maxDoc()));
}
// return null if no docvalues encountered anywhere
if (promotedType == TypePromoter.getIdentityPromoter()) {
return null;
}
// populate starts and fill gaps with empty docvalues
int starts[] = new int[slices.size()];
for (int i = 0; i < slices.size(); i++) {
DocValuesSlice slice = slices.get(i);
starts[i] = slice.start;
if (slice.docValues == null) {
Type promoted = promotedType.type();
switch(promoted) {
case BYTES_FIXED_DEREF:
case BYTES_FIXED_STRAIGHT:
case BYTES_FIXED_SORTED:
assert promotedType.getValueSize() >= 0;
slice.docValues = new EmptyFixedDocValues(slice.length, promoted, promotedType.getValueSize());
break;
default:
slice.docValues = new EmptyDocValues(slice.length, promoted);
}
}
}
return new MultiDocValues(slices.toArray(new DocValuesSlice[slices.size()]), starts, promotedType);
}
}
@Override
protected Source loadSource() throws IOException {
return new MultiSource(slices, starts, false, type);
}
public static class EmptyDocValues extends DocValues {
final int maxDoc;
final Source emptySource;
public EmptyDocValues(int maxDoc, Type type) {
this.maxDoc = maxDoc;
this.emptySource = new EmptySource(type);
}
@Override
protected Source loadSource() throws IOException {
return emptySource;
}
@Override
public Type getType() {
return emptySource.getType();
}
@Override
protected Source loadDirectSource() throws IOException {
return emptySource;
}
}
public static class EmptyFixedDocValues extends DocValues {
final int maxDoc;
final Source emptyFixedSource;
final int valueSize;
public EmptyFixedDocValues(int maxDoc, Type type, int valueSize) {
this.maxDoc = maxDoc;
this.emptyFixedSource = new EmptyFixedSource(type, valueSize);
this.valueSize = valueSize;
}
@Override
protected Source loadSource() throws IOException {
return emptyFixedSource;
}
@Override
public Type getType() {
return emptyFixedSource.getType();
}
@Override
public int getValueSize() {
return valueSize;
}
@Override
protected Source loadDirectSource() throws IOException {
return emptyFixedSource;
}
}
private static class MultiSource extends Source {
private int numDocs = 0;
private int start = 0;
private Source current;
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);
this.slices = slices;
this.starts = starts;
assert slices.length != 0;
this.direct = direct;
}
@Override
public long getInt(int docID) {
final int doc = ensureSource(docID);
return current.getInt(doc);
}
private final int ensureSource(int docID) {
if (docID >= start && docID < start+numDocs) {
return docID - start;
} else {
final int idx = ReaderUtil.subIndex(docID, starts);
assert idx >= 0 && idx < slices.length : "idx was " + idx
+ " for doc id: " + docID + " slices : " + Arrays.toString(starts);
assert slices[idx] != null;
try {
if (direct) {
current = slices[idx].docValues.getDirectSource();
} else {
current = slices[idx].docValues.getSource();
}
} catch (IOException e) {
throw new RuntimeException("load failed", e); // TODO how should we
// handle this
}
start = slices[idx].start;
numDocs = slices[idx].length;
return docID - start;
}
}
@Override
public double getFloat(int docID) {
final int doc = ensureSource(docID);
return current.getFloat(doc);
}
@Override
public BytesRef getBytes(int docID, BytesRef bytesRef) {
final int doc = ensureSource(docID);
return current.getBytes(doc, bytesRef);
}
@Override
public SortedSource asSortedSource() {
try {
if (type == Type.BYTES_FIXED_SORTED || type == Type.BYTES_VAR_SORTED) {
DocValues[] values = new DocValues[slices.length];
Comparator<BytesRef> comp = null;
for (int i = 0; i < values.length; i++) {
values[i] = slices[i].docValues;
if (!(values[i] instanceof EmptyDocValues)) {
Comparator<BytesRef> comparator = values[i].getDirectSource()
.asSortedSource().getComparator();
assert comp == null || comp == comparator;
comp = comparator;
}
}
assert comp != null;
final int globalNumDocs = globalNumDocs();
final MergeContext ctx = SortedBytesMergeUtils.init(type, values,
comp, globalNumDocs);
List<SortedSourceSlice> slices = SortedBytesMergeUtils.buildSlices(
docBases(), new MergeState.DocMap[values.length], values, ctx);
RecordingBytesRefConsumer consumer = new RecordingBytesRefConsumer(
type);
final int maxOrd = SortedBytesMergeUtils.mergeRecords(ctx, consumer,
slices);
final int[] docToOrd = new int[globalNumDocs];
for (SortedSourceSlice slice : slices) {
slice.toAbsolutOrds(docToOrd);
}
return new MultiSortedSource(type, comp, consumer.pagedBytes,
ctx.sizePerValues, maxOrd, docToOrd, consumer.ordToOffset);
}
} catch (IOException e) {
throw new RuntimeException("load failed", e);
}
return super.asSortedSource();
}
private int globalNumDocs() {
int docs = 0;
for (int i = 0; i < slices.length; i++) {
docs += slices[i].length;
}
return docs;
}
private int[] docBases() {
int[] docBases = new int[slices.length];
for (int i = 0; i < slices.length; i++) {
docBases[i] = slices[i].start;
}
return docBases;
}
@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);
}
}
}
private static final class RecordingBytesRefConsumer implements SortedBytesMergeUtils.BytesRefConsumer {
private final static int PAGED_BYTES_BITS = 15;
final PagedBytes pagedBytes = new PagedBytes(PAGED_BYTES_BITS);
long[] ordToOffset;
public RecordingBytesRefConsumer(Type type) {
ordToOffset = type == Type.BYTES_VAR_SORTED ? new long[2] : null;
}
@Override
public void consume(BytesRef ref, int ord, long offset) {
pagedBytes.copy(ref);
if (ordToOffset != null) {
if (ord+1 >= ordToOffset.length) {
ordToOffset = ArrayUtil.grow(ordToOffset, ord + 2);
}
ordToOffset[ord+1] = offset;
}
}
}
private static final class MultiSortedSource extends SortedSource {
private final PagedBytes.Reader data;
private final int[] docToOrd;
private final long[] ordToOffset;
private int size;
private int valueCount;
public MultiSortedSource(Type type, Comparator<BytesRef> comparator, PagedBytes pagedBytes, int size, int numValues, int[] docToOrd, long[] ordToOffset) {
super(type, comparator);
data = pagedBytes.freeze(true);
this.size = size;
this.valueCount = numValues;
this.docToOrd = docToOrd;
this.ordToOffset = ordToOffset;
}
@Override
public int ord(int docID) {
return docToOrd[docID];
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
int size = this.size;
long offset = (ord*size);
if (ordToOffset != null) {
offset = ordToOffset[ord];
size = (int) (ordToOffset[1 + ord] - offset);
}
assert size >=0;
return data.fillSlice(bytesRef, offset, size);
}
@Override
public Reader getDocToOrd() {
return null;
}
@Override
public int getValueCount() {
return valueCount;
}
}
// TODO: this is dup of DocValues.getDefaultSource()?
private static class EmptySource extends SortedSource {
public EmptySource(Type type) {
super(type, BytesRef.getUTF8SortedAsUnicodeComparator());
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
ref.length = 0;
return ref;
}
@Override
public double getFloat(int docID) {
return 0d;
}
@Override
public long getInt(int docID) {
return 0;
}
@Override
public SortedSource asSortedSource() {
if (getType() == Type.BYTES_FIXED_SORTED || getType() == Type.BYTES_VAR_SORTED) {
}
return super.asSortedSource();
}
@Override
public int ord(int docID) {
return 0;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
bytesRef.length = 0;
bytesRef.offset = 0;
return bytesRef;
}
@Override
public Reader getDocToOrd() {
return null;
}
@Override
public int getValueCount() {
return 1;
}
}
private static class EmptyFixedSource extends EmptySource {
private final int valueSize;
private final byte[] valueArray;
public EmptyFixedSource(Type type, int valueSize) {
super(type);
this.valueSize = valueSize;
valueArray = new byte[valueSize];
}
@Override
public BytesRef getBytes(int docID, BytesRef ref) {
ref.grow(valueSize);
ref.length = valueSize;
Arrays.fill(ref.bytes, ref.offset, ref.offset+valueSize, (byte)0);
return ref;
}
@Override
public double getFloat(int docID) {
return 0d;
}
@Override
public long getInt(int docID) {
return 0;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
bytesRef.bytes = valueArray;
bytesRef.length = valueSize;
bytesRef.offset = 0;
return bytesRef;
}
}
@Override
public Type getType() {
return type;
}
@Override
public int getValueSize() {
return valueSize;
}
@Override
protected Source loadDirectSource() throws IOException {
return new MultiSource(slices, starts, true, type);
}
}

View File

@ -1,189 +0,0 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.document.ByteDocValuesField;
import org.apache.lucene.document.DerefBytesDocValuesField;
import org.apache.lucene.document.DoubleDocValuesField;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FloatDocValuesField;
import org.apache.lucene.document.IntDocValuesField;
import org.apache.lucene.document.LongDocValuesField;
import org.apache.lucene.document.PackedLongDocValuesField;
import org.apache.lucene.document.ShortDocValuesField;
import org.apache.lucene.document.SortedBytesDocValuesField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StraightBytesDocValuesField;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.BytesRef;
/**
* Stores the normalization value with {@link StorableField} computed in
* {@link Similarity#computeNorm(FieldInvertState, Norm)} per field.
* Normalization values must be consistent within a single field, different
* value types are not permitted within a single field. All values set must be
* fixed size values ie. all values passed to {@link Norm#setBytes(BytesRef)}
* must have the same length per field.
*
* @lucene.experimental
* @lucene.internal
*/
public final class Norm {
private StoredField field;
private BytesRef spare;
/** Sole constructor. */
public Norm() {
}
/**
* Returns the {@link StorableField} representation for this norm
*/
public StorableField field() {
return field;
}
/**
* Returns the {@link Type} for this norm.
*/
public Type type() {
return field == null? null : field.fieldType().docValueType();
}
/**
* Returns a spare {@link BytesRef}
*/
public BytesRef getSpare() {
if (spare == null) {
spare = new BytesRef();
}
return spare;
}
/**
* Sets a float norm value
*/
public void setFloat(float norm) {
setType(Type.FLOAT_32);
this.field.setFloatValue(norm);
}
/**
* Sets a double norm value
*/
public void setDouble(double norm) {
setType(Type.FLOAT_64);
this.field.setDoubleValue(norm);
}
/**
* Sets a short norm value
*/
public void setShort(short norm) {
setType(Type.FIXED_INTS_16);
this.field.setShortValue(norm);
}
/**
* Sets a int norm value
*/
public void setInt(int norm) {
setType(Type.FIXED_INTS_32);
this.field.setIntValue(norm);
}
/**
* Sets a long norm value
*/
public void setLong(long norm) {
setType(Type.FIXED_INTS_64);
this.field.setLongValue(norm);
}
/**
* Sets a byte norm value
*/
public void setByte(byte norm) {
setType(Type.FIXED_INTS_8);
this.field.setByteValue(norm);
}
/**
* Sets a fixed byte array norm value
*/
public void setBytes(BytesRef norm) {
setType(Type.BYTES_FIXED_STRAIGHT);
this.field.setBytesValue(norm);
}
private void setType(Type type) {
if (field != null) {
if (type != field.fieldType().docValueType()) {
throw new IllegalArgumentException("FieldType missmatch - expected "+type+" but was " + field.fieldType().docValueType());
}
} else {
switch(type) {
case VAR_INTS:
field = new PackedLongDocValuesField("", (long) 0);
break;
case FIXED_INTS_8:
field = new ByteDocValuesField("", (byte) 0);
break;
case FIXED_INTS_16:
field = new ShortDocValuesField("", (short) 0);
break;
case FIXED_INTS_32:
field = new IntDocValuesField("", 0);
break;
case FIXED_INTS_64:
field = new LongDocValuesField("", (byte) 0);
break;
case FLOAT_32:
field = new FloatDocValuesField("", 0f);
break;
case FLOAT_64:
field = new DoubleDocValuesField("", 0d);
break;
case BYTES_FIXED_STRAIGHT:
field = new StraightBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_STRAIGHT:
field = new StraightBytesDocValuesField("", new BytesRef(), false);
break;
case BYTES_FIXED_DEREF:
field = new DerefBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_DEREF:
field = new DerefBytesDocValuesField("", new BytesRef(), false);
break;
case BYTES_FIXED_SORTED:
field = new SortedBytesDocValuesField("", new BytesRef(), true);
break;
case BYTES_VAR_SORTED:
field = new SortedBytesDocValuesField("", new BytesRef(), false);
break;
default:
throw new IllegalArgumentException("unknown Type: " + type);
}
}
}
}

View File

@ -20,12 +20,9 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Map;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.codecs.SimpleNormsFormat;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.util.IOUtils;
// TODO FI: norms could actually be stored as doc store
@ -36,61 +33,40 @@ import org.apache.lucene.util.IOUtils;
*/
final class NormsConsumer extends InvertedDocEndConsumer {
private final NormsFormat normsFormat;
private PerDocConsumer consumer;
public NormsConsumer(DocumentsWriterPerThread dwpt) {
normsFormat = dwpt.codec.normsFormat();
}
@Override
public void abort(){
if (consumer != null) {
consumer.abort();
}
}
void abort() {}
@Override
public void flush(Map<String,InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
boolean success = false;
SimpleDVConsumer normsConsumer = null;
boolean anythingFlushed = false;
try {
if (state.fieldInfos.hasNorms()) {
SimpleNormsFormat normsFormat = state.segmentInfo.getCodec().simpleNormsFormat();
// nocommit change this to assert normsFormat != null
if (normsFormat != null) {
normsConsumer = normsFormat.normsConsumer(state);
}
assert normsFormat != null;
normsConsumer = normsFormat.normsConsumer(state);
for (FieldInfo fi : state.fieldInfos) {
final NormsConsumerPerField toWrite = (NormsConsumerPerField) fieldsToFlush.get(fi.name);
// 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.omitsNorms()) {
if (toWrite != null && toWrite.initialized()) {
anythingFlushed = true;
final Type type = toWrite.flush(state, normsConsumer);
assert fi.getNormType() == type;
if (toWrite != null && !toWrite.isEmpty()) {
toWrite.flush(state, normsConsumer);
assert fi.getNormType() == DocValuesType.NUMERIC;
} else if (fi.isIndexed()) {
anythingFlushed = true;
assert fi.getNormType() == null: "got " + fi.getNormType() + "; field=" + fi.name;
}
}
}
}
success = true;
if (!anythingFlushed && consumer != null) {
consumer.abort();
// nocommit do we also need to normsConsumer.abort!?
}
success = true;
} finally {
if (success) {
IOUtils.close(consumer, normsConsumer);
IOUtils.close(normsConsumer);
} else {
IOUtils.closeWhileHandlingException(consumer, normsConsumer);
IOUtils.closeWhileHandlingException(normsConsumer);
}
}
}
@ -102,18 +78,7 @@ final class NormsConsumer extends InvertedDocEndConsumer {
void startDocument() {}
@Override
InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField,
FieldInfo fieldInfo) {
InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo) {
return new NormsConsumerPerField(docInverterPerField, fieldInfo, this);
}
DocValuesConsumer newConsumer(PerDocWriteState perDocWriteState,
FieldInfo fieldInfo, Type type) throws IOException {
if (consumer == null) {
consumer = normsFormat.docsConsumer(perDocWriteState);
}
DocValuesConsumer addValuesField = consumer.addValuesField(type, fieldInfo);
return addValuesField;
}
}

View File

@ -17,9 +17,7 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.search.similarities.Similarity;
final class NormsConsumerPerField extends InvertedDocEndConsumerPerField implements Comparable<NormsConsumerPerField> {
@ -27,20 +25,13 @@ final class NormsConsumerPerField extends InvertedDocEndConsumerPerField impleme
private final DocumentsWriterPerThread.DocState docState;
private final Similarity similarity;
private final FieldInvertState fieldState;
private DocValuesConsumer consumer;
private final Norm norm;
private final NormsConsumer parent;
private Type initType;
private final NumberDVWriter simpleNormsWriter;
private NumberDVWriter consumer;
public NormsConsumerPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo, NormsConsumer parent) {
this.fieldInfo = fieldInfo;
this.parent = parent;
docState = docInverterPerField.docState;
fieldState = docInverterPerField.fieldState;
similarity = docState.similarity;
norm = new Norm();
simpleNormsWriter = new NumberDVWriter(fieldInfo, docState.docWriter.bytesUsed);
}
@Override
@ -51,65 +42,28 @@ final class NormsConsumerPerField extends InvertedDocEndConsumerPerField impleme
@Override
void finish() throws IOException {
if (fieldInfo.isIndexed() && !fieldInfo.omitsNorms()) {
similarity.computeNorm(fieldState, norm);
if (norm.type() != null) {
StorableField field = norm.field();
// some similarity might not compute any norms
DocValuesConsumer consumer = getConsumer(norm.type());
consumer.add(docState.docID, field);
if (consumer == null) {
consumer = new NumberDVWriter(fieldInfo, docState.docWriter.bytesUsed);
}
long norm = similarity.computeSimpleNorm(fieldState);
simpleNormsWriter.addValue(docState.docID, norm);
consumer.addValue(docState.docID, similarity.computeNorm(fieldState));
}
}
Type flush(SegmentWriteState state, SimpleDVConsumer normsConsumer) throws IOException {
void flush(SegmentWriteState state, SimpleDVConsumer normsWriter) throws IOException {
int docCount = state.segmentInfo.getDocCount();
if (!initialized()) {
return null; // null type - not omitted but not written
if (consumer == null) {
return; // null type - not omitted but not written
}
consumer.finish(docCount);
// nocommit change to assert normsConsumer != null
if (normsConsumer != null) {
// nocommit we need to change the suffix? ie so norms
// don't step on dvs? hmmm.... where does this happen
// today ...
simpleNormsWriter.finish(docCount);
simpleNormsWriter.flush(state, normsConsumer);
} else {
// nocommit remove:
simpleNormsWriter.reset();
}
return initType;
consumer.flush(state, normsWriter);
}
private DocValuesConsumer getConsumer(Type type) throws IOException {
if (consumer == null) {
if (fieldInfo.getNormType() != null && fieldInfo.getNormType() != type) {
throw new IllegalArgumentException("cannot change Norm type from " + fieldInfo.getNormType() + " to " + type + " for field \"" + fieldInfo.name + "\"");
}
if (!DocValues.isNumber(type) && !DocValues.isFloat(type)) {
throw new IllegalArgumentException("Norm type must be numeric (got type " + type + " for field \"" + fieldInfo.name + "\"");
}
fieldInfo.setNormValueType(type);
consumer = parent.newConsumer(docState.docWriter.newPerDocWriteState(""), fieldInfo, type);
this.initType = type;
}
if (initType != type) {
throw new IllegalArgumentException("NormTypes for field: " + fieldInfo.name + " doesn't match " + initType + " != " + type);
}
return consumer;
}
boolean initialized() {
return consumer != null;
boolean isEmpty() {
return consumer == null;
}
@Override
void abort() {
//
}
}

View File

@ -262,13 +262,6 @@ public final class ParallelAtomicReader extends AtomicReader {
// throw the first exception
if (ioe != null) throw ioe;
}
@Override
public DocValues docValues(String field) throws IOException {
ensureOpen();
AtomicReader reader = fieldToReader.get(field);
return reader == null ? null : reader.docValues(field);
}
@Override
public NumericDocValues getNumericDocValues(String field) throws IOException {
@ -290,13 +283,6 @@ public final class ParallelAtomicReader extends AtomicReader {
AtomicReader reader = fieldToReader.get(field);
return reader == null ? null : reader.getSortedDocValues(field);
}
@Override
public DocValues normValues(String field) throws IOException {
ensureOpen();
AtomicReader reader = fieldToReader.get(field);
return reader == null ? null : reader.normValues(field);
}
@Override
public NumericDocValues simpleNormValues(String field) throws IOException {

View File

@ -1,86 +0,0 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.InfoStream;
/**
* Encapsulates all necessary state to initiate a {@link PerDocConsumer} and
* create all necessary files in order to consume and merge per-document values.
*
* @lucene.experimental
*/
public class PerDocWriteState {
/** InfoStream used for debugging. */
public final InfoStream infoStream;
/** {@link Directory} to write all files to. */
public final Directory directory;
/** {@link SegmentInfo} describing this segment. */
public final SegmentInfo segmentInfo;
/** Number of bytes allocated in RAM to hold this state. */
public final Counter bytesUsed;
/** Segment suffix to pass to {@link
* IndexFileNames#segmentFileName(String,String,String)}. */
public final String segmentSuffix;
/** {@link IOContext} to use for all file writing. */
public final IOContext context;
/** Creates a {@code PerDocWriteState}. */
public PerDocWriteState(InfoStream infoStream, Directory directory,
SegmentInfo segmentInfo, Counter bytesUsed,
String segmentSuffix, IOContext context) {
this.infoStream = infoStream;
this.directory = directory;
this.segmentInfo = segmentInfo;
this.segmentSuffix = segmentSuffix;
this.bytesUsed = bytesUsed;
this.context = context;
}
/** Creates a {@code PerDocWriteState}, copying fields
* from another and allocating a new {@link #bytesUsed}. */
public PerDocWriteState(SegmentWriteState state) {
infoStream = state.infoStream;
directory = state.directory;
segmentInfo = state.segmentInfo;
segmentSuffix = state.segmentSuffix;
bytesUsed = Counter.newCounter();
context = state.context;
}
/** Creates a {@code PerDocWriteState}, copying fields
* from another (copy constructor) but setting a new
* {@link #segmentSuffix}. */
public PerDocWriteState(PerDocWriteState state, String segmentSuffix) {
this.infoStream = state.infoStream;
this.directory = state.directory;
this.segmentInfo = state.segmentInfo;
this.segmentSuffix = segmentSuffix;
this.bytesUsed = state.bytesUsed;
this.context = state.context;
}
}

View File

@ -27,11 +27,11 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.FieldsProducer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SimpleDVProducer;
import org.apache.lucene.codecs.StoredFieldsReader;
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.index.SegmentReader.CoreClosedListener;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
@ -56,8 +56,6 @@ final class SegmentCoreReaders {
final FieldsProducer fields;
final SimpleDVProducer simpleDVProducer;
final SimpleDVProducer simpleNormsProducer;
final PerDocProducer perDocProducer;
final PerDocProducer norms;
final int termsIndexDivisor;
@ -133,8 +131,6 @@ 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().docsProducer(segmentReadState);
perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
// nocommit shouldn't need null check:
if (codec.simpleDocValuesFormat() != null) {
if (fieldInfos.hasDocValues()) {
@ -192,7 +188,7 @@ final class SegmentCoreReaders {
// Field was not indexed with doc values
return null;
}
if (!DocValues.isNumber(fi.getDocValuesType()) && !DocValues.isFloat(fi.getDocValuesType())) {
if (fi.getDocValuesType() != DocValuesType.NUMERIC) {
// DocValues were not numeric
return null;
}
@ -223,7 +219,7 @@ final class SegmentCoreReaders {
// Field was not indexed with doc values
return null;
}
if (!DocValues.isBytes(fi.getDocValuesType())) {
if (fi.getDocValuesType() != DocValuesType.BINARY) {
// DocValues were not binary
return null;
}
@ -254,7 +250,7 @@ final class SegmentCoreReaders {
// Field was not indexed with doc values
return null;
}
if (!DocValues.isSortedBytes(fi.getDocValuesType())) {
if (fi.getDocValuesType() != DocValuesType.SORTED) {
// DocValues were not sorted
return null;
}
@ -303,8 +299,7 @@ final class SegmentCoreReaders {
void decRef() throws IOException {
if (ref.decrementAndGet() == 0) {
IOUtils.close(termVectorsLocal, fieldsReaderLocal, simpleDocValuesLocal, simpleNormsLocal, fields, simpleDVProducer,
perDocProducer, termVectorsReaderOrig, fieldsReaderOrig, cfsReader, norms,
simpleNormsProducer);
termVectorsReaderOrig, fieldsReaderOrig, cfsReader, simpleNormsProducer);
notifyCoreClosedListeners();
}
}

View File

@ -26,10 +26,10 @@ 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.PerDocConsumer;
import org.apache.lucene.codecs.SimpleDVConsumer;
import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsWriter;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
@ -97,7 +97,7 @@ final class SegmentMerger {
// threads.
mergeState.segmentInfo.setDocCount(setDocMaps());
mergeDocValuesAndNormsFieldInfos();
mergeFieldInfos();
setMatchingSegmentReaders();
long t0 = 0;
if (mergeState.infoStream.isEnabled("SM")) {
@ -124,7 +124,9 @@ final class SegmentMerger {
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
}
mergePerDoc(segmentWriteState);
if (mergeState.fieldInfos.hasDocValues()) {
mergeSimpleDocValues(segmentWriteState);
}
if (mergeState.infoStream.isEnabled("SM")) {
long t1 = System.nanoTime();
mergeState.infoStream.message("SM", ((t1-t0)/1000000) + " msec to merge doc values [" + numMerged + " docs]");
@ -134,7 +136,6 @@ final class SegmentMerger {
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
}
mergeNorms(segmentWriteState);
mergeSimpleNorms(segmentWriteState);
if (mergeState.infoStream.isEnabled("SM")) {
long t1 = System.nanoTime();
@ -142,12 +143,6 @@ final class SegmentMerger {
}
}
// Merge simple doc values:
if (mergeState.fieldInfos.hasDocValues()) {
// nocommit shouldn't need null check:
mergeSimpleDocValues(segmentWriteState);
}
if (mergeState.fieldInfos.hasVectors()) {
if (mergeState.infoStream.isEnabled("SM")) {
t0 = System.nanoTime();
@ -174,9 +169,9 @@ final class SegmentMerger {
boolean success = false;
try {
for (FieldInfo field : mergeState.fieldInfos) {
DocValues.Type type = field.getDocValuesType();
DocValuesType type = field.getDocValuesType();
if (type != null) {
if (DocValues.isNumber(type) || DocValues.isFloat(type)) {
if (type == DocValuesType.NUMERIC) {
List<NumericDocValues> toMerge = new ArrayList<NumericDocValues>();
for (AtomicReader reader : mergeState.readers) {
NumericDocValues values = reader.getNumericDocValues(field.name);
@ -186,7 +181,7 @@ final class SegmentMerger {
toMerge.add(values);
}
consumer.mergeNumericField(field, mergeState, toMerge);
} else if (DocValues.isBytes(type)) {
} else if (type == DocValuesType.BINARY) {
List<BinaryDocValues> toMerge = new ArrayList<BinaryDocValues>();
for (AtomicReader reader : mergeState.readers) {
BinaryDocValues values = reader.getBinaryDocValues(field.name);
@ -196,7 +191,7 @@ final class SegmentMerger {
toMerge.add(values);
}
consumer.mergeBinaryField(field, mergeState, toMerge);
} else if (DocValues.isSortedBytes(type)) {
} else if (type == DocValuesType.SORTED) {
List<SortedDocValues> toMerge = new ArrayList<SortedDocValues>();
for (AtomicReader reader : mergeState.readers) {
SortedDocValues values = reader.getSortedDocValues(field.name);
@ -295,72 +290,15 @@ final class SegmentMerger {
}
}
// returns an updated typepromoter (tracking type and size) given a previous one,
// and a newly encountered docvalues
private TypePromoter mergeDocValuesType(TypePromoter previous, DocValues docValues) {
TypePromoter incoming = TypePromoter.create(docValues.getType(), docValues.getValueSize());
if (previous == null) {
previous = TypePromoter.getIdentityPromoter();
}
return previous.promote(incoming);
}
// NOTE: this is actually merging all the fieldinfos
public void mergeDocValuesAndNormsFieldInfos() throws IOException {
// mapping from all docvalues fields found to their promoted types
// this is because FieldInfos does not store the
// valueSize
Map<FieldInfo,TypePromoter> docValuesTypes = new HashMap<FieldInfo,TypePromoter>();
Map<FieldInfo,TypePromoter> normValuesTypes = new HashMap<FieldInfo,TypePromoter>();
public void mergeFieldInfos() throws IOException {
for (AtomicReader reader : mergeState.readers) {
FieldInfos readerFieldInfos = reader.getFieldInfos();
for (FieldInfo fi : readerFieldInfos) {
FieldInfo merged = fieldInfosBuilder.add(fi);
// update the type promotion mapping for this reader
if (fi.hasDocValues()) {
TypePromoter previous = docValuesTypes.get(merged);
docValuesTypes.put(merged, mergeDocValuesType(previous, reader.docValues(fi.name)));
}
if (fi.hasNorms()) {
TypePromoter previous = normValuesTypes.get(merged);
normValuesTypes.put(merged, mergeDocValuesType(previous, reader.normValues(fi.name)));
}
fieldInfosBuilder.add(fi);
}
}
updatePromoted(normValuesTypes, true);
updatePromoted(docValuesTypes, false);
mergeState.fieldInfos = fieldInfosBuilder.finish();
}
protected void updatePromoted(Map<FieldInfo,TypePromoter> infoAndPromoter, boolean norms) {
// update any promoted doc values types:
for (Map.Entry<FieldInfo,TypePromoter> e : infoAndPromoter.entrySet()) {
FieldInfo fi = e.getKey();
TypePromoter promoter = e.getValue();
if (promoter == null) {
if (norms) {
fi.setNormValueType(null);
} else {
fi.setDocValuesType(null);
}
} else {
assert promoter != TypePromoter.getIdentityPromoter();
if (norms) {
if (fi.getNormType() != promoter.type() && !fi.omitsNorms()) {
// reset the type if we got promoted
fi.setNormValueType(promoter.type());
}
} else {
if (fi.getDocValuesType() != promoter.type()) {
// reset the type if we got promoted
fi.setDocValuesType(promoter.type());
}
}
}
}
}
/**
*
@ -451,38 +389,4 @@ final class SegmentMerger {
}
}
}
private void mergePerDoc(SegmentWriteState segmentWriteState) throws IOException {
final PerDocConsumer docsConsumer = codec.docValuesFormat()
.docsConsumer(new PerDocWriteState(segmentWriteState));
assert docsConsumer != null;
boolean success = false;
try {
docsConsumer.merge(mergeState);
success = true;
} finally {
if (success) {
IOUtils.close(docsConsumer);
} else {
IOUtils.closeWhileHandlingException(docsConsumer);
}
}
}
private void mergeNorms(SegmentWriteState segmentWriteState) throws IOException {
final PerDocConsumer docsConsumer = codec.normsFormat()
.docsConsumer(new PerDocWriteState(segmentWriteState));
assert docsConsumer != null;
boolean success = false;
try {
docsConsumer.merge(mergeState);
success = true;
} finally {
if (success) {
IOUtils.close(docsConsumer);
} else {
IOUtils.closeWhileHandlingException(docsConsumer);
}
}
}
}

View File

@ -20,7 +20,6 @@ package org.apache.lucene.index;
import java.io.IOException;
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.search.FieldCache; // javadocs
@ -248,32 +247,11 @@ public final class SegmentReader extends AtomicReader {
return core.getSortedDocValues(field);
}
@Override
public DocValues docValues(String field) throws IOException {
ensureOpen();
final PerDocProducer perDoc = core.perDocProducer;
if (perDoc == null) {
return null;
}
return perDoc.docValues(field);
}
@Override
public NumericDocValues simpleNormValues(String field) throws IOException {
ensureOpen();
return core.getSimpleNormValues(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

View File

@ -18,8 +18,6 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.util.Bits;
@ -31,7 +29,7 @@ import org.apache.lucene.index.MultiReader; // javadoc
* MultiReader} or {@link DirectoryReader}) to emulate an
* atomic reader. This requires implementing the postings
* APIs on-the-fly, using the static methods in {@link
* MultiFields}, {@link MultiDocValues}, by stepping through
* MultiFields}, {@link MultiSimpleDocValues}, by stepping through
* the sub-readers to merge fields/terms, appending docs, etc.
*
* <p><b>NOTE</b>: this class almost always results in a
@ -44,7 +42,6 @@ import org.apache.lucene.index.MultiReader; // javadoc
public final class SlowCompositeReaderWrapper extends AtomicReader {
private final CompositeReader in;
private final Map<String, DocValues> normsCache = new HashMap<String, DocValues>();
private final Fields fields;
private final Bits liveDocs;
@ -82,12 +79,6 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
return fields;
}
@Override
public DocValues docValues(String field) throws IOException {
ensureOpen();
return MultiDocValues.getDocValues(in, field);
}
@Override
public NumericDocValues getNumericDocValues(String field) throws IOException {
ensureOpen();
@ -105,17 +96,6 @@ public final class SlowCompositeReaderWrapper extends AtomicReader {
ensureOpen();
return MultiSimpleDocValues.simpleSortedValues(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 NumericDocValues simpleNormValues(String field) throws IOException {

View File

@ -1,435 +0,0 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.List;
import org.apache.lucene.index.DocValues.SortedSource;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.PriorityQueue;
import org.apache.lucene.util.packed.PackedInts;
/**
* Utility class for merging SortedBytes DocValues
* instances.
*
* @lucene.internal
*/
public final class SortedBytesMergeUtils {
private SortedBytesMergeUtils() {
// no instance
}
/** Creates the {@link MergeContext} necessary for merging
* the ordinals. */
public static MergeContext init(Type type, DocValues[] docValues,
Comparator<BytesRef> comp, int mergeDocCount) {
int size = -1;
if (type == Type.BYTES_FIXED_SORTED) {
for (DocValues indexDocValues : docValues) {
if (indexDocValues != null) {
size = indexDocValues.getValueSize();
break;
}
}
assert size >= 0;
}
return new MergeContext(comp, mergeDocCount, size, type);
}
/**
* Encapsulates contextual information about the merge.
* This class holds document id to ordinal mappings, offsets for
* variable length values and the comparator to sort the merged
* bytes.
*
* @lucene.internal
*/
public static final class MergeContext {
private final Comparator<BytesRef> comp;
private final BytesRef missingValue = new BytesRef();
/** How many bytes each value occupies, or -1 if it
* varies. */
public final int sizePerValues; // -1 if var length
final Type type;
/** Maps each document to the ordinal for its value. */
public final int[] docToEntry;
/** File-offset for each document; will be null if it's
* not needed (eg fixed-size values). */
public long[] offsets; // if non-null #mergeRecords collects byte offsets here
/** Sole constructor. */
public MergeContext(Comparator<BytesRef> comp, int mergeDocCount,
int size, Type type) {
assert type == Type.BYTES_FIXED_SORTED || type == Type.BYTES_VAR_SORTED;
this.comp = comp;
this.sizePerValues = size;
this.type = type;
if (size > 0) {
missingValue.grow(size);
missingValue.length = size;
}
docToEntry = new int[mergeDocCount];
}
/** Returns number of documents merged. */
public int getMergeDocCount() {
return docToEntry.length;
}
}
/** Creates the {@link SortedSourceSlice}s for
* merging. */
public static List<SortedSourceSlice> buildSlices(
int[] docBases, MergeState.DocMap[] docMaps,
DocValues[] docValues, MergeContext ctx) throws IOException {
final List<SortedSourceSlice> slices = new ArrayList<SortedSourceSlice>();
for (int i = 0; i < docValues.length; i++) {
final SortedSourceSlice nextSlice;
final Source directSource;
if (docValues[i] != null
&& (directSource = docValues[i].getDirectSource()) != null) {
final SortedSourceSlice slice = new SortedSourceSlice(i, directSource
.asSortedSource(), docBases, ctx.getMergeDocCount(), ctx.docToEntry);
nextSlice = slice;
} else {
nextSlice = new SortedSourceSlice(i, new MissingValueSource(ctx),
docBases, ctx.getMergeDocCount(), ctx.docToEntry);
}
createOrdMapping(docBases, docMaps, nextSlice);
slices.add(nextSlice);
}
return Collections.unmodifiableList(slices);
}
/*
* In order to merge we need to map the ords used in each segment to the new
* global ords in the new segment. Additionally we need to drop values that
* are not referenced anymore due to deleted documents. This method walks all
* live documents and fetches their current ordinal. We store this ordinal per
* slice and (SortedSourceSlice#ordMapping) and remember the doc to ord
* mapping in docIDToRelativeOrd. After the merge SortedSourceSlice#ordMapping
* contains the new global ordinals for the relative index.
*/
private static void createOrdMapping(int[] docBases, MergeState.DocMap[] docMaps,
SortedSourceSlice currentSlice) {
final int readerIdx = currentSlice.readerIdx;
final MergeState.DocMap currentDocMap = docMaps[readerIdx];
final int docBase = currentSlice.docToOrdStart;
assert docBase == docBases[readerIdx];
if (currentDocMap != null && currentDocMap.hasDeletions()) { // we have deletes
for (int i = 0; i < currentDocMap.maxDoc(); i++) {
final int doc = currentDocMap.get(i);
if (doc != -1) { // not deleted
final int ord = currentSlice.source.ord(i); // collect ords strictly
// increasing
currentSlice.docIDToRelativeOrd[docBase + doc] = ord;
// use ord + 1 to identify unreferenced values (ie. == 0)
currentSlice.ordMapping[ord] = ord + 1;
}
}
} else { // no deletes
final int numDocs = currentSlice.docToOrdEnd - currentSlice.docToOrdStart;
for (int doc = 0; doc < numDocs; doc++) {
final int ord = currentSlice.source.ord(doc);
currentSlice.docIDToRelativeOrd[docBase + doc] = ord;
// use ord + 1 to identify unreferenced values (ie. == 0)
currentSlice.ordMapping[ord] = ord + 1;
}
}
}
/** Does the "real work" of merging the slices and
* computing the ord mapping. */
public static int mergeRecords(MergeContext ctx, BytesRefConsumer consumer,
List<SortedSourceSlice> slices) throws IOException {
final RecordMerger merger = new RecordMerger(new MergeQueue(slices.size(),
ctx.comp), slices.toArray(new SortedSourceSlice[0]));
long[] offsets = ctx.offsets;
final boolean recordOffsets = offsets != null;
long offset = 0;
BytesRef currentMergedBytes;
merger.pushTop();
while (merger.queue.size() > 0) {
merger.pullTop();
currentMergedBytes = merger.current;
assert ctx.sizePerValues == -1 || ctx.sizePerValues == currentMergedBytes.length : "size: "
+ ctx.sizePerValues + " spare: " + currentMergedBytes.length;
offset += currentMergedBytes.length;
if (recordOffsets) {
if (merger.currentOrd >= offsets.length) {
offsets = ArrayUtil.grow(offsets, merger.currentOrd + 1);
}
offsets[merger.currentOrd] = offset;
}
consumer.consume(currentMergedBytes, merger.currentOrd, offset);
merger.pushTop();
}
ctx.offsets = offsets;
assert offsets == null || offsets[merger.currentOrd - 1] == offset;
return merger.currentOrd;
}
/**
* Implementation of this interface consume the merged bytes with their
* corresponding ordinal and byte offset. The offset is the byte offset in
* target sorted source where the currently merged {@link BytesRef} instance
* should be stored at.
*/
public static interface BytesRefConsumer {
/**
* Consumes a single {@link BytesRef}. The provided {@link BytesRef}
* instances are strictly increasing with respect to the used
* {@link Comparator} used for merging
*
* @param ref
* the {@link BytesRef} to consume
* @param ord
* the ordinal of the given {@link BytesRef} in the merge target
* @param offset
* the byte offset of the given {@link BytesRef} in the merge
* target
* @throws IOException
* if an {@link IOException} occurs
*/
public void consume(BytesRef ref, int ord, long offset) throws IOException;
}
/**
* A simple {@link BytesRefConsumer} that writes the merged {@link BytesRef}
* instances sequentially to an {@link IndexOutput}.
*/
public static final class IndexOutputBytesRefConsumer implements BytesRefConsumer {
private final IndexOutput datOut;
/** Sole constructor. */
public IndexOutputBytesRefConsumer(IndexOutput datOut) {
this.datOut = datOut;
}
@Override
public void consume(BytesRef currentMergedBytes, int ord, long offset) throws IOException {
datOut.writeBytes(currentMergedBytes.bytes, currentMergedBytes.offset,
currentMergedBytes.length);
}
}
/**
* {@link RecordMerger} merges a list of {@link SortedSourceSlice} lazily by
* consuming the sorted source records one by one and de-duplicates records
* that are shared across slices. The algorithm is based on a lazy priority queue
* that prevents reading merge sources into heap memory.
*
* @lucene.internal
*/
private static final class RecordMerger {
private final MergeQueue queue;
private final SortedSourceSlice[] top;
private int numTop;
BytesRef current;
int currentOrd = -1;
RecordMerger(MergeQueue queue, SortedSourceSlice[] top) {
super();
this.queue = queue;
this.top = top;
this.numTop = top.length;
}
private void pullTop() {
// extract all subs from the queue that have the same
// top record
assert numTop == 0;
assert currentOrd >= 0;
while (true) {
final SortedSourceSlice popped = top[numTop++] = queue.pop();
// use ord + 1 to identify unreferenced values (ie. == 0)
popped.ordMapping[popped.relativeOrd] = currentOrd + 1;
if (queue.size() == 0
|| !(queue.top()).current.bytesEquals(top[0].current)) {
break;
}
}
current = top[0].current;
}
private void pushTop() {
// call next() on each top, and put back into queue
for (int i = 0; i < numTop; i++) {
top[i].current = top[i].next();
if (top[i].current != null) {
queue.add(top[i]);
}
}
currentOrd++;
numTop = 0;
}
}
/**
* {@link SortedSourceSlice} represents a single {@link SortedSource} merge candidate.
* It encapsulates ordinal and pre-calculated target doc id to ordinal mappings.
* This class also holds state private to the merge process.
* @lucene.internal
*/
public static class SortedSourceSlice {
final SortedSource source;
final int readerIdx;
/* global array indexed by docID containg the relative ord for the doc */
final int[] docIDToRelativeOrd;
/*
* maps relative ords to merged global ords - index is relative ord value
* new global ord this map gets updates as we merge ords. later we use the
* docIDtoRelativeOrd to get the previous relative ord to get the new ord
* from the relative ord map.
*/
final int[] ordMapping;
/* start index into docIDToRelativeOrd */
final int docToOrdStart;
/* end index into docIDToRelativeOrd */
final int docToOrdEnd;
BytesRef current = new BytesRef();
/* the currently merged relative ordinal */
int relativeOrd = -1;
SortedSourceSlice(int readerIdx, SortedSource source, int[] docBase, int mergeDocCount,
int[] docToOrd) {
super();
this.readerIdx = readerIdx;
this.source = source;
this.docIDToRelativeOrd = docToOrd;
this.ordMapping = new int[source.getValueCount()];
this.docToOrdStart = docBase[readerIdx];
this.docToOrdEnd = this.docToOrdStart + numDocs(docBase, mergeDocCount, readerIdx);
}
private static int numDocs(int[] docBase, int mergedDocCount, int readerIndex) {
if (readerIndex == docBase.length - 1) {
return mergedDocCount - docBase[readerIndex];
}
return docBase[readerIndex + 1] - docBase[readerIndex];
}
BytesRef next() {
for (int i = relativeOrd + 1; i < ordMapping.length; i++) {
if (ordMapping[i] != 0) { // skip ords that are not referenced anymore
source.getByOrd(i, current);
relativeOrd = i;
return current;
}
}
return null;
}
/** Fills in the absolute ords for this slice.
*
* @return the provided {@code docToOrd} */
public int[] toAbsolutOrds(int[] docToOrd) {
for (int i = docToOrdStart; i < docToOrdEnd; i++) {
final int mappedOrd = docIDToRelativeOrd[i];
assert mappedOrd < ordMapping.length;
assert ordMapping[mappedOrd] > 0 : "illegal mapping ord maps to an unreferenced value";
docToOrd[i] = ordMapping[mappedOrd] -1;
}
return docToOrd;
}
/** Writes ords for this slice. */
public void writeOrds(PackedInts.Writer writer) throws IOException {
for (int i = docToOrdStart; i < docToOrdEnd; i++) {
final int mappedOrd = docIDToRelativeOrd[i];
assert mappedOrd < ordMapping.length;
assert ordMapping[mappedOrd] > 0 : "illegal mapping ord maps to an unreferenced value";
writer.add(ordMapping[mappedOrd] - 1);
}
}
}
/*
* if a segment has no values at all we use this source to fill in the missing
* value in the right place (depending on the comparator used)
*/
private static final class MissingValueSource extends SortedSource {
private BytesRef missingValue;
public MissingValueSource(MergeContext ctx) {
super(ctx.type, ctx.comp);
this.missingValue = ctx.missingValue;
}
@Override
public int ord(int docID) {
return 0;
}
@Override
public BytesRef getByOrd(int ord, BytesRef bytesRef) {
bytesRef.copyBytes(missingValue);
return bytesRef;
}
@Override
public PackedInts.Reader getDocToOrd() {
return null;
}
@Override
public int getValueCount() {
return 1;
}
}
/*
* merge queue
*/
private static final class MergeQueue extends
PriorityQueue<SortedSourceSlice> {
final Comparator<BytesRef> comp;
public MergeQueue(int maxSize, Comparator<BytesRef> comp) {
super(maxSize);
this.comp = comp;
}
@Override
protected boolean lessThan(SortedSourceSlice a, SortedSourceSlice b) {
int cmp = comp.compare(a.current, b.current);
if (cmp != 0) {
return cmp < 0;
} else { // just a tie-breaker
return a.docToOrdStart < b.docToOrdStart;
}
}
}
}

View File

@ -1,5 +1,7 @@
package org.apache.lucene.index;
import org.apache.lucene.index.FieldInfo.DocValuesType;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@ -27,5 +29,5 @@ public interface StorableFieldType {
/** DocValues type; if non-null then the field's value
* will be indexed into docValues */
public DocValues.Type docValueType();
public DocValuesType docValueType();
}

View File

@ -1,362 +0,0 @@
package org.apache.lucene.index;
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
import java.util.HashMap;
import java.util.Map;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.util.BytesRef;
// TODO: maybe we should not automagically promote
// types... and instead require a given field always has the
// same type?
/**
* Type promoter that promotes {@link DocValues} during merge based on their
* {@link Type} and {@link #getValueSize()}
*
* @lucene.internal
*/
class TypePromoter {
private final static Map<Integer,Type> FLAGS_MAP = new HashMap<Integer,Type>();
private static final TypePromoter IDENTITY_PROMOTER = new IdentityTypePromoter();
public static final int VAR_TYPE_VALUE_SIZE = -1;
private static final int IS_INT = 1 << 0 | 1 << 2;
private static final int IS_BYTE = 1 << 1;
private static final int IS_FLOAT = 1 << 2 ;
/* VAR & FIXED == VAR */
private static final int IS_VAR = 1 << 3;
private static final int IS_FIXED = 1 << 3 | 1 << 4;
/* if we have FIXED & FIXED with different size we promote to VAR */
private static final int PROMOTE_TO_VAR_SIZE_MASK = ~(1 << 3);
/* STRAIGHT & DEREF == STRAIGHT (dense values win) */
private static final int IS_STRAIGHT = 1 << 5;
private static final int IS_DEREF = 1 << 5 | 1 << 6;
private static final int IS_SORTED = 1 << 7;
/* more bits wins (int16 & int32 == int32) */
private static final int IS_8_BIT = 1 << 8 | 1 << 9 | 1 << 10 | 1 << 11 | 1 << 12 | 1 << 13; // 8
private static final int IS_16_BIT = 1 << 9 | 1 << 10 | 1 << 11 | 1 << 12 | 1 << 13; // 9
private static final int IS_32_BIT = 1 << 10 | 1 << 11 | 1 << 13;
private static final int IS_64_BIT = 1 << 11;
private static final int IS_32_BIT_FLOAT = 1 << 12 | 1 << 13;
private static final int IS_64_BIT_FLOAT = 1 << 13;
private Type type;
private int flags;
private int valueSize;
/**
* Returns a positive value size if this {@link TypePromoter} represents a
* fixed variant, otherwise <code>-1</code>
*
* @return a positive value size if this {@link TypePromoter} represents a
* fixed variant, otherwise <code>-1</code>
*/
public int getValueSize() {
return valueSize;
}
static {
for (Type type : Type.values()) {
TypePromoter create = create(type, VAR_TYPE_VALUE_SIZE);
FLAGS_MAP.put(create.flags, type);
}
}
/**
* Creates a new {@link TypePromoter}
*
*/
protected TypePromoter() {}
/**
* Creates a new {@link TypePromoter}
*
* @param type
* the {@link Type} this promoter represents
*
* @param flags
* the promoters flags
* @param valueSize
* the value size if {@link #IS_FIXED} or <code>-1</code> otherwise.
*/
protected TypePromoter(Type type, int flags, int valueSize) {
this.type = type;
this.flags = flags;
this.valueSize = valueSize;
}
/**
* Resets the {@link TypePromoter}
*
* @param type
* the {@link Type} this promoter represents
*
* @param flags
* the promoters flags
* @param valueSize
* the value size if {@link #IS_FIXED} or <code>-1</code> otherwise.
*/
protected TypePromoter set(Type type, int flags, int valueSize) {
this.type = type;
this.flags = flags;
this.valueSize = valueSize;
return this;
}
/**
* Creates a new promoted {@link TypePromoter} based on this and the given
* {@link TypePromoter} or <code>null</code> iff the {@link TypePromoter}
* aren't compatible.
*
* @param promoter
* the incoming promoter
* @return a new promoted {@link TypePromoter} based on this and the given
* {@link TypePromoter} or <code>null</code> iff the
* {@link TypePromoter} aren't compatible.
*/
public TypePromoter promote(TypePromoter promoter) {
return promote(promoter, newPromoter());
}
private TypePromoter promote(TypePromoter promoter, TypePromoter spare) {
int promotedFlags = promoter.flags & this.flags;
TypePromoter promoted = reset(FLAGS_MAP.get(promotedFlags), valueSize,
spare);
if (promoted == null) {
return TypePromoter.create(DocValues.Type.BYTES_VAR_STRAIGHT,
TypePromoter.VAR_TYPE_VALUE_SIZE);
}
if ((promoted.flags & IS_BYTE) != 0
&& (promoted.flags & IS_FIXED) == IS_FIXED) {
if (this.valueSize == promoter.valueSize) {
return promoted;
}
return reset(FLAGS_MAP.get(promoted.flags & PROMOTE_TO_VAR_SIZE_MASK),
VAR_TYPE_VALUE_SIZE, spare);
}
return promoted;
}
/**
* Returns the {@link Type} of this {@link TypePromoter}
*
* @return the {@link Type} of this {@link TypePromoter}
*/
public Type type() {
return type;
}
private boolean isTypeCompatible(TypePromoter promoter) {
int promotedFlags = promoter.flags & this.flags;
return (promotedFlags & 0x7) > 0;
}
private boolean isBytesCompatible(TypePromoter promoter) {
int promotedFlags = promoter.flags & this.flags;
return (promotedFlags & IS_BYTE) > 0
&& (promotedFlags & (IS_FIXED | IS_VAR)) > 0;
}
private boolean isNumericSizeCompatible(TypePromoter promoter) {
int promotedFlags = promoter.flags & this.flags;
return (promotedFlags & IS_BYTE) == 0
&& (((promotedFlags & IS_FIXED) > 0 && (promotedFlags & (IS_8_BIT)) > 0) || (promotedFlags & IS_VAR) > 0);
}
@Override
public String toString() {
return "TypePromoter [type=" + type + ", sizeInBytes=" + valueSize + "]";
}
@Override
public int hashCode() {
final int prime = 31;
int result = 1;
result = prime * result + flags;
result = prime * result + ((type == null) ? 0 : type.hashCode());
result = prime * result + valueSize;
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (obj == null) return false;
if (getClass() != obj.getClass()) return false;
TypePromoter other = (TypePromoter) obj;
if (flags != other.flags) return false;
if (type != other.type) return false;
if (valueSize != other.valueSize) return false;
return true;
}
/**
* Creates a new {@link TypePromoter} for the given type and size per value.
*
* @param type
* the {@link Type} to create the promoter for
* @param valueSize
* the size per value in bytes or <code>-1</code> iff the types have
* variable length.
* @return a new {@link TypePromoter}
*/
public static TypePromoter create(Type type, int valueSize) {
return reset(type, valueSize, new TypePromoter());
}
private static TypePromoter reset(Type type, int valueSize,
TypePromoter promoter) {
if (type == null) {
return null;
}
switch (type) {
case BYTES_FIXED_DEREF:
return promoter.set(type, IS_BYTE | IS_FIXED | IS_DEREF, valueSize);
case BYTES_FIXED_SORTED:
return promoter.set(type, IS_BYTE | IS_FIXED | IS_SORTED, valueSize);
case BYTES_FIXED_STRAIGHT:
return promoter.set(type, IS_BYTE | IS_FIXED | IS_STRAIGHT, valueSize);
case BYTES_VAR_DEREF:
return promoter.set(type, IS_BYTE | IS_VAR | IS_DEREF,
VAR_TYPE_VALUE_SIZE);
case BYTES_VAR_SORTED:
return promoter.set(type, IS_BYTE | IS_VAR | IS_SORTED,
VAR_TYPE_VALUE_SIZE);
case BYTES_VAR_STRAIGHT:
return promoter.set(type, IS_BYTE | IS_VAR | IS_STRAIGHT,
VAR_TYPE_VALUE_SIZE);
case FIXED_INTS_16:
return promoter.set(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_16_BIT,
valueSize);
case FIXED_INTS_32:
return promoter.set(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_32_BIT,
valueSize);
case FIXED_INTS_64:
return promoter.set(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_64_BIT,
valueSize);
case FIXED_INTS_8:
return promoter.set(type, IS_INT | IS_FIXED | IS_STRAIGHT | IS_8_BIT,
valueSize);
case FLOAT_32:
return promoter.set(type,
IS_FLOAT | IS_FIXED | IS_STRAIGHT | IS_32_BIT_FLOAT, valueSize);
case FLOAT_64:
return promoter.set(type,
IS_FLOAT | IS_FIXED | IS_STRAIGHT | IS_64_BIT_FLOAT, valueSize);
case VAR_INTS:
return promoter.set(type, IS_INT | IS_VAR | IS_STRAIGHT,
VAR_TYPE_VALUE_SIZE);
default:
throw new IllegalStateException();
}
}
public static int getValueSize(DocValues.Type type, BytesRef ref) {
switch (type) {
case VAR_INTS:
case BYTES_VAR_DEREF:
case BYTES_VAR_SORTED:
case BYTES_VAR_STRAIGHT:
return -1;
case BYTES_FIXED_DEREF:
case BYTES_FIXED_SORTED:
case BYTES_FIXED_STRAIGHT:
assert ref != null;
return ref.length;
case FIXED_INTS_16:
return 2;
case FLOAT_32:
case FIXED_INTS_32:
return 4;
case FLOAT_64:
case FIXED_INTS_64:
return 8;
case FIXED_INTS_8:
return 1;
default:
throw new IllegalArgumentException("unknonw docvalues type: "
+ type.name());
}
}
/**
* Returns a {@link TypePromoter} that always promotes to the type provided to
* {@link #promote(TypePromoter)}
*/
public static TypePromoter getIdentityPromoter() {
return IDENTITY_PROMOTER;
}
private static TypePromoter newPromoter() {
return new TypePromoter(null, 0, -1);
}
private static class IdentityTypePromoter extends TypePromoter {
public IdentityTypePromoter() {
super(null, 0, -1);
}
@Override
protected TypePromoter set(Type type, int flags, int valueSize) {
throw new UnsupportedOperationException("can not reset IdendityPromotoer");
}
@Override
public TypePromoter promote(TypePromoter promoter) {
return promoter;
}
}
static class TypeCompatibility {
private final TypePromoter base;
private final TypePromoter spare;
TypeCompatibility(Type type, int valueSize) {
this.base = create(type, valueSize);
spare = newPromoter();
}
boolean isCompatible(Type type, int valueSize) {
TypePromoter reset = reset(type, valueSize, spare);
if (base.isTypeCompatible(reset)) {
if (base.isBytesCompatible(reset)) {
return base.valueSize == -1 || base.valueSize == valueSize;
} else if (base.flags == reset.flags) {
return true;
} else if (base.isNumericSizeCompatible(reset)) {
return base.valueSize == -1
|| (base.valueSize > valueSize && valueSize > 0);
}
}
return false;
}
Type getBaseType() {
return base.type();
}
int getBaseSize() {
return base.valueSize;
}
}
}

View File

@ -20,9 +20,8 @@ package org.apache.lucene.search.similarities;
import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.Norm;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.TermStatistics;
@ -136,9 +135,9 @@ public class BM25Similarity extends Similarity {
@Override
public final void computeNorm(FieldInvertState state, Norm norm) {
public final long computeNorm(FieldInvertState state) {
final int numTerms = discountOverlaps ? state.getLength() - state.getNumOverlap() : state.getLength();
norm.setByte(encodeNormValue(state.getBoost(), numTerms));
return encodeNormValue(state.getBoost(), numTerms);
}
/**
@ -215,7 +214,7 @@ public class BM25Similarity extends Similarity {
@Override
public final ExactSimScorer exactSimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
BM25Stats bm25stats = (BM25Stats) stats;
final DocValues norms = context.reader().normValues(bm25stats.field);
final NumericDocValues norms = context.reader().simpleNormValues(bm25stats.field);
return norms == null
? new ExactBM25DocScorerNoNorms(bm25stats)
: new ExactBM25DocScorer(bm25stats, norms);
@ -224,26 +223,26 @@ public class BM25Similarity extends Similarity {
@Override
public final SloppySimScorer sloppySimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
BM25Stats bm25stats = (BM25Stats) stats;
return new SloppyBM25DocScorer(bm25stats, context.reader().normValues(bm25stats.field));
return new SloppyBM25DocScorer(bm25stats, context.reader().simpleNormValues(bm25stats.field));
}
private class ExactBM25DocScorer extends ExactSimScorer {
private final BM25Stats stats;
private final float weightValue;
private final byte[] norms;
private final NumericDocValues norms;
private final float[] cache;
ExactBM25DocScorer(BM25Stats stats, DocValues norms) throws IOException {
ExactBM25DocScorer(BM25Stats stats, NumericDocValues norms) throws IOException {
assert norms != null;
this.stats = stats;
this.weightValue = stats.weight * (k1 + 1); // boost * idf * (k1 + 1)
this.cache = stats.cache;
this.norms = (byte[])norms.getSource().getArray();
this.norms = norms;
}
@Override
public float score(int doc, int freq) {
return weightValue * freq / (freq + cache[norms[doc] & 0xFF]);
return weightValue * freq / (freq + cache[(byte)norms.get(doc) & 0xFF]);
}
@Override
@ -283,20 +282,20 @@ public class BM25Similarity extends Similarity {
private class SloppyBM25DocScorer extends SloppySimScorer {
private final BM25Stats stats;
private final float weightValue; // boost * idf * (k1 + 1)
private final byte[] norms;
private final NumericDocValues norms;
private final float[] cache;
SloppyBM25DocScorer(BM25Stats stats, DocValues norms) throws IOException {
SloppyBM25DocScorer(BM25Stats stats, NumericDocValues norms) throws IOException {
this.stats = stats;
this.weightValue = stats.weight * (k1 + 1);
this.cache = stats.cache;
this.norms = norms == null ? null : (byte[])norms.getSource().getArray();
this.norms = norms;
}
@Override
public float score(int doc, float freq) {
// if there are no norms, we act as if b=0
float norm = norms == null ? k1 : cache[norms[doc] & 0xFF];
float norm = norms == null ? k1 : cache[(byte)norms.get(doc) & 0xFF];
return weightValue * freq / (freq + norm);
}
@ -356,7 +355,7 @@ public class BM25Similarity extends Similarity {
}
}
private Explanation explainScore(int doc, Explanation freq, BM25Stats stats, byte[] norms) {
private Explanation explainScore(int doc, Explanation freq, BM25Stats stats, NumericDocValues norms) {
Explanation result = new Explanation();
result.setDescription("score(doc="+doc+",freq="+freq+"), product of:");
@ -374,7 +373,7 @@ public class BM25Similarity extends Similarity {
tfNormExpl.addDetail(new Explanation(0, "parameter b (norms omitted for field)"));
tfNormExpl.setValue((freq.getValue() * (k1 + 1)) / (freq.getValue() + k1));
} else {
float doclen = decodeNormValue(norms[doc]);
float doclen = decodeNormValue((byte)norms.get(doc));
tfNormExpl.addDetail(new Explanation(b, "parameter b"));
tfNormExpl.addDetail(new Explanation(stats.avgdl, "avgFieldLength"));
tfNormExpl.addDetail(new Explanation(doclen, "fieldLength"));

View File

@ -18,7 +18,6 @@ package org.apache.lucene.search.similarities;
*/
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.Norm;
import org.apache.lucene.util.BytesRef;
/** Expert: Default scoring implementation. */

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.Norm;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.TermStatistics;
@ -44,8 +43,8 @@ public class MultiSimilarity extends Similarity {
}
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
sims[0].computeNorm(state, norm);
public long computeNorm(FieldInvertState state) {
return sims[0].computeNorm(state);
}
@Override

View File

@ -21,7 +21,6 @@ import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.Norm;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.TermStatistics;
@ -42,8 +41,8 @@ public abstract class PerFieldSimilarityWrapper extends Similarity {
public PerFieldSimilarityWrapper() {}
@Override
public final void computeNorm(FieldInvertState state, Norm norm) {
get(state.getName()).computeNorm(state, norm);
public final long computeNorm(FieldInvertState state) {
return get(state.getName()).computeNorm(state);
}
@Override

View File

@ -23,9 +23,7 @@ import org.apache.lucene.document.ByteDocValuesField; // javadoc
import org.apache.lucene.document.FloatDocValuesField; // javadoc
import org.apache.lucene.index.AtomicReader; // javadoc
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.Norm;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.Explanation;
@ -53,9 +51,9 @@ import org.apache.lucene.util.SmallFloat; // javadoc
* <a href="#querytime">query-time</a>.
* <p>
* <a name="indextime"/>
* At indexing time, the indexer calls {@link #computeNorm(FieldInvertState, Norm)}, allowing
* At indexing time, the indexer calls {@link #computeNorm(FieldInvertState)}, allowing
* the Similarity implementation to set a per-document value for the field that will
* be later accessible via {@link AtomicReader#normValues(String)}. Lucene makes no assumption
* be later accessible via {@link AtomicReader#simpleNormValues(String)}. Lucene makes no assumption
* about what is in this norm, but it is most useful for encoding length normalization
* information.
* <p>
@ -72,7 +70,7 @@ import org.apache.lucene.util.SmallFloat; // javadoc
* Additional scoring factors can be stored in named
* <code>*DocValuesField</code>s (such as {@link
* ByteDocValuesField} or {@link FloatDocValuesField}), and accessed
* at query-time with {@link AtomicReader#docValues(String)}.
* at query-time with {@link AtomicReader#getNumericDocValues(String)}.
* <p>
* Finally, using index-time boosts (either via folding into the normalization byte or
* via DocValues), is an inefficient way to boost the scores of different fields if the
@ -150,9 +148,6 @@ public abstract class Similarity {
/**
* Computes the normalization value for a field, given the accumulated
* state of term processing for this field (see {@link FieldInvertState}).
*
* <p>Implementations should calculate a norm value based on the field
* state and set that value to the given {@link Norm}.
*
* <p>Matches in longer fields are less precise, so implementations of this
* method usually set smaller values when <code>state.getLength()</code> is large,
@ -161,21 +156,9 @@ public abstract class Similarity {
* @lucene.experimental
*
* @param state current processing state for this field
* @param norm holds the computed norm value when this method returns
* @return computed norm value
*/
public abstract void computeNorm(FieldInvertState state, Norm norm);
// nocommit replace computeNorm w/ this, make it abstract:
public long computeSimpleNorm(FieldInvertState state) {
Norm normScratch = new Norm();
computeNorm(state, normScratch);
if (normScratch.type() != null && DocValues.isNumber(normScratch.type())) {
return normScratch.field().numericValue().longValue();
} else {
// nocommit hmm how to return "not set"...?
return -1;
}
}
public abstract long computeNorm(FieldInvertState state);
/**
* Compute any collection-level weight (e.g. IDF, average document length, etc) needed for scoring a query.

View File

@ -20,9 +20,8 @@ package org.apache.lucene.search.similarities;
import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.Norm;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.Explanation;
import org.apache.lucene.search.TermStatistics;
@ -199,12 +198,12 @@ public abstract class SimilarityBase extends Similarity {
ExactSimScorer subScorers[] = new ExactSimScorer[subStats.length];
for (int i = 0; i < subScorers.length; i++) {
BasicStats basicstats = (BasicStats) subStats[i];
subScorers[i] = new BasicExactDocScorer(basicstats, context.reader().normValues(basicstats.field));
subScorers[i] = new BasicExactDocScorer(basicstats, context.reader().simpleNormValues(basicstats.field));
}
return new MultiSimilarity.MultiExactDocScorer(subScorers);
} else {
BasicStats basicstats = (BasicStats) stats;
return new BasicExactDocScorer(basicstats, context.reader().normValues(basicstats.field));
return new BasicExactDocScorer(basicstats, context.reader().simpleNormValues(basicstats.field));
}
}
@ -217,12 +216,12 @@ public abstract class SimilarityBase extends Similarity {
SloppySimScorer subScorers[] = new SloppySimScorer[subStats.length];
for (int i = 0; i < subScorers.length; i++) {
BasicStats basicstats = (BasicStats) subStats[i];
subScorers[i] = new BasicSloppyDocScorer(basicstats, context.reader().normValues(basicstats.field));
subScorers[i] = new BasicSloppyDocScorer(basicstats, context.reader().simpleNormValues(basicstats.field));
}
return new MultiSimilarity.MultiSloppyDocScorer(subScorers);
} else {
BasicStats basicstats = (BasicStats) stats;
return new BasicSloppyDocScorer(basicstats, context.reader().normValues(basicstats.field));
return new BasicSloppyDocScorer(basicstats, context.reader().simpleNormValues(basicstats.field));
}
}
@ -247,13 +246,13 @@ public abstract class SimilarityBase extends Similarity {
/** Encodes the document length in the same way as {@link TFIDFSimilarity}. */
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
public long computeNorm(FieldInvertState state) {
final float numTerms;
if (discountOverlaps)
numTerms = state.getLength() - state.getNumOverlap();
else
numTerms = state.getLength() / state.getBoost();
norm.setByte(encodeNormValue(state.getBoost(), numTerms));
return encodeNormValue(state.getBoost(), numTerms);
}
/** Decodes a normalization factor (document length) stored in an index.
@ -286,24 +285,24 @@ public abstract class SimilarityBase extends Similarity {
*/
private class BasicExactDocScorer extends ExactSimScorer {
private final BasicStats stats;
private final byte[] norms;
private final NumericDocValues norms;
BasicExactDocScorer(BasicStats stats, DocValues norms) throws IOException {
BasicExactDocScorer(BasicStats stats, NumericDocValues norms) throws IOException {
this.stats = stats;
this.norms = norms == null ? null : (byte[])norms.getSource().getArray();
this.norms = norms;
}
@Override
public float score(int doc, int freq) {
// We have to supply something in case norms are omitted
return SimilarityBase.this.score(stats, freq,
norms == null ? 1F : decodeNormValue(norms[doc]));
norms == null ? 1F : decodeNormValue((byte)norms.get(doc)));
}
@Override
public Explanation explain(int doc, Explanation freq) {
return SimilarityBase.this.explain(stats, doc, freq,
norms == null ? 1F : decodeNormValue(norms[doc]));
norms == null ? 1F : decodeNormValue((byte)norms.get(doc)));
}
}
@ -315,23 +314,23 @@ public abstract class SimilarityBase extends Similarity {
*/
private class BasicSloppyDocScorer extends SloppySimScorer {
private final BasicStats stats;
private final byte[] norms;
private final NumericDocValues norms;
BasicSloppyDocScorer(BasicStats stats, DocValues norms) throws IOException {
BasicSloppyDocScorer(BasicStats stats, NumericDocValues norms) throws IOException {
this.stats = stats;
this.norms = norms == null ? null : (byte[])norms.getSource().getArray();
this.norms = norms;
}
@Override
public float score(int doc, float freq) {
// We have to supply something in case norms are omitted
return SimilarityBase.this.score(stats, freq,
norms == null ? 1F : decodeNormValue(norms[doc]));
norms == null ? 1F : decodeNormValue((byte)norms.get(doc)));
}
@Override
public Explanation explain(int doc, Explanation freq) {
return SimilarityBase.this.explain(stats, doc, freq,
norms == null ? 1F : decodeNormValue(norms[doc]));
norms == null ? 1F : decodeNormValue((byte)norms.get(doc)));
}
@Override

View File

@ -17,13 +17,10 @@ package org.apache.lucene.search.similarities;
* limitations under the License.
*/
import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.Norm;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.Explanation;
@ -683,9 +680,9 @@ public abstract class TFIDFSimilarity extends Similarity {
public abstract float lengthNorm(FieldInvertState state);
@Override
public final void computeNorm(FieldInvertState state, Norm norm) {
public final long computeNorm(FieldInvertState state) {
float normValue = lengthNorm(state);
norm.setByte(encodeNormValue(normValue));
return encodeNormValue(normValue);
}
/** Cache of decoded bytes. */
@ -760,65 +757,28 @@ public abstract class TFIDFSimilarity extends Similarity {
@Override
public final ExactSimScorer exactSimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
IDFStats idfstats = (IDFStats) stats;
NumericDocValues normValues = context.reader().simpleNormValues(idfstats.field);
if (normValues != null) {
return new SimpleExactTFIDFDocScorer(idfstats, normValues);
} else {
return new ExactTFIDFDocScorer(idfstats, context.reader().normValues(idfstats.field));
}
return new ExactTFIDFDocScorer(idfstats, context.reader().simpleNormValues(idfstats.field));
}
@Override
public final SloppySimScorer sloppySimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
IDFStats idfstats = (IDFStats) stats;
return new SloppyTFIDFDocScorer(idfstats, context.reader().normValues(idfstats.field));
return new SloppyTFIDFDocScorer(idfstats, context.reader().simpleNormValues(idfstats.field));
}
// TODO: we can specialize these for omitNorms up front, but we should test that it doesn't confuse stupid hotspot.
private final class SimpleExactTFIDFDocScorer extends ExactSimScorer {
private final class ExactTFIDFDocScorer extends ExactSimScorer {
private final IDFStats stats;
private final float weightValue;
private final NumericDocValues norms;
private static final int SCORE_CACHE_SIZE = 32;
private float[] scoreCache = new float[SCORE_CACHE_SIZE];
SimpleExactTFIDFDocScorer(IDFStats stats, NumericDocValues norms) throws IOException {
ExactTFIDFDocScorer(IDFStats stats, NumericDocValues norms) throws IOException {
this.stats = stats;
this.weightValue = stats.value;
this.norms = norms;
for (int i = 0; i < SCORE_CACHE_SIZE; i++) {
scoreCache[i] = tf(i) * weightValue;
}
}
@Override
public float score(int doc, int freq) {
final float raw = // compute tf(f)*weight
freq < SCORE_CACHE_SIZE // check cache
? scoreCache[freq] // cache hit
: tf(freq)*weightValue; // cache miss
return norms == null ? raw : raw * decodeNormValue((byte) norms.get(doc)); // normalize for field
}
@Override
public Explanation explain(int doc, Explanation freq) {
return explainScore(doc, freq, stats, norms);
}
}
private final class ExactTFIDFDocScorer extends ExactSimScorer {
private final IDFStats stats;
private final float weightValue;
private final byte[] norms;
private static final int SCORE_CACHE_SIZE = 32;
private float[] scoreCache = new float[SCORE_CACHE_SIZE];
ExactTFIDFDocScorer(IDFStats stats, DocValues norms) throws IOException {
this.stats = stats;
this.weightValue = stats.value;
this.norms = norms == null ? null : (byte[])norms.getSource().getArray();
this.norms = norms;
for (int i = 0; i < SCORE_CACHE_SIZE; i++)
scoreCache[i] = tf(i) * weightValue;
}
@ -830,7 +790,7 @@ public abstract class TFIDFSimilarity extends Similarity {
? scoreCache[freq] // cache hit
: tf(freq)*weightValue; // cache miss
return norms == null ? raw : raw * decodeNormValue(norms[doc]); // normalize for field
return norms == null ? raw : raw * decodeNormValue((byte)norms.get(doc)); // normalize for field
}
@Override
@ -842,19 +802,19 @@ public abstract class TFIDFSimilarity extends Similarity {
private final class SloppyTFIDFDocScorer extends SloppySimScorer {
private final IDFStats stats;
private final float weightValue;
private final byte[] norms;
private final NumericDocValues norms;
SloppyTFIDFDocScorer(IDFStats stats, DocValues norms) throws IOException {
SloppyTFIDFDocScorer(IDFStats stats, NumericDocValues norms) throws IOException {
this.stats = stats;
this.weightValue = stats.value;
this.norms = norms == null ? null : (byte[])norms.getSource().getArray();
this.norms = norms;
}
@Override
public float score(int doc, float freq) {
final float raw = tf(freq) * weightValue; // compute tf(f)*weight
return norms == null ? raw : raw * decodeNormValue(norms[doc]); // normalize for field
return norms == null ? raw : raw * decodeNormValue((byte)norms.get(doc)); // normalize for field
}
@Override
@ -904,63 +864,7 @@ public abstract class TFIDFSimilarity extends Similarity {
queryWeight *= this.queryNorm; // normalize query weight
value = queryWeight * idf.getValue(); // idf for document
}
}
private Explanation explainScore(int doc, Explanation freq, IDFStats stats, byte[] norms) {
Explanation result = new Explanation();
result.setDescription("score(doc="+doc+",freq="+freq+"), product of:");
// explain query weight
Explanation queryExpl = new Explanation();
queryExpl.setDescription("queryWeight, product of:");
Explanation boostExpl = new Explanation(stats.queryBoost, "boost");
if (stats.queryBoost != 1.0f)
queryExpl.addDetail(boostExpl);
queryExpl.addDetail(stats.idf);
Explanation queryNormExpl = new Explanation(stats.queryNorm,"queryNorm");
queryExpl.addDetail(queryNormExpl);
queryExpl.setValue(boostExpl.getValue() *
stats.idf.getValue() *
queryNormExpl.getValue());
result.addDetail(queryExpl);
// explain field weight
Explanation fieldExpl = new Explanation();
fieldExpl.setDescription("fieldWeight in "+doc+
", product of:");
Explanation tfExplanation = new Explanation();
tfExplanation.setValue(tf(freq.getValue()));
tfExplanation.setDescription("tf(freq="+freq.getValue()+"), with freq of:");
tfExplanation.addDetail(freq);
fieldExpl.addDetail(tfExplanation);
fieldExpl.addDetail(stats.idf);
Explanation fieldNormExpl = new Explanation();
float fieldNorm =
norms!=null ? decodeNormValue(norms[doc]) : 1.0f;
fieldNormExpl.setValue(fieldNorm);
fieldNormExpl.setDescription("fieldNorm(doc="+doc+")");
fieldExpl.addDetail(fieldNormExpl);
fieldExpl.setValue(tfExplanation.getValue() *
stats.idf.getValue() *
fieldNormExpl.getValue());
result.addDetail(fieldExpl);
// combine them
result.setValue(queryExpl.getValue() * fieldExpl.getValue());
if (queryExpl.getValue() == 1.0f)
return fieldExpl;
return result;
}
}
private Explanation explainScore(int doc, Explanation freq, IDFStats stats, NumericDocValues norms) {
Explanation result = new Explanation();

View File

@ -17,25 +17,17 @@ package org.apache.lucene.index;
* limitations under the License.
*/
import java.io.IOException;
import java.util.Random;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.TermStatistics;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.search.similarities.PerFieldSimilarityWrapper;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.search.similarities.Similarity.ExactSimScorer;
import org.apache.lucene.search.similarities.Similarity.SimWeight;
import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
@ -72,106 +64,18 @@ public class TestCustomNorms extends LuceneTestCase {
writer.commit();
writer.close();
AtomicReader open = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
DocValues normValues = open.normValues(floatTestField);
assertNotNull(normValues);
Source source = normValues.getSource();
assertTrue(source.hasArray());
assertEquals(Type.FLOAT_32, normValues.getType());
float[] norms = (float[]) source.getArray();
NumericDocValues norms = open.simpleNormValues(floatTestField);
assertNotNull(norms);
for (int i = 0; i < open.maxDoc(); i++) {
StoredDocument document = open.document(i);
float expected = Float.parseFloat(document.get(floatTestField));
assertEquals(expected, norms[i], 0.0f);
assertEquals(expected, Float.intBitsToFloat((int)norms.get(i)), 0.0f);
}
open.close();
dir.close();
docs.close();
}
public void testExceptionOnRandomType() throws IOException {
Directory dir = newDirectory();
IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random()));
Similarity provider = new MySimProvider();
config.setSimilarity(provider);
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, config);
final LineFileDocs docs = new LineFileDocs(random());
int num = atLeast(100);
try {
for (int i = 0; i < num; i++) {
Document doc = docs.nextDoc();
float nextFloat = random().nextFloat();
Field f = new TextField(exceptionTestField, "" + nextFloat, Field.Store.YES);
f.setBoost(nextFloat);
doc.add(f);
writer.addDocument(doc);
doc.removeField(exceptionTestField);
if (rarely()) {
writer.commit();
}
}
fail("expected exception - incompatible types");
} catch (IllegalArgumentException e) {
// expected
}
writer.commit();
writer.close();
dir.close();
docs.close();
}
public void testIllegalCustomEncoder() throws Exception {
Directory dir = newDirectory();
IllegalCustomEncodingSimilarity similarity = new IllegalCustomEncodingSimilarity();
IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
config.setSimilarity(similarity);
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, config);
Document doc = new Document();
Field foo = newTextField("foo", "", Field.Store.NO);
Field bar = newTextField("bar", "", Field.Store.NO);
doc.add(foo);
doc.add(bar);
int numAdded = 0;
for (int i = 0; i < 100; i++) {
try {
bar.setStringValue("singleton");
similarity.useByte = random().nextBoolean();
writer.addDocument(doc);
numAdded++;
} catch (IllegalArgumentException e) {}
}
IndexReader reader = writer.getReader();
writer.close();
assertEquals(numAdded, reader.numDocs());
IndexReaderContext topReaderContext = reader.getContext();
for (final AtomicReaderContext ctx : topReaderContext.leaves()) {
AtomicReader atomicReader = ctx.reader();
Source source = random().nextBoolean() ? atomicReader.normValues("foo").getSource() : atomicReader.normValues("foo").getDirectSource();
Bits liveDocs = atomicReader.getLiveDocs();
Type t = source.getType();
for (int i = 0; i < atomicReader.maxDoc(); i++) {
assertEquals(0, source.getFloat(i), 0.000f);
}
source = random().nextBoolean() ? atomicReader.normValues("bar").getSource() : atomicReader.normValues("bar").getDirectSource();
for (int i = 0; i < atomicReader.maxDoc(); i++) {
if (liveDocs == null || liveDocs.get(i)) {
assertEquals("type: " + t, 1, source.getFloat(i), 0.000f);
} else {
assertEquals("type: " + t, 0, source.getFloat(i), 0.000f);
}
}
}
reader.close();
dir.close();
}
public class MySimProvider extends PerFieldSimilarityWrapper {
Similarity delegate = new DefaultSimilarity();
@ -184,8 +88,6 @@ public class TestCustomNorms extends LuceneTestCase {
public Similarity get(String field) {
if (floatTestField.equals(field)) {
return new FloatEncodingBoostSimilarity();
} else if (exceptionTestField.equals(field)) {
return new RandomTypeSimilarity(random());
} else {
return delegate;
}
@ -200,9 +102,8 @@ public class TestCustomNorms extends LuceneTestCase {
public static class FloatEncodingBoostSimilarity extends Similarity {
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
float boost = state.getBoost();
norm.setFloat(boost);
public long computeNorm(FieldInvertState state) {
return Float.floatToIntBits(state.getBoost());
}
@Override
@ -220,87 +121,4 @@ public class TestCustomNorms extends LuceneTestCase {
throw new UnsupportedOperationException();
}
}
public static class RandomTypeSimilarity extends Similarity {
private final Random random;
public RandomTypeSimilarity(Random random) {
this.random = random;
}
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
float boost = state.getBoost();
int nextInt = random.nextInt(10);
switch (nextInt) {
case 0:
norm.setDouble((double) boost);
break;
case 1:
norm.setFloat(boost);
break;
case 2:
norm.setLong((long) boost);
break;
case 3:
norm.setBytes(new BytesRef(new byte[6]));
break;
case 4:
norm.setInt((int) boost);
break;
case 5:
norm.setShort((short) boost);
break;
default:
norm.setByte((byte) boost);
}
}
@Override
public SimWeight computeWeight(float queryBoost, CollectionStatistics collectionStats, TermStatistics... termStats) {
throw new UnsupportedOperationException();
}
@Override
public ExactSimScorer exactSimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public SloppySimScorer sloppySimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
throw new UnsupportedOperationException();
}
}
class IllegalCustomEncodingSimilarity extends Similarity {
public boolean useByte = false;
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
if (useByte) {
norm.setByte((byte)state.getLength());
} else {
norm.setFloat((float)state.getLength());
}
}
@Override
public SimWeight computeWeight(float queryBoost, CollectionStatistics collectionStats, TermStatistics... termStats) {
throw new UnsupportedOperationException();
}
@Override
public ExactSimScorer exactSimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
throw new UnsupportedOperationException();
}
@Override
public SloppySimScorer sloppySimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
throw new UnsupportedOperationException();
}
}
}

View File

@ -546,7 +546,6 @@ public void testFilesOpenClose() throws IOException {
// TODO: maybe this can reuse the logic of test dueling codecs?
public static void assertIndexEquals(DirectoryReader index1, DirectoryReader index2) throws IOException {
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
assertEquals("IndexReaders have different values for numDocs.", index1.numDocs(), index2.numDocs());
assertEquals("IndexReaders have different values for maxDoc.", index1.maxDoc(), index2.maxDoc());
assertEquals("Only one IndexReader has deletions.", index1.hasDeletions(), index2.hasDeletions());

View File

@ -39,12 +39,10 @@ import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.Assume;
public class TestDirectoryReaderReopen extends LuceneTestCase {
public void testReopen() throws Exception {
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
final Directory dir1 = newDirectory();
createIndex(random(), dir1, false);
@ -196,7 +194,6 @@ public class TestDirectoryReaderReopen extends LuceneTestCase {
}
public void testThreadSafety() throws Exception {
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
final Directory dir = newDirectory();
// NOTE: this also controls the number of threads!
final int n = _TestUtil.nextInt(random(), 20, 40);

View File

@ -97,7 +97,7 @@ public class TestDocumentWriter extends LuceneTestCase {
// omitNorms is true
for (FieldInfo fi : reader.getFieldInfos()) {
if (fi.isIndexed()) {
assertTrue(fi.omitsNorms() == (reader.normValues(fi.name) == null));
assertTrue(fi.omitsNorms() == (reader.simpleNormValues(fi.name) == null));
}
}
reader.close();

View File

@ -38,11 +38,9 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.automaton.AutomatonTestUtil;
import org.apache.lucene.util.automaton.CompiledAutomaton;
import org.apache.lucene.util.automaton.RegExp;
import org.junit.Assume;
/**
* Compares one codec against another
@ -68,7 +66,6 @@ public class TestDuelingCodecs extends LuceneTestCase {
leftCodec = Codec.forName("SimpleText");
rightCodec = new RandomCodec(random());
Assume.assumeTrue(rightCodec.simpleNormsFormat() != null);
leftDir = newDirectory();
rightDir = newDirectory();
@ -519,7 +516,6 @@ public class TestDuelingCodecs extends LuceneTestCase {
* checks that norms are the same across all fields
*/
public void assertNorms(IndexReader leftReader, IndexReader rightReader) throws Exception {
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
Fields leftFields = MultiFields.getFields(leftReader);
Fields rightFields = MultiFields.getFields(rightReader);
// Fields could be null if there are no postings,

View File

@ -25,7 +25,7 @@ import java.util.Iterator;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.index.FieldInfo.DocValuesType;
import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.DocIdSetIterator;
@ -89,7 +89,7 @@ public class TestIndexableField extends LuceneTestCase {
}
@Override
public Type docValueType() {
public DocValuesType docValueType() {
return null;
}
};

View File

@ -30,7 +30,6 @@ import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.Assume;
/**
* Tests the maxTermFrequency statistic in FieldInvertState
@ -68,8 +67,6 @@ public class TestMaxTermFrequency extends LuceneTestCase {
}
public void test() throws Exception {
// nocommit remove
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
NumericDocValues fooNorms = MultiSimpleDocValues.simpleNormValues(reader, "foo");
for (int i = 0; i < reader.maxDoc(); i++) {
assertEquals(expected.get(i).intValue(), fooNorms.get(i) & 0xff);

View File

@ -24,8 +24,6 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.DocValues.Source;
import org.apache.lucene.index.DocValues.Type;
import org.apache.lucene.search.CollectionStatistics;
import org.apache.lucene.search.TermStatistics;
import org.apache.lucene.search.similarities.DefaultSimilarity;
@ -37,7 +35,6 @@ import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.Assume;
/**
* Test that norms info is preserved during index life - including
@ -68,8 +65,6 @@ public class TestNorms extends LuceneTestCase {
// LUCENE-1260
public void testCustomEncoder() throws Exception {
// nocommit remove:
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
Directory dir = newDirectory();
IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
config.setSimilarity(new CustomNormEncodingSimilarity());
@ -104,99 +99,33 @@ public class TestNorms extends LuceneTestCase {
public void testMaxByteNorms() throws IOException {
Directory dir = newFSDirectory(_TestUtil.getTempDir("TestNorms.testMaxByteNorms"));
buildIndex(dir, true);
buildIndex(dir);
AtomicReader open = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dir));
DocValues normValues = open.normValues(byteTestField);
NumericDocValues normValues = open.simpleNormValues(byteTestField);
assertNotNull(normValues);
Source source = normValues.getSource();
assertTrue(source.hasArray());
assertEquals(Type.FIXED_INTS_8, normValues.getType());
byte[] norms = (byte[]) source.getArray();
for (int i = 0; i < open.maxDoc(); i++) {
StoredDocument document = open.document(i);
int expected = Integer.parseInt(document.get(byteTestField));
assertEquals((byte)expected, norms[i]);
assertEquals((byte)expected, normValues.get(i));
}
open.close();
dir.close();
}
/**
* this test randomly creates segments with or without norms but not omitting
* norms. The similarity used doesn't write a norm value if writeNorms = false is
* passed. This differs from omitNorm since norms are simply not written for this segment
* while merging fills in default values based on the Norm {@link Type}
*/
public void testNormsNotPresent() throws IOException {
Directory dir = newFSDirectory(_TestUtil.getTempDir("TestNorms.testNormsNotPresent.1"));
boolean firstWriteNorm = random().nextBoolean();
buildIndex(dir, firstWriteNorm);
// TODO: create a testNormsNotPresent ourselves by adding/deleting/merging docs
Directory otherDir = newFSDirectory(_TestUtil.getTempDir("TestNorms.testNormsNotPresent.2"));
boolean secondWriteNorm = random().nextBoolean();
buildIndex(otherDir, secondWriteNorm);
AtomicReader reader = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(otherDir));
FieldInfos fieldInfos = reader.getFieldInfos();
FieldInfo fieldInfo = fieldInfos.fieldInfo(byteTestField);
assertFalse(fieldInfo.omitsNorms());
assertTrue(fieldInfo.isIndexed());
if (secondWriteNorm) {
assertTrue(fieldInfo.hasNorms());
} else {
assertFalse(fieldInfo.hasNorms());
}
IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random()));
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, config);
writer.addIndexes(reader);
AtomicReader mergedReader = SlowCompositeReaderWrapper.wrap(writer.getReader());
if (!firstWriteNorm && !secondWriteNorm) {
DocValues normValues = mergedReader.normValues(byteTestField);
assertNull(normValues);
FieldInfo fi = mergedReader.getFieldInfos().fieldInfo(byteTestField);
assertFalse(fi.omitsNorms());
assertTrue(fi.isIndexed());
assertFalse(fi.hasNorms());
} else {
FieldInfo fi = mergedReader.getFieldInfos().fieldInfo(byteTestField);
assertFalse(fi.omitsNorms());
assertTrue(fi.isIndexed());
assertTrue(fi.hasNorms());
DocValues normValues = mergedReader.normValues(byteTestField);
assertNotNull(normValues);
Source source = normValues.getSource();
assertTrue(source.hasArray());
assertEquals(Type.FIXED_INTS_8, normValues.getType());
byte[] norms = (byte[]) source.getArray();
for (int i = 0; i < mergedReader.maxDoc(); i++) {
StoredDocument document = mergedReader.document(i);
int expected = Integer.parseInt(document.get(byteTestField));
assertEquals((byte) expected, norms[i]);
}
}
mergedReader.close();
reader.close();
writer.close();
dir.close();
otherDir.close();
}
public void buildIndex(Directory dir, boolean writeNorms) throws IOException {
public void buildIndex(Directory dir) throws IOException {
Random random = random();
IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random()));
Similarity provider = new MySimProvider(writeNorms);
Similarity provider = new MySimProvider();
config.setSimilarity(provider);
RandomIndexWriter writer = new RandomIndexWriter(random, dir, config);
final LineFileDocs docs = new LineFileDocs(random, true);
int num = atLeast(100);
for (int i = 0; i < num; i++) {
Document doc = docs.nextDoc();
int boost = writeNorms ? 1 + random().nextInt(255) : 0;
int boost = random().nextInt(255);
Field f = new TextField(byteTestField, "" + boost, Field.Store.YES);
f.setBoost(boost);
doc.add(f);
@ -214,10 +143,7 @@ public class TestNorms extends LuceneTestCase {
public class MySimProvider extends PerFieldSimilarityWrapper {
Similarity delegate = new DefaultSimilarity();
private boolean writeNorms;
public MySimProvider(boolean writeNorms) {
this.writeNorms = writeNorms;
}
@Override
public float queryNorm(float sumOfSquaredWeights) {
@ -227,7 +153,7 @@ public class TestNorms extends LuceneTestCase {
@Override
public Similarity get(String field) {
if (byteTestField.equals(field)) {
return new ByteEncodingBoostSimilarity(writeNorms);
return new ByteEncodingBoostSimilarity();
} else {
return delegate;
}
@ -242,18 +168,10 @@ public class TestNorms extends LuceneTestCase {
public static class ByteEncodingBoostSimilarity extends Similarity {
private boolean writeNorms;
public ByteEncodingBoostSimilarity(boolean writeNorms) {
this.writeNorms = writeNorms;
}
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
if (writeNorms) {
int boost = (int) state.getBoost();
norm.setByte((byte) (0xFF & boost));
}
public long computeNorm(FieldInvertState state) {
int boost = (int) state.getBoost();
return (0xFF & boost);
}
@Override

View File

@ -28,7 +28,6 @@ import org.apache.lucene.document.TextField;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.Assume;
public class TestOmitNorms extends LuceneTestCase {
// Tests whether the DocumentWriter correctly enable the
@ -267,8 +266,6 @@ public class TestOmitNorms extends LuceneTestCase {
* returns the norms for "field".
*/
NumericDocValues getNorms(String field, Field f1, Field f2) throws IOException {
// nocommit remove
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
Directory dir = newDirectory();
IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random())).setMergePolicy(newLogMergePolicy());
RandomIndexWriter riw = new RandomIndexWriter(random(), dir, iwc);

View File

@ -30,8 +30,6 @@ import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.apache.lucene.util.packed.PackedInts;
import org.junit.Assume;
public class TestSegmentMerger extends LuceneTestCase {
//The variables for the new merged segment
@ -79,7 +77,6 @@ public class TestSegmentMerger extends LuceneTestCase {
}
public void testMerge() throws IOException {
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
final Codec codec = Codec.getDefault();
final SegmentInfo si = new SegmentInfo(mergedDir, Constants.LUCENE_MAIN_VERSION, mergedSegment, -1, false, codec, null, null);

View File

@ -179,9 +179,9 @@ public class TestSegmentReader extends LuceneTestCase {
for (int i=0; i<DocHelper.fields.length; i++) {
IndexableField f = DocHelper.fields[i];
if (f.fieldType().indexed()) {
assertEquals(reader.normValues(f.name()) != null, !f.fieldType().omitNorms());
assertEquals(reader.normValues(f.name()) != null, !DocHelper.noNorms.containsKey(f.name()));
if (reader.normValues(f.name()) == null) {
assertEquals(reader.simpleNormValues(f.name()) != null, !f.fieldType().omitNorms());
assertEquals(reader.simpleNormValues(f.name()) != null, !DocHelper.noNorms.containsKey(f.name()));
if (reader.simpleNormValues(f.name()) == null) {
// test for norms of null
NumericDocValues norms = MultiSimpleDocValues.simpleNormValues(reader, f.name());
assertNull(norms);

View File

@ -80,7 +80,6 @@ public class TestSimpleDocValuesIndexing extends LuceneTestCase {
* Simple test case to show how to use the API
*/
public void testDocValuesSimple() throws IOException {
Assume.assumeTrue(_TestUtil.canUseSimpleDV());
Directory dir = newDirectory();
IndexWriter writer = new IndexWriter(dir, writerConfig(false));
for (int i = 0; i < 5; i++) {

View File

@ -31,7 +31,6 @@ import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
import org.junit.Assume;
/**
* Tests the uniqueTermCount statistic in FieldInvertState
@ -46,8 +45,9 @@ public class TestUniqueTermCount extends LuceneTestCase {
public void setUp() throws Exception {
super.setUp();
dir = newDirectory();
IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT,
new MockAnalyzer(random(), MockTokenizer.SIMPLE, true)).setMergePolicy(newLogMergePolicy());
MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
IndexWriterConfig config = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
config.setMergePolicy(newLogMergePolicy());
config.setSimilarity(new TestSimilarity());
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, config);
Document doc = new Document();
@ -69,10 +69,9 @@ public class TestUniqueTermCount extends LuceneTestCase {
}
public void test() throws Exception {
Assume.assumeTrue(_TestUtil.canUseSimpleNorms());
NumericDocValues fooNorms = MultiSimpleDocValues.simpleNormValues(reader, "foo");
for (int i = 0; i < reader.maxDoc(); i++) {
assertEquals(expected.get(i).intValue(), fooNorms.get(i) & 0xff);
assertEquals(expected.get(i).longValue(), fooNorms.get(i));
}
}
@ -95,13 +94,13 @@ public class TestUniqueTermCount extends LuceneTestCase {
}
/**
* Simple similarity that encodes maxTermFrequency directly as a byte
* Simple similarity that encodes maxTermFrequency directly
*/
class TestSimilarity extends Similarity {
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
norm.setByte((byte) state.getUniqueTermCount());
public long computeNorm(FieldInvertState state) {
return state.getUniqueTermCount();
}
@Override

View File

@ -20,7 +20,6 @@ package org.apache.lucene.search;
import java.io.IOException;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.Norm;
import org.apache.lucene.search.similarities.Similarity;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@ -255,7 +254,7 @@ final class JustCompileSearch {
}
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
public long computeNorm(FieldInvertState state) {
throw new UnsupportedOperationException(UNSUPPORTED_MSG);
}
}

View File

@ -29,7 +29,6 @@ import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.Norm;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.similarities.Similarity;
@ -90,8 +89,8 @@ public class TestConjunctions extends LuceneTestCase {
private static class TFSimilarity extends Similarity {
@Override
public void computeNorm(FieldInvertState state, Norm norm) {
norm.setByte((byte)1); // we dont care
public long computeNorm(FieldInvertState state) {
return 1; // we dont care
}
@Override

View File

@ -25,7 +25,6 @@ import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.AtomicReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Norm;
import org.apache.lucene.index.SlowCompositeReaderWrapper;
import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.RandomIndexWriter;

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