SOLR-9589: Remove jackson dependency from SolrJ
This commit is contained in:
Noble Paul 2016-10-04 19:15:50 +05:30
commit 5fbf135b80
42 changed files with 4472 additions and 735 deletions

View File

@ -29,7 +29,7 @@ import org.apache.lucene.benchmark.byTask.PerfRunData;
import org.apache.lucene.benchmark.byTask.utils.Config;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexDeletionPolicy;
@ -139,7 +139,7 @@ public class CreateIndexTask extends PerfTask {
if (defaultCodec == null && postingsFormat != null) {
try {
final PostingsFormat postingsFormatChosen = PostingsFormat.forName(postingsFormat);
iwConf.setCodec(new Lucene62Codec() {
iwConf.setCodec(new Lucene70Codec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return postingsFormatChosen;

View File

@ -57,7 +57,7 @@ public abstract class Codec implements NamedSPILoader.NamedSPI {
}
// TODO: should we use this, or maybe a system property is better?
static Codec defaultCodec = LOADER.lookup("Lucene62");
static Codec defaultCodec = LOADER.lookup("Lucene70");
}
private final String name;

View File

@ -22,6 +22,7 @@ import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
@ -95,7 +96,9 @@ import org.apache.lucene.util.packed.DirectWriter;
* <li><tt>.dvm</tt>: DocValues metadata</li>
* </ol>
* @lucene.experimental
* @deprecated Use {@link Lucene70DocValuesFormat}.
*/
@Deprecated
public final class Lucene54DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */

View File

@ -0,0 +1,177 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import java.util.Objects;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
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.PointsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene53.Lucene53NormsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.codecs.lucene60.Lucene60PointsFormat;
import org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
/**
* Implements the Lucene 7.0 index format, with configurable per-field postings
* and docvalues formats.
* <p>
* If you want to reuse functionality of this codec in another codec, extend
* {@link FilterCodec}.
*
* @see org.apache.lucene.codecs.lucene70 package documentation for file format details.
*
* @lucene.experimental
*/
public class Lucene70Codec extends Codec {
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene62SegmentInfoFormat();
private final LiveDocsFormat liveDocsFormat = new Lucene50LiveDocsFormat();
private final CompoundFormat compoundFormat = new Lucene50CompoundFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return Lucene70Codec.this.getPostingsFormatForField(field);
}
};
private final DocValuesFormat docValuesFormat = new PerFieldDocValuesFormat() {
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return Lucene70Codec.this.getDocValuesFormatForField(field);
}
};
private final StoredFieldsFormat storedFieldsFormat;
/**
* Instantiates a new codec.
*/
public Lucene70Codec() {
this(Mode.BEST_SPEED);
}
/**
* Instantiates a new codec, specifying the stored fields compression
* mode to use.
* @param mode stored fields compression mode to use for newly
* flushed/merged segments.
*/
public Lucene70Codec(Mode mode) {
super("Lucene70");
this.storedFieldsFormat = new Lucene50StoredFieldsFormat(Objects.requireNonNull(mode));
}
@Override
public final StoredFieldsFormat storedFieldsFormat() {
return storedFieldsFormat;
}
@Override
public final TermVectorsFormat termVectorsFormat() {
return vectorsFormat;
}
@Override
public final PostingsFormat postingsFormat() {
return postingsFormat;
}
@Override
public final FieldInfosFormat fieldInfosFormat() {
return fieldInfosFormat;
}
@Override
public final SegmentInfoFormat segmentInfoFormat() {
return segmentInfosFormat;
}
@Override
public final LiveDocsFormat liveDocsFormat() {
return liveDocsFormat;
}
@Override
public final CompoundFormat compoundFormat() {
return compoundFormat;
}
@Override
public final PointsFormat pointsFormat() {
return new Lucene60PointsFormat();
}
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene50".
* <p>
* <b>WARNING:</b> if you subclass, you are responsible for index
* backwards compatibility: future version of Lucene are only
* guaranteed to be able to read the default implementation.
*/
public PostingsFormat getPostingsFormatForField(String field) {
return defaultFormat;
}
/** Returns the docvalues format that should be used for writing
* new segments of <code>field</code>.
*
* The default implementation always returns "Lucene70".
* <p>
* <b>WARNING:</b> if you subclass, you are responsible for index
* backwards compatibility: future version of Lucene are only
* guaranteed to be able to read the default implementation.
*/
public DocValuesFormat getDocValuesFormatForField(String field) {
return defaultDVFormat;
}
@Override
public final DocValuesFormat docValuesFormat() {
return docValuesFormat;
}
private final PostingsFormat defaultFormat = PostingsFormat.forName("Lucene50");
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene70");
private final NormsFormat normsFormat = new Lucene53NormsFormat();
@Override
public final NormsFormat normsFormat() {
return normsFormat;
}
}

View File

@ -0,0 +1,797 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import java.io.Closeable; // javadocs
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.SortedSet;
import java.util.TreeSet;
import java.util.stream.StreamSupport;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.codecs.LegacyDocValuesIterables;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LongsRef;
import org.apache.lucene.util.MathUtil;
import org.apache.lucene.util.PagedBytes.PagedBytesDataInput;
import org.apache.lucene.util.PagedBytes;
import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.packed.DirectMonotonicWriter;
import org.apache.lucene.util.packed.DirectWriter;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
import static org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat.*;
/** writer for {@link Lucene70DocValuesFormat} */
final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Closeable {
enum NumberType {
/** Dense ordinals */
ORDINAL,
/** Random long values */
VALUE;
}
IndexOutput data, meta;
final int maxDoc;
/** expert: Creates a new writer */
public Lucene70DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
boolean success = false;
try {
String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
data = state.directory.createOutput(dataName, state.context);
CodecUtil.writeIndexHeader(data, dataCodec, Lucene70DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, metaExtension);
meta = state.directory.createOutput(metaName, state.context);
CodecUtil.writeIndexHeader(meta, metaCodec, Lucene70DocValuesFormat.VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix);
maxDoc = state.segmentInfo.maxDoc();
success = true;
} finally {
if (!success) {
IOUtils.closeWhileHandlingException(this);
}
}
}
@Override
public void addNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
addNumericField(field, LegacyDocValuesIterables.numericIterable(field, valuesProducer, maxDoc), NumberType.VALUE);
}
void addNumericField(FieldInfo field, Iterable<Number> values, NumberType numberType) throws IOException {
long count = 0;
long minValue = Long.MAX_VALUE;
long maxValue = Long.MIN_VALUE;
long gcd = 0;
long missingCount = 0;
long zeroCount = 0;
// TODO: more efficient?
HashSet<Long> uniqueValues = null;
long missingOrdCount = 0;
if (numberType == NumberType.VALUE) {
uniqueValues = new HashSet<>();
for (Number nv : values) {
final long v;
if (nv == null) {
v = 0;
missingCount++;
zeroCount++;
} else {
v = nv.longValue();
if (v == 0) {
zeroCount++;
}
}
if (gcd != 1) {
if (v < Long.MIN_VALUE / 2 || v > Long.MAX_VALUE / 2) {
// in that case v - minValue might overflow and make the GCD computation return
// wrong results. Since these extreme values are unlikely, we just discard
// GCD computation for them
gcd = 1;
} else if (count != 0) { // minValue needs to be set first
gcd = MathUtil.gcd(gcd, v - minValue);
}
}
minValue = Math.min(minValue, v);
maxValue = Math.max(maxValue, v);
if (uniqueValues != null) {
if (uniqueValues.add(v)) {
if (uniqueValues.size() > 256) {
uniqueValues = null;
}
}
}
++count;
}
} else {
for (Number nv : values) {
long v = nv.longValue();
if (v == -1L) {
missingOrdCount++;
}
minValue = Math.min(minValue, v);
maxValue = Math.max(maxValue, v);
++count;
}
}
final long delta = maxValue - minValue;
final int deltaBitsRequired = DirectWriter.unsignedBitsRequired(delta);
final int tableBitsRequired = uniqueValues == null
? Integer.MAX_VALUE
: DirectWriter.bitsRequired(uniqueValues.size() - 1);
final boolean sparse; // 10% of docs or less have a value
switch (numberType) {
case VALUE:
sparse = (double) missingCount / count >= 0.90;
break;
case ORDINAL:
sparse = (double) missingOrdCount / count >= 0.90;
break;
default:
throw new AssertionError();
}
final int format;
if (uniqueValues != null
&& count <= Integer.MAX_VALUE
&& (uniqueValues.size() == 1
|| (uniqueValues.size() == 2 && missingCount > 0 && zeroCount == missingCount))) {
// either one unique value C or two unique values: "missing" and C
format = CONST_COMPRESSED;
} else if (sparse && count >= 1024) {
// require at least 1024 docs to avoid flipping back and forth when doing NRT search
format = SPARSE_COMPRESSED;
} else if (uniqueValues != null && tableBitsRequired < deltaBitsRequired) {
format = TABLE_COMPRESSED;
} else if (gcd != 0 && gcd != 1) {
final long gcdDelta = (maxValue - minValue) / gcd;
final long gcdBitsRequired = DirectWriter.unsignedBitsRequired(gcdDelta);
format = gcdBitsRequired < deltaBitsRequired ? GCD_COMPRESSED : DELTA_COMPRESSED;
} else {
format = DELTA_COMPRESSED;
}
meta.writeVInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.NUMERIC);
meta.writeVInt(format);
if (format == SPARSE_COMPRESSED) {
meta.writeLong(data.getFilePointer());
final long numDocsWithValue;
switch (numberType) {
case VALUE:
numDocsWithValue = count - missingCount;
break;
case ORDINAL:
numDocsWithValue = count - missingOrdCount;
break;
default:
throw new AssertionError();
}
final long maxDoc = writeSparseMissingBitset(values, numberType, numDocsWithValue);
assert maxDoc == count;
} else if (missingCount == 0) {
meta.writeLong(ALL_LIVE);
} else if (missingCount == count) {
meta.writeLong(ALL_MISSING);
} else {
meta.writeLong(data.getFilePointer());
writeMissingBitset(values);
}
meta.writeLong(data.getFilePointer());
meta.writeVLong(count);
switch (format) {
case CONST_COMPRESSED:
// write the constant (nonzero value in the n=2 case, singleton value otherwise)
meta.writeLong(minValue < 0 ? Collections.min(uniqueValues) : Collections.max(uniqueValues));
break;
case GCD_COMPRESSED:
meta.writeLong(minValue);
meta.writeLong(gcd);
final long maxDelta = (maxValue - minValue) / gcd;
final int bits = DirectWriter.unsignedBitsRequired(maxDelta);
meta.writeVInt(bits);
final DirectWriter quotientWriter = DirectWriter.getInstance(data, count, bits);
for (Number nv : values) {
long value = nv == null ? 0 : nv.longValue();
quotientWriter.add((value - minValue) / gcd);
}
quotientWriter.finish();
break;
case DELTA_COMPRESSED:
final long minDelta = delta < 0 ? 0 : minValue;
meta.writeLong(minDelta);
meta.writeVInt(deltaBitsRequired);
final DirectWriter writer = DirectWriter.getInstance(data, count, deltaBitsRequired);
for (Number nv : values) {
long v = nv == null ? 0 : nv.longValue();
writer.add(v - minDelta);
}
writer.finish();
break;
case TABLE_COMPRESSED:
final Long[] decode = uniqueValues.toArray(new Long[uniqueValues.size()]);
Arrays.sort(decode);
final HashMap<Long,Integer> encode = new HashMap<>();
meta.writeVInt(decode.length);
for (int i = 0; i < decode.length; i++) {
meta.writeLong(decode[i]);
encode.put(decode[i], i);
}
meta.writeVInt(tableBitsRequired);
final DirectWriter ordsWriter = DirectWriter.getInstance(data, count, tableBitsRequired);
for (Number nv : values) {
ordsWriter.add(encode.get(nv == null ? 0 : nv.longValue()));
}
ordsWriter.finish();
break;
case SPARSE_COMPRESSED:
final Iterable<Number> filteredMissingValues;
switch (numberType) {
case VALUE:
meta.writeByte((byte) 0);
filteredMissingValues = new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return StreamSupport
.stream(values.spliterator(), false)
.filter(value -> value != null)
.iterator();
}
};
break;
case ORDINAL:
meta.writeByte((byte) 1);
filteredMissingValues = new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
return StreamSupport
.stream(values.spliterator(), false)
.filter(value -> value.longValue() != -1L)
.iterator();
}
};
break;
default:
throw new AssertionError();
}
// Write non-missing values as a numeric field
addNumericField(field, filteredMissingValues, numberType);
break;
default:
throw new AssertionError();
}
meta.writeLong(data.getFilePointer());
}
// TODO: in some cases representing missing with minValue-1 wouldn't take up additional space and so on,
// but this is very simple, and algorithms only check this for values of 0 anyway (doesnt slow down normal decode)
void writeMissingBitset(Iterable<?> values) throws IOException {
long bits = 0;
int count = 0;
for (Object v : values) {
if (count == 64) {
data.writeLong(bits);
count = 0;
bits = 0;
}
if (v != null) {
bits |= 1L << count;
}
count++;
}
if (count > 0) {
data.writeLong(bits);
}
}
long writeSparseMissingBitset(Iterable<Number> values, NumberType numberType, long numDocsWithValue) throws IOException {
meta.writeVLong(numDocsWithValue);
// Write doc IDs that have a value
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
final DirectMonotonicWriter docIdsWriter = DirectMonotonicWriter.getInstance(meta, data, numDocsWithValue, DIRECT_MONOTONIC_BLOCK_SHIFT);
long docID = 0;
for (Number nv : values) {
switch (numberType) {
case VALUE:
if (nv != null) {
docIdsWriter.add(docID);
}
break;
case ORDINAL:
if (nv.longValue() != -1L) {
docIdsWriter.add(docID);
}
break;
default:
throw new AssertionError();
}
docID++;
}
docIdsWriter.finish();
return docID;
}
@Override
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
addBinaryField(field, LegacyDocValuesIterables.binaryIterable(field, valuesProducer, maxDoc));
}
private void addBinaryField(FieldInfo field, Iterable<BytesRef> values) throws IOException {
// write the byte[] data
meta.writeVInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.BINARY);
int minLength = Integer.MAX_VALUE;
int maxLength = Integer.MIN_VALUE;
final long startFP = data.getFilePointer();
long count = 0;
long missingCount = 0;
for(BytesRef v : values) {
final int length;
if (v == null) {
length = 0;
missingCount++;
} else {
length = v.length;
}
minLength = Math.min(minLength, length);
maxLength = Math.max(maxLength, length);
if (v != null) {
data.writeBytes(v.bytes, v.offset, v.length);
}
count++;
}
meta.writeVInt(minLength == maxLength ? BINARY_FIXED_UNCOMPRESSED : BINARY_VARIABLE_UNCOMPRESSED);
if (missingCount == 0) {
meta.writeLong(ALL_LIVE);
} else if (missingCount == count) {
meta.writeLong(ALL_MISSING);
} else {
meta.writeLong(data.getFilePointer());
writeMissingBitset(values);
}
meta.writeVInt(minLength);
meta.writeVInt(maxLength);
meta.writeVLong(count);
meta.writeLong(startFP);
// if minLength == maxLength, it's a fixed-length byte[], we are done (the addresses are implicit)
// otherwise, we need to record the length fields...
if (minLength != maxLength) {
meta.writeLong(data.getFilePointer());
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, count + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
long addr = 0;
writer.add(addr);
for (BytesRef v : values) {
if (v != null) {
addr += v.length;
}
writer.add(addr);
}
writer.finish();
meta.writeLong(data.getFilePointer());
}
}
/** expert: writes a value dictionary for a sorted/sortedset field */
private void addTermsDict(FieldInfo field, final Iterable<BytesRef> values) throws IOException {
// first check if it's a "fixed-length" terms dict, and compressibility if so
int minLength = Integer.MAX_VALUE;
int maxLength = Integer.MIN_VALUE;
long numValues = 0;
BytesRefBuilder previousValue = new BytesRefBuilder();
long prefixSum = 0; // only valid for fixed-width data, as we have a choice there
for (BytesRef v : values) {
minLength = Math.min(minLength, v.length);
maxLength = Math.max(maxLength, v.length);
if (minLength == maxLength) {
int termPosition = (int) (numValues & INTERVAL_MASK);
if (termPosition == 0) {
// first term in block, save it away to compare against the last term later
previousValue.copyBytes(v);
} else if (termPosition == INTERVAL_COUNT - 1) {
// last term in block, accumulate shared prefix against first term
prefixSum += StringHelper.bytesDifference(previousValue.get(), v);
}
}
numValues++;
}
// for fixed width data, look at the avg(shared prefix) before deciding how to encode:
// prefix compression "costs" worst case 2 bytes per term because we must store suffix lengths.
// so if we share at least 3 bytes on average, always compress.
if (minLength == maxLength && prefixSum <= 3*(numValues >> INTERVAL_SHIFT)) {
// no index needed: not very compressible, direct addressing by mult
addBinaryField(field, values);
} else if (numValues < REVERSE_INTERVAL_COUNT) {
// low cardinality: waste a few KB of ram, but can't really use fancy index etc
addBinaryField(field, values);
} else {
assert numValues > 0; // we don't have to handle the empty case
// header
meta.writeVInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.BINARY);
meta.writeVInt(BINARY_PREFIX_COMPRESSED);
meta.writeLong(-1L);
// now write the bytes: sharing prefixes within a block
final long startFP = data.getFilePointer();
// currently, we have to store the delta from expected for every 1/nth term
// we could avoid this, but it's not much and less overall RAM than the previous approach!
RAMOutputStream addressBuffer = new RAMOutputStream();
MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, MONOTONIC_BLOCK_SIZE);
// buffers up 16 terms
RAMOutputStream bytesBuffer = new RAMOutputStream();
// buffers up block header
RAMOutputStream headerBuffer = new RAMOutputStream();
BytesRefBuilder lastTerm = new BytesRefBuilder();
lastTerm.grow(maxLength);
long count = 0;
int suffixDeltas[] = new int[INTERVAL_COUNT];
for (BytesRef v : values) {
int termPosition = (int) (count & INTERVAL_MASK);
if (termPosition == 0) {
termAddresses.add(data.getFilePointer() - startFP);
// abs-encode first term
headerBuffer.writeVInt(v.length);
headerBuffer.writeBytes(v.bytes, v.offset, v.length);
lastTerm.copyBytes(v);
} else {
// prefix-code: we only share at most 255 characters, to encode the length as a single
// byte and have random access. Larger terms just get less compression.
int sharedPrefix = Math.min(255, StringHelper.bytesDifference(lastTerm.get(), v));
bytesBuffer.writeByte((byte) sharedPrefix);
bytesBuffer.writeBytes(v.bytes, v.offset + sharedPrefix, v.length - sharedPrefix);
// we can encode one smaller, because terms are unique.
suffixDeltas[termPosition] = v.length - sharedPrefix - 1;
}
count++;
// flush block
if ((count & INTERVAL_MASK) == 0) {
flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
}
}
// flush trailing crap
int leftover = (int) (count & INTERVAL_MASK);
if (leftover > 0) {
Arrays.fill(suffixDeltas, leftover, suffixDeltas.length, 0);
flushTermsDictBlock(headerBuffer, bytesBuffer, suffixDeltas);
}
final long indexStartFP = data.getFilePointer();
// write addresses of indexed terms
termAddresses.finish();
addressBuffer.writeTo(data);
addressBuffer = null;
termAddresses = null;
meta.writeVInt(minLength);
meta.writeVInt(maxLength);
meta.writeVLong(count);
meta.writeLong(startFP);
meta.writeLong(indexStartFP);
meta.writeVInt(PackedInts.VERSION_CURRENT);
meta.writeVInt(MONOTONIC_BLOCK_SIZE);
addReverseTermIndex(field, values, maxLength);
}
}
// writes term dictionary "block"
// first term is absolute encoded as vint length + bytes.
// lengths of subsequent N terms are encoded as either N bytes or N shorts.
// in the double-byte case, the first byte is indicated with -1.
// subsequent terms are encoded as byte suffixLength + bytes.
private void flushTermsDictBlock(RAMOutputStream headerBuffer, RAMOutputStream bytesBuffer, int suffixDeltas[]) throws IOException {
boolean twoByte = false;
for (int i = 1; i < suffixDeltas.length; i++) {
if (suffixDeltas[i] > 254) {
twoByte = true;
}
}
if (twoByte) {
headerBuffer.writeByte((byte)255);
for (int i = 1; i < suffixDeltas.length; i++) {
headerBuffer.writeShort((short) suffixDeltas[i]);
}
} else {
for (int i = 1; i < suffixDeltas.length; i++) {
headerBuffer.writeByte((byte) suffixDeltas[i]);
}
}
headerBuffer.writeTo(data);
headerBuffer.reset();
bytesBuffer.writeTo(data);
bytesBuffer.reset();
}
// writes reverse term index: used for binary searching a term into a range of 64 blocks
// for every 64 blocks (1024 terms) we store a term, trimming any suffix unnecessary for comparison
// terms are written as a contiguous byte[], but never spanning 2^15 byte boundaries.
private void addReverseTermIndex(FieldInfo field, final Iterable<BytesRef> values, int maxLength) throws IOException {
long count = 0;
BytesRefBuilder priorTerm = new BytesRefBuilder();
priorTerm.grow(maxLength);
BytesRef indexTerm = new BytesRef();
long startFP = data.getFilePointer();
PagedBytes pagedBytes = new PagedBytes(15);
MonotonicBlockPackedWriter addresses = new MonotonicBlockPackedWriter(data, MONOTONIC_BLOCK_SIZE);
for (BytesRef b : values) {
int termPosition = (int) (count & REVERSE_INTERVAL_MASK);
if (termPosition == 0) {
int len = StringHelper.sortKeyLength(priorTerm.get(), b);
indexTerm.bytes = b.bytes;
indexTerm.offset = b.offset;
indexTerm.length = len;
addresses.add(pagedBytes.copyUsingLengthPrefix(indexTerm));
} else if (termPosition == REVERSE_INTERVAL_MASK) {
priorTerm.copyBytes(b);
}
count++;
}
addresses.finish();
long numBytes = pagedBytes.getPointer();
pagedBytes.freeze(true);
PagedBytesDataInput in = pagedBytes.getDataInput();
meta.writeLong(startFP);
data.writeVLong(numBytes);
data.copyBytes(in, numBytes);
}
@Override
public void addSortedField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.SORTED);
addTermsDict(field, LegacyDocValuesIterables.valuesIterable(valuesProducer.getSorted(field)));
addNumericField(field, LegacyDocValuesIterables.sortedOrdIterable(valuesProducer, field, maxDoc), NumberType.ORDINAL);
}
private void addSortedField(FieldInfo field, Iterable<BytesRef> values, Iterable<Number> ords) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.SORTED);
addTermsDict(field, values);
addNumericField(field, ords, NumberType.ORDINAL);
}
@Override
public void addSortedNumericField(FieldInfo field, final DocValuesProducer valuesProducer) throws IOException {
final Iterable<Number> docToValueCount = LegacyDocValuesIterables.sortedNumericToDocCount(valuesProducer, field, maxDoc);
final Iterable<Number> values = LegacyDocValuesIterables.sortedNumericToValues(valuesProducer, field);
meta.writeVInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.SORTED_NUMERIC);
if (isSingleValued(docToValueCount)) {
meta.writeVInt(SORTED_SINGLE_VALUED);
// The field is single-valued, we can encode it as NUMERIC
addNumericField(field, singletonView(docToValueCount, values, null), NumberType.VALUE);
} else {
final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToValueCount, values);
if (uniqueValueSets != null) {
meta.writeVInt(SORTED_SET_TABLE);
// write the set_id -> values mapping
writeDictionary(uniqueValueSets);
// write the doc -> set_id as a numeric field
addNumericField(field, docToSetId(uniqueValueSets, docToValueCount, values), NumberType.ORDINAL);
} else {
meta.writeVInt(SORTED_WITH_ADDRESSES);
// write the stream of values as a numeric field
addNumericField(field, values, NumberType.VALUE);
// write the doc -> ord count as a absolute index to the stream
addOrdIndex(field, docToValueCount);
}
}
}
@Override
public void addSortedSetField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
Iterable<BytesRef> values = LegacyDocValuesIterables.valuesIterable(valuesProducer.getSortedSet(field));
Iterable<Number> docToOrdCount = LegacyDocValuesIterables.sortedSetOrdCountIterable(valuesProducer, field, maxDoc);
Iterable<Number> ords = LegacyDocValuesIterables.sortedSetOrdsIterable(valuesProducer, field);
meta.writeVInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.SORTED_SET);
if (isSingleValued(docToOrdCount)) {
meta.writeVInt(SORTED_SINGLE_VALUED);
// The field is single-valued, we can encode it as SORTED
addSortedField(field, values, singletonView(docToOrdCount, ords, -1L));
} else {
final SortedSet<LongsRef> uniqueValueSets = uniqueValueSets(docToOrdCount, ords);
if (uniqueValueSets != null) {
meta.writeVInt(SORTED_SET_TABLE);
// write the set_id -> ords mapping
writeDictionary(uniqueValueSets);
// write the ord -> byte[] as a binary field
addTermsDict(field, values);
// write the doc -> set_id as a numeric field
addNumericField(field, docToSetId(uniqueValueSets, docToOrdCount, ords), NumberType.ORDINAL);
} else {
meta.writeVInt(SORTED_WITH_ADDRESSES);
// write the ord -> byte[] as a binary field
addTermsDict(field, values);
// write the stream of ords as a numeric field
// NOTE: we could return an iterator that delta-encodes these within a doc
addNumericField(field, ords, NumberType.ORDINAL);
// write the doc -> ord count as a absolute index to the stream
addOrdIndex(field, docToOrdCount);
}
}
}
private SortedSet<LongsRef> uniqueValueSets(Iterable<Number> docToValueCount, Iterable<Number> values) {
Set<LongsRef> uniqueValueSet = new HashSet<>();
LongsRef docValues = new LongsRef(256);
Iterator<Number> valueCountIterator = docToValueCount.iterator();
Iterator<Number> valueIterator = values.iterator();
int totalDictSize = 0;
while (valueCountIterator.hasNext()) {
docValues.length = valueCountIterator.next().intValue();
if (docValues.length > 256) {
return null;
}
for (int i = 0; i < docValues.length; ++i) {
docValues.longs[i] = valueIterator.next().longValue();
}
if (uniqueValueSet.contains(docValues)) {
continue;
}
totalDictSize += docValues.length;
if (totalDictSize > 256) {
return null;
}
uniqueValueSet.add(new LongsRef(Arrays.copyOf(docValues.longs, docValues.length), 0, docValues.length));
}
assert valueIterator.hasNext() == false;
return new TreeSet<>(uniqueValueSet);
}
private void writeDictionary(SortedSet<LongsRef> uniqueValueSets) throws IOException {
int lengthSum = 0;
for (LongsRef longs : uniqueValueSets) {
lengthSum += longs.length;
}
meta.writeInt(lengthSum);
for (LongsRef valueSet : uniqueValueSets) {
for (int i = 0; i < valueSet.length; ++i) {
meta.writeLong(valueSet.longs[valueSet.offset + i]);
}
}
meta.writeInt(uniqueValueSets.size());
for (LongsRef valueSet : uniqueValueSets) {
meta.writeInt(valueSet.length);
}
}
private Iterable<Number> docToSetId(SortedSet<LongsRef> uniqueValueSets, Iterable<Number> docToValueCount, Iterable<Number> values) {
final Map<LongsRef, Integer> setIds = new HashMap<>();
int i = 0;
for (LongsRef set : uniqueValueSets) {
setIds.put(set, i++);
}
assert i == uniqueValueSets.size();
return new Iterable<Number>() {
@Override
public Iterator<Number> iterator() {
final Iterator<Number> valueCountIterator = docToValueCount.iterator();
final Iterator<Number> valueIterator = values.iterator();
final LongsRef docValues = new LongsRef(256);
return new Iterator<Number>() {
@Override
public boolean hasNext() {
return valueCountIterator.hasNext();
}
@Override
public Number next() {
docValues.length = valueCountIterator.next().intValue();
for (int i = 0; i < docValues.length; ++i) {
docValues.longs[i] = valueIterator.next().longValue();
}
final Integer id = setIds.get(docValues);
assert id != null;
return id;
}
};
}
};
}
// writes addressing information as MONOTONIC_COMPRESSED integer
private void addOrdIndex(FieldInfo field, Iterable<Number> values) throws IOException {
meta.writeVInt(field.number);
meta.writeByte(Lucene70DocValuesFormat.NUMERIC);
meta.writeVInt(MONOTONIC_COMPRESSED);
meta.writeLong(-1L);
meta.writeLong(data.getFilePointer());
meta.writeVLong(maxDoc);
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, maxDoc + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
long addr = 0;
writer.add(addr);
for (Number v : values) {
addr += v.longValue();
writer.add(addr);
}
writer.finish();
meta.writeLong(data.getFilePointer());
}
@Override
public void close() throws IOException {
boolean success = false;
try {
if (meta != null) {
meta.writeVInt(-1); // write EOF marker
CodecUtil.writeFooter(meta); // write checksum
}
if (data != null) {
CodecUtil.writeFooter(data); // write checksum
}
success = true;
} finally {
if (success) {
IOUtils.close(data, meta);
} else {
IOUtils.closeWhileHandlingException(data, meta);
}
meta = data = null;
}
}
}

View File

@ -0,0 +1,183 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import java.io.IOException;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.DocValuesProducer;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.util.SmallFloat;
import org.apache.lucene.util.packed.DirectWriter;
/**
* Lucene 7.0 DocValues format.
* <p>
* Encodes the five per-document value types (Numeric,Binary,Sorted,SortedSet,SortedNumeric) with these strategies:
* <p>
* {@link DocValuesType#NUMERIC NUMERIC}:
* <ul>
* <li>Delta-compressed: per-document integers written as deltas from the minimum value,
* compressed with bitpacking. For more information, see {@link DirectWriter}.
* <li>Table-compressed: when the number of unique values is very small (&lt; 256), and
* when there are unused "gaps" in the range of values used (such as {@link SmallFloat}),
* a lookup table is written instead. Each per-document entry is instead the ordinal
* to this table, and those ordinals are compressed with bitpacking ({@link DirectWriter}).
* <li>GCD-compressed: when all numbers share a common divisor, such as dates, the greatest
* common denominator (GCD) is computed, and quotients are stored using Delta-compressed Numerics.
* <li>Monotonic-compressed: when all numbers are monotonically increasing offsets, they are written
* as blocks of bitpacked integers, encoding the deviation from the expected delta.
* <li>Const-compressed: when there is only one possible non-missing value, only the missing
* bitset is encoded.
* <li>Sparse-compressed: only documents with a value are stored, and lookups are performed
* using binary search.
* </ul>
* <p>
* {@link DocValuesType#BINARY BINARY}:
* <ul>
* <li>Fixed-width Binary: one large concatenated byte[] is written, along with the fixed length.
* Each document's value can be addressed directly with multiplication ({@code docID * length}).
* <li>Variable-width Binary: one large concatenated byte[] is written, along with end addresses
* for each document. The addresses are written as Monotonic-compressed numerics.
* <li>Prefix-compressed Binary: values are written in chunks of 16, with the first value written
* completely and other values sharing prefixes. chunk addresses are written as Monotonic-compressed
* numerics. A reverse lookup index is written from a portion of every 1024th term.
* </ul>
* <p>
* {@link DocValuesType#SORTED SORTED}:
* <ul>
* <li>Sorted: a mapping of ordinals to deduplicated terms is written as Binary,
* along with the per-document ordinals written using one of the numeric strategies above.
* </ul>
* <p>
* {@link DocValuesType#SORTED_SET SORTED_SET}:
* <ul>
* <li>Single: if all documents have 0 or 1 value, then data are written like SORTED.
* <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
* an id, a lookup table is written and the mapping from document to set id is written using the
* numeric strategies above.
* <li>SortedSet: a mapping of ordinals to deduplicated terms is written as Binary,
* an ordinal list and per-document index into this list are written using the numeric strategies
* above.
* </ul>
* <p>
* {@link DocValuesType#SORTED_NUMERIC SORTED_NUMERIC}:
* <ul>
* <li>Single: if all documents have 0 or 1 value, then data are written like NUMERIC.
* <li>SortedSet table: when there are few unique sets of values (&lt; 256) then each set is assigned
* an id, a lookup table is written and the mapping from document to set id is written using the
* numeric strategies above.
* <li>SortedNumeric: a value list and per-document index into this list are written using the numeric
* strategies above.
* </ul>
* <p>
* Files:
* <ol>
* <li><tt>.dvd</tt>: DocValues data</li>
* <li><tt>.dvm</tt>: DocValues metadata</li>
* </ol>
* @lucene.experimental
*/
public final class Lucene70DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */
public Lucene70DocValuesFormat() {
super("Lucene70");
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene70DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
@Override
public DocValuesProducer fieldsProducer(SegmentReadState state) throws IOException {
return new Lucene70DocValuesProducer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
}
static final String DATA_CODEC = "Lucene54DocValuesData";
static final String DATA_EXTENSION = "dvd";
static final String META_CODEC = "Lucene54DocValuesMetadata";
static final String META_EXTENSION = "dvm";
static final int VERSION_START = 0;
static final int VERSION_CURRENT = VERSION_START;
// indicates docvalues type
static final byte NUMERIC = 0;
static final byte BINARY = 1;
static final byte SORTED = 2;
static final byte SORTED_SET = 3;
static final byte SORTED_NUMERIC = 4;
// address terms in blocks of 16 terms
static final int INTERVAL_SHIFT = 4;
static final int INTERVAL_COUNT = 1 << INTERVAL_SHIFT;
static final int INTERVAL_MASK = INTERVAL_COUNT - 1;
// build reverse index from every 1024th term
static final int REVERSE_INTERVAL_SHIFT = 10;
static final int REVERSE_INTERVAL_COUNT = 1 << REVERSE_INTERVAL_SHIFT;
static final int REVERSE_INTERVAL_MASK = REVERSE_INTERVAL_COUNT - 1;
// for conversion from reverse index to block
static final int BLOCK_INTERVAL_SHIFT = REVERSE_INTERVAL_SHIFT - INTERVAL_SHIFT;
static final int BLOCK_INTERVAL_COUNT = 1 << BLOCK_INTERVAL_SHIFT;
static final int BLOCK_INTERVAL_MASK = BLOCK_INTERVAL_COUNT - 1;
/** Compressed using packed blocks of ints. */
static final int DELTA_COMPRESSED = 0;
/** Compressed by computing the GCD. */
static final int GCD_COMPRESSED = 1;
/** Compressed by giving IDs to unique values. */
static final int TABLE_COMPRESSED = 2;
/** Compressed with monotonically increasing values */
static final int MONOTONIC_COMPRESSED = 3;
/** Compressed with constant value (uses only missing bitset) */
static final int CONST_COMPRESSED = 4;
/** Compressed with sparse arrays. */
static final int SPARSE_COMPRESSED = 5;
/** Uncompressed binary, written directly (fixed length). */
static final int BINARY_FIXED_UNCOMPRESSED = 0;
/** Uncompressed binary, written directly (variable length). */
static final int BINARY_VARIABLE_UNCOMPRESSED = 1;
/** Compressed binary with shared prefixes */
static final int BINARY_PREFIX_COMPRESSED = 2;
/** Standard storage for sorted set values with 1 level of indirection:
* {@code docId -> address -> ord}. */
static final int SORTED_WITH_ADDRESSES = 0;
/** Single-valued sorted set values, encoded as sorted values, so no level
* of indirection: {@code docId -> ord}. */
static final int SORTED_SINGLE_VALUED = 1;
/** Compressed giving IDs to unique sets of values:
* {@code docId -> setId -> ords} */
static final int SORTED_SET_TABLE = 2;
/** placeholder for missing offset that means there are no missing values */
static final int ALL_LIVE = -1;
/** placeholder for missing offset that means all values are missing */
static final int ALL_MISSING = -2;
// addressing uses 16k blocks
static final int MONOTONIC_BLOCK_SIZE = 16384;
static final int DIRECT_MONOTONIC_BLOCK_SHIFT = 16;
}

View File

@ -0,0 +1,392 @@
/*
* 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.
*/
/**
* Lucene 7.0 file format.
*
* <h1>Apache Lucene - Index File Formats</h1>
* <div>
* <ul>
* <li><a href="#Introduction">Introduction</a></li>
* <li><a href="#Definitions">Definitions</a>
* <ul>
* <li><a href="#Inverted_Indexing">Inverted Indexing</a></li>
* <li><a href="#Types_of_Fields">Types of Fields</a></li>
* <li><a href="#Segments">Segments</a></li>
* <li><a href="#Document_Numbers">Document Numbers</a></li>
* </ul>
* </li>
* <li><a href="#Overview">Index Structure Overview</a></li>
* <li><a href="#File_Naming">File Naming</a></li>
* <li><a href="#file-names">Summary of File Extensions</a>
* <ul>
* <li><a href="#Lock_File">Lock File</a></li>
* <li><a href="#History">History</a></li>
* <li><a href="#Limitations">Limitations</a></li>
* </ul>
* </li>
* </ul>
* </div>
* <a name="Introduction"></a>
* <h2>Introduction</h2>
* <div>
* <p>This document defines the index file formats used in this version of Lucene.
* If you are using a different version of Lucene, please consult the copy of
* <code>docs/</code> that was distributed with
* the version you are using.</p>
* <p>This document attempts to provide a high-level definition of the Apache
* Lucene file formats.</p>
* </div>
* <a name="Definitions"></a>
* <h2>Definitions</h2>
* <div>
* <p>The fundamental concepts in Lucene are index, document, field and term.</p>
* <p>An index contains a sequence of documents.</p>
* <ul>
* <li>A document is a sequence of fields.</li>
* <li>A field is a named sequence of terms.</li>
* <li>A term is a sequence of bytes.</li>
* </ul>
* <p>The same sequence of bytes in two different fields is considered a different
* term. Thus terms are represented as a pair: the string naming the field, and the
* bytes within the field.</p>
* <a name="Inverted_Indexing"></a>
* <h3>Inverted Indexing</h3>
* <p>The index stores statistics about terms in order to make term-based search
* more efficient. Lucene's index falls into the family of indexes known as an
* <i>inverted index.</i> This is because it can list, for a term, the documents
* that contain it. This is the inverse of the natural relationship, in which
* documents list terms.</p>
* <a name="Types_of_Fields"></a>
* <h3>Types of Fields</h3>
* <p>In Lucene, fields may be <i>stored</i>, in which case their text is stored
* in the index literally, in a non-inverted manner. Fields that are inverted are
* called <i>indexed</i>. A field may be both stored and indexed.</p>
* <p>The text of a field may be <i>tokenized</i> into terms to be indexed, or the
* text of a field may be used literally as a term to be indexed. Most fields are
* tokenized, but sometimes it is useful for certain identifier fields to be
* indexed literally.</p>
* <p>See the {@link org.apache.lucene.document.Field Field}
* java docs for more information on Fields.</p>
* <a name="Segments"></a>
* <h3>Segments</h3>
* <p>Lucene indexes may be composed of multiple sub-indexes, or <i>segments</i>.
* Each segment is a fully independent index, which could be searched separately.
* Indexes evolve by:</p>
* <ol>
* <li>Creating new segments for newly added documents.</li>
* <li>Merging existing segments.</li>
* </ol>
* <p>Searches may involve multiple segments and/or multiple indexes, each index
* potentially composed of a set of segments.</p>
* <a name="Document_Numbers"></a>
* <h3>Document Numbers</h3>
* <p>Internally, Lucene refers to documents by an integer <i>document number</i>.
* The first document added to an index is numbered zero, and each subsequent
* document added gets a number one greater than the previous.</p>
* <p>Note that a document's number may change, so caution should be taken when
* storing these numbers outside of Lucene. In particular, numbers may change in
* the following situations:</p>
* <ul>
* <li>
* <p>The numbers stored in each segment are unique only within the segment, and
* must be converted before they can be used in a larger context. The standard
* technique is to allocate each segment a range of values, based on the range of
* numbers used in that segment. To convert a document number from a segment to an
* external value, the segment's <i>base</i> document number is added. To convert
* an external value back to a segment-specific value, the segment is identified
* by the range that the external value is in, and the segment's base value is
* subtracted. For example two five document segments might be combined, so that
* the first segment has a base value of zero, and the second of five. Document
* three from the second segment would have an external value of eight.</p>
* </li>
* <li>
* <p>When documents are deleted, gaps are created in the numbering. These are
* eventually removed as the index evolves through merging. Deleted documents are
* dropped when segments are merged. A freshly-merged segment thus has no gaps in
* its numbering.</p>
* </li>
* </ul>
* </div>
* <a name="Overview"></a>
* <h2>Index Structure Overview</h2>
* <div>
* <p>Each segment index maintains the following:</p>
* <ul>
* <li>
* {@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment info}.
* This contains metadata about a segment, such as the number of documents,
* what files it uses,
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Field names}.
* This contains the set of field names used in the index.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Stored Field values}.
* This contains, for each document, a list of attribute-value pairs, where the attributes
* are field names. These are used to store auxiliary information about the document, such as
* its title, url, or an identifier to access a database. The set of stored fields are what is
* returned for each hit when searching. This is keyed by document number.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term dictionary}.
* A dictionary containing all of the terms used in all of the
* indexed fields of all of the documents. The dictionary also contains the number
* of documents which contain the term, and pointers to the term's frequency and
* proximity data.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Frequency data}.
* For each term in the dictionary, the numbers of all the
* documents that contain that term, and the frequency of the term in that
* document, unless frequencies are omitted (IndexOptions.DOCS_ONLY)
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Proximity data}.
* For each term in the dictionary, the positions that the
* term occurs in each document. Note that this will not exist if all fields in
* all documents omit position data.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Normalization factors}.
* For each field in each document, a value is stored
* that is multiplied into the score for hits on that field.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vectors}.
* For each field in each document, the term vector (sometimes
* called document vector) may be stored. A term vector consists of term text and
* term frequency. To add Term Vectors to your index see the
* {@link org.apache.lucene.document.Field Field} constructors
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat Per-document values}.
* Like stored values, these are also keyed by document
* number, but are generally intended to be loaded into main memory for fast
* access. Whereas stored values are generally intended for summary results from
* searches, per-document values are useful for things like scoring factors.
* </li>
* <li>
* {@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live documents}.
* An optional file indicating which documents are live.
* </li>
* </ul>
* <p>Details on each of these are provided in their linked pages.</p>
* </div>
* <a name="File_Naming"></a>
* <h2>File Naming</h2>
* <div>
* <p>All files belonging to a segment have the same name with varying extensions.
* The extensions correspond to the different file formats described below. When
* using the Compound File format (default for small segments) these files (except
* for the Segment info file, the Lock file, and Deleted documents file) are collapsed
* into a single .cfs file (see below for details)</p>
* <p>Typically, all segments in an index are stored in a single directory,
* although this is not required.</p>
* <p>File names are never re-used. That is, when any file is saved
* to the Directory it is given a never before used filename. This is achieved
* using a simple generations approach. For example, the first segments file is
* segments_1, then segments_2, etc. The generation is a sequential long integer
* represented in alpha-numeric (base 36) form.</p>
* </div>
* <a name="file-names"></a>
* <h2>Summary of File Extensions</h2>
* <div>
* <p>The following table summarizes the names and extensions of the files in
* Lucene:</p>
* <table cellspacing="1" cellpadding="4" summary="lucene filenames by extension">
* <tr>
* <th>Name</th>
* <th>Extension</th>
* <th>Brief Description</th>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.index.SegmentInfos Segments File}</td>
* <td>segments_N</td>
* <td>Stores information about a commit point</td>
* </tr>
* <tr>
* <td><a href="#Lock_File">Lock File</a></td>
* <td>write.lock</td>
* <td>The Write lock prevents multiple IndexWriters from writing to the same
* file.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene62.Lucene62SegmentInfoFormat Segment Info}</td>
* <td>.si</td>
* <td>Stores metadata about a segment</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat Compound File}</td>
* <td>.cfs, .cfe</td>
* <td>An optional "virtual" file consisting of all the other index files for
* systems that frequently run out of file handles.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50FieldInfosFormat Fields}</td>
* <td>.fnm</td>
* <td>Stores information about the fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Index}</td>
* <td>.fdx</td>
* <td>Contains pointers to field data</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat Field Data}</td>
* <td>.fdt</td>
* <td>The stored fields for documents</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Dictionary}</td>
* <td>.tim</td>
* <td>The term dictionary, stores term info</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Term Index}</td>
* <td>.tip</td>
* <td>The index into the Term Dictionary</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Frequencies}</td>
* <td>.doc</td>
* <td>Contains the list of docs which contain each term along with frequency</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Positions}</td>
* <td>.pos</td>
* <td>Stores position information about where a term occurs in the index</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat Payloads}</td>
* <td>.pay</td>
* <td>Stores additional per-position metadata information such as character offsets and user payloads</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene53.Lucene53NormsFormat Norms}</td>
* <td>.nvd, .nvm</td>
* <td>Encodes length and boost factors for docs and fields</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat Per-Document Values}</td>
* <td>.dvd, .dvm</td>
* <td>Encodes additional scoring factors or other per-document information.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Index}</td>
* <td>.tvx</td>
* <td>Stores offset into the document data file</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat Term Vector Data}</td>
* <td>.tvd</td>
* <td>Contains term vector data.</td>
* </tr>
* <tr>
* <td>{@link org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat Live Documents}</td>
* <td>.liv</td>
* <td>Info about what files are live</td>
* </tr>
* </table>
* </div>
* <a name="Lock_File"></a>
* <h2>Lock File</h2>
* The write lock, which is stored in the index directory by default, is named
* "write.lock". If the lock directory is different from the index directory then
* the write lock will be named "XXXX-write.lock" where XXXX is a unique prefix
* derived from the full path to the index directory. When this file is present, a
* writer is currently modifying the index (adding or removing documents). This
* lock file ensures that only one writer is modifying the index at a time.
* <a name="History"></a>
* <h2>History</h2>
* <p>Compatibility notes are provided in this document, describing how file
* formats have changed from prior versions:</p>
* <ul>
* <li>In version 2.1, the file format was changed to allow lock-less commits (ie,
* no more commit lock). The change is fully backwards compatible: you can open a
* pre-2.1 index for searching or adding/deleting of docs. When the new segments
* file is saved (committed), it will be written in the new file format (meaning
* no specific "upgrade" process is needed). But note that once a commit has
* occurred, pre-2.1 Lucene will not be able to read the index.</li>
* <li>In version 2.3, the file format was changed to allow segments to share a
* single set of doc store (vectors &amp; stored fields) files. This allows for
* faster indexing in certain cases. The change is fully backwards compatible (in
* the same way as the lock-less commits change in 2.1).</li>
* <li>In version 2.4, Strings are now written as true UTF-8 byte sequence, not
* Java's modified UTF-8. See <a href="http://issues.apache.org/jira/browse/LUCENE-510">
* LUCENE-510</a> for details.</li>
* <li>In version 2.9, an optional opaque Map&lt;String,String&gt; CommitUserData
* may be passed to IndexWriter's commit methods (and later retrieved), which is
* recorded in the segments_N file. See <a href="http://issues.apache.org/jira/browse/LUCENE-1382">
* LUCENE-1382</a> for details. Also,
* diagnostics were added to each segment written recording details about why it
* was written (due to flush, merge; which OS/JRE was used; etc.). See issue
* <a href="http://issues.apache.org/jira/browse/LUCENE-1654">LUCENE-1654</a> for details.</li>
* <li>In version 3.0, compressed fields are no longer written to the index (they
* can still be read, but on merge the new segment will write them, uncompressed).
* See issue <a href="http://issues.apache.org/jira/browse/LUCENE-1960">LUCENE-1960</a>
* for details.</li>
* <li>In version 3.1, segments records the code version that created them. See
* <a href="http://issues.apache.org/jira/browse/LUCENE-2720">LUCENE-2720</a> for details.
* Additionally segments track explicitly whether or not they have term vectors.
* See <a href="http://issues.apache.org/jira/browse/LUCENE-2811">LUCENE-2811</a>
* for details.</li>
* <li>In version 3.2, numeric fields are written as natively to stored fields
* file, previously they were stored in text format only.</li>
* <li>In version 3.4, fields can omit position data while still indexing term
* frequencies.</li>
* <li>In version 4.0, the format of the inverted index became extensible via
* the {@link org.apache.lucene.codecs.Codec Codec} api. Fast per-document storage
* ({@code DocValues}) was introduced. Normalization factors need no longer be a
* single byte, they can be any {@link org.apache.lucene.index.NumericDocValues NumericDocValues}.
* Terms need not be unicode strings, they can be any byte sequence. Term offsets
* can optionally be indexed into the postings lists. Payloads can be stored in the
* term vectors.</li>
* <li>In version 4.1, the format of the postings list changed to use either
* of FOR compression or variable-byte encoding, depending upon the frequency
* of the term. Terms appearing only once were changed to inline directly into
* the term dictionary. Stored fields are compressed by default. </li>
* <li>In version 4.2, term vectors are compressed by default. DocValues has
* a new multi-valued type (SortedSet), that can be used for faceting/grouping/joining
* on multi-valued fields.</li>
* <li>In version 4.5, DocValues were extended to explicitly represent missing values.</li>
* <li>In version 4.6, FieldInfos were extended to support per-field DocValues generation, to
* allow updating NumericDocValues fields.</li>
* <li>In version 4.8, checksum footers were added to the end of each index file
* for improved data integrity. Specifically, the last 8 bytes of every index file
* contain the zlib-crc32 checksum of the file.</li>
* <li>In version 4.9, DocValues has a new multi-valued numeric type (SortedNumeric)
* that is suitable for faceting/sorting/analytics.
* <li>In version 5.4, DocValues have been improved to store more information on disk:
* addresses for binary fields and ord indexes for multi-valued fields.
* <li>In version 7.0, DocValues have been improved to better support sparse doc values
* thanks to an iterator API.
* </li>
* </ul>
* <a name="Limitations"></a>
* <h2>Limitations</h2>
* <div>
* <p>Lucene uses a Java <code>int</code> to refer to
* document numbers, and the index file format uses an <code>Int32</code>
* on-disk to store document numbers. This is a limitation
* of both the index file format and the current implementation. Eventually these
* should be replaced with either <code>UInt64</code> values, or
* better yet, {@link org.apache.lucene.store.DataOutput#writeVInt VInt} values which have no limit.</p>
* </div>
*/
package org.apache.lucene.codecs.lucene70;

View File

@ -723,8 +723,10 @@ final class DefaultIndexingChain extends DocConsumer {
if (invertState.position < invertState.lastPosition) {
if (posIncr == 0) {
throw new IllegalArgumentException("first position increment must be > 0 (got 0) for field '" + field.name() + "'");
} else if (posIncr < 0) {
throw new IllegalArgumentException("position increment must be >= 0 (got " + posIncr + ") for field '" + field.name() + "'");
} else {
throw new IllegalArgumentException("position increments (and gaps) must be >= 0 (got " + posIncr + ") for field '" + field.name() + "'");
throw new IllegalArgumentException("position overflowed Integer.MAX_VALUE (got posIncr=" + posIncr + " lastPosition=" + invertState.lastPosition + " position=" + invertState.position + ") for field '" + field.name() + "'");
}
} else if (invertState.position > IndexWriter.MAX_POSITION) {
throw new IllegalArgumentException("position " + invertState.position + " is too large for field '" + field.name() + "': max allowed position is " + IndexWriter.MAX_POSITION);

View File

@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene62.Lucene62Codec
org.apache.lucene.codecs.lucene70.Lucene70Codec

View File

@ -14,3 +14,4 @@
# limitations under the License.
org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat
org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat

View File

@ -19,7 +19,7 @@ package org.apache.lucene.codecs.lucene50;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;
@ -33,7 +33,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomPicks;
public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFieldsFormatTestCase {
@Override
protected Codec getCodec() {
return new Lucene62Codec(Mode.BEST_COMPRESSION);
return new Lucene70Codec(Mode.BEST_COMPRESSION);
}
/**
@ -44,7 +44,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
Directory dir = newDirectory();
for (int i = 0; i < 10; i++) {
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setCodec(new Lucene62Codec(RandomPicks.randomFrom(random(), Mode.values())));
iwc.setCodec(new Lucene70Codec(RandomPicks.randomFrom(random(), Mode.values())));
IndexWriter iw = new IndexWriter(dir, newIndexWriterConfig());
Document doc = new Document();
doc.add(new StoredField("field1", "value1"));
@ -71,7 +71,7 @@ public class TestLucene50StoredFieldsFormatHighCompression extends BaseStoredFie
public void testInvalidOptions() throws Exception {
expectThrows(NullPointerException.class, () -> {
new Lucene62Codec(null);
new Lucene70Codec(null);
});
expectThrows(NullPointerException.class, () -> {

View File

@ -18,14 +18,14 @@ package org.apache.lucene.codecs.lucene53;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.lucene.index.BaseNormsFormatTestCase;
/**
* Tests Lucene53NormsFormat
*/
public class TestLucene53NormsFormat extends BaseNormsFormatTestCase {
private final Codec codec = new Lucene62Codec();
private final Codec codec = new Lucene70Codec();
@Override
protected Codec getCodec() {

View File

@ -0,0 +1,613 @@
/*
* 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.
*/
package org.apache.lucene.codecs.lucene70;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.TreeSet;
import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesProducer.SparseNumericDocValues;
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesProducer.SparseNumericDocValuesRandomAccessWrapper;
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
import org.apache.lucene.document.BinaryDocValuesField;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.SortedDocValuesField;
import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.document.SortedSetDocValuesField;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.document.StringField;
import org.apache.lucene.index.BaseCompressingDocValuesFormatTestCase;
import org.apache.lucene.index.BinaryDocValues;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NumericDocValues;
import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.index.SerialMergeScheduler;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMFile;
import org.apache.lucene.store.RAMInputStream;
import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.LongValues;
import org.apache.lucene.util.TestUtil;
/**
* Tests Lucene70DocValuesFormat
*/
public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene70DocValuesFormat());
@Override
protected Codec getCodec() {
return codec;
}
// TODO: these big methods can easily blow up some of the other ram-hungry codecs...
// for now just keep them here, as we want to test this for this format.
@Slow
public void testSortedSetVariableLengthBigVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedSetVsStoredFields(atLeast(300), 1, 32766, 16, 100);
}
}
@Nightly
public void testSortedSetVariableLengthManyVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedSetVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500, 16, 100);
}
}
@Slow
public void testSortedVariableLengthBigVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedVsStoredFields(atLeast(300), 1, 32766);
}
}
@Nightly
public void testSortedVariableLengthManyVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSortedVsStoredFields(TestUtil.nextInt(random(), 1024, 2049), 1, 500);
}
}
@Slow
public void testTermsEnumFixedWidth() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 10, 10);
}
}
@Slow
public void testTermsEnumVariableWidth() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 5121), 1, 500);
}
}
@Nightly
public void testTermsEnumRandomMany() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestTermsEnumRandom(TestUtil.nextInt(random(), 1025, 8121), 1, 500);
}
}
@Slow
public void testSparseDocValuesVsStoredFields() throws Exception {
int numIterations = atLeast(1);
for (int i = 0; i < numIterations; i++) {
doTestSparseDocValuesVsStoredFields();
}
}
private void doTestSparseDocValuesVsStoredFields() throws Exception {
final long[] values = new long[TestUtil.nextInt(random(), 1, 500)];
for (int i = 0; i < values.length; ++i) {
values[i] = random().nextLong();
}
Directory dir = newFSDirectory(createTempDir());
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMergeScheduler(new SerialMergeScheduler());
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
// sparse compression is only enabled if less than 1% of docs have a value
final int avgGap = 100;
final int numDocs = atLeast(200);
for (int i = random().nextInt(avgGap * 2); i >= 0; --i) {
writer.addDocument(new Document());
}
final int maxNumValuesPerDoc = random().nextBoolean() ? 1 : TestUtil.nextInt(random(), 2, 5);
for (int i = 0; i < numDocs; ++i) {
Document doc = new Document();
// single-valued
long docValue = values[random().nextInt(values.length)];
doc.add(new NumericDocValuesField("numeric", docValue));
doc.add(new SortedDocValuesField("sorted", new BytesRef(Long.toString(docValue))));
doc.add(new BinaryDocValuesField("binary", new BytesRef(Long.toString(docValue))));
doc.add(new StoredField("value", docValue));
// multi-valued
final int numValues = TestUtil.nextInt(random(), 1, maxNumValuesPerDoc);
for (int j = 0; j < numValues; ++j) {
docValue = values[random().nextInt(values.length)];
doc.add(new SortedNumericDocValuesField("sorted_numeric", docValue));
doc.add(new SortedSetDocValuesField("sorted_set", new BytesRef(Long.toString(docValue))));
doc.add(new StoredField("values", docValue));
}
writer.addDocument(doc);
// add a gap
for (int j = TestUtil.nextInt(random(), 0, avgGap * 2); j >= 0; --j) {
writer.addDocument(new Document());
}
}
if (random().nextBoolean()) {
writer.forceMerge(1);
}
final IndexReader indexReader = writer.getReader();
writer.close();
for (LeafReaderContext context : indexReader.leaves()) {
final LeafReader reader = context.reader();
final NumericDocValues numeric = DocValues.getNumeric(reader, "numeric");
final SortedDocValues sorted = DocValues.getSorted(reader, "sorted");
final BinaryDocValues binary = DocValues.getBinary(reader, "binary");
final SortedNumericDocValues sortedNumeric = DocValues.getSortedNumeric(reader, "sorted_numeric");
final SortedSetDocValues sortedSet = DocValues.getSortedSet(reader, "sorted_set");
for (int i = 0; i < reader.maxDoc(); ++i) {
final Document doc = reader.document(i);
final IndexableField valueField = doc.getField("value");
final Long value = valueField == null ? null : valueField.numericValue().longValue();
if (value == null) {
assertTrue(numeric.docID() + " vs " + i, numeric.docID() < i);
} else {
assertEquals(i, numeric.nextDoc());
assertEquals(i, binary.nextDoc());
assertEquals(i, sorted.nextDoc());
assertEquals(value.longValue(), numeric.longValue());
assertTrue(sorted.ordValue() >= 0);
assertEquals(new BytesRef(Long.toString(value)), sorted.lookupOrd(sorted.ordValue()));
assertEquals(new BytesRef(Long.toString(value)), binary.binaryValue());
}
final IndexableField[] valuesFields = doc.getFields("values");
if (valuesFields.length == 0) {
assertTrue(sortedNumeric.docID() + " vs " + i, sortedNumeric.docID() < i);
} else {
final Set<Long> valueSet = new HashSet<>();
for (IndexableField sf : valuesFields) {
valueSet.add(sf.numericValue().longValue());
}
assertEquals(i, sortedNumeric.nextDoc());
assertEquals(valuesFields.length, sortedNumeric.docValueCount());
for (int j = 0; j < sortedNumeric.docValueCount(); ++j) {
assertTrue(valueSet.contains(sortedNumeric.nextValue()));
}
assertEquals(i, sortedSet.nextDoc());
int sortedSetCount = 0;
while (true) {
long ord = sortedSet.nextOrd();
if (ord == SortedSetDocValues.NO_MORE_ORDS) {
break;
}
assertTrue(valueSet.contains(Long.parseLong(sortedSet.lookupOrd(ord).utf8ToString())));
sortedSetCount++;
}
assertEquals(valueSet.size(), sortedSetCount);
}
}
}
indexReader.close();
dir.close();
}
// TODO: try to refactor this and some termsenum tests into the base class.
// to do this we need to fix the test class to get a DVF not a Codec so we can setup
// the postings format correctly.
private void doTestTermsEnumRandom(int numDocs, int minLength, int maxLength) throws Exception {
Directory dir = newFSDirectory(createTempDir());
IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
conf.setMergeScheduler(new SerialMergeScheduler());
// set to duel against a codec which has ordinals:
final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
final DocValuesFormat dv = new Lucene70DocValuesFormat();
conf.setCodec(new AssertingCodec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
return pf;
}
@Override
public DocValuesFormat getDocValuesFormatForField(String field) {
return dv;
}
});
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, conf);
// index some docs
for (int i = 0; i < numDocs; i++) {
Document doc = new Document();
Field idField = new StringField("id", Integer.toString(i), Field.Store.NO);
doc.add(idField);
final int length = TestUtil.nextInt(random(), minLength, maxLength);
int numValues = random().nextInt(17);
// create a random list of strings
List<String> values = new ArrayList<>();
for (int v = 0; v < numValues; v++) {
values.add(TestUtil.randomSimpleString(random(), minLength, length));
}
// add in any order to the indexed field
ArrayList<String> unordered = new ArrayList<>(values);
Collections.shuffle(unordered, random());
for (String v : values) {
doc.add(newStringField("indexed", v, Field.Store.NO));
}
// add in any order to the dv field
ArrayList<String> unordered2 = new ArrayList<>(values);
Collections.shuffle(unordered2, random());
for (String v : unordered2) {
doc.add(new SortedSetDocValuesField("dv", new BytesRef(v)));
}
writer.addDocument(doc);
if (random().nextInt(31) == 0) {
writer.commit();
}
}
// delete some docs
int numDeletions = random().nextInt(numDocs/10);
for (int i = 0; i < numDeletions; i++) {
int id = random().nextInt(numDocs);
writer.deleteDocuments(new Term("id", Integer.toString(id)));
}
// compare per-segment
DirectoryReader ir = writer.getReader();
for (LeafReaderContext context : ir.leaves()) {
LeafReader r = context.reader();
Terms terms = r.terms("indexed");
if (terms != null) {
SortedSetDocValues ssdv = r.getSortedSetDocValues("dv");
assertEquals(terms.size(), ssdv.getValueCount());
TermsEnum expected = terms.iterator();
TermsEnum actual = r.getSortedSetDocValues("dv").termsEnum();
assertEquals(terms.size(), expected, actual);
doTestSortedSetEnumAdvanceIndependently(ssdv);
}
}
ir.close();
writer.forceMerge(1);
// now compare again after the merge
ir = writer.getReader();
LeafReader ar = getOnlyLeafReader(ir);
Terms terms = ar.terms("indexed");
if (terms != null) {
assertEquals(terms.size(), ar.getSortedSetDocValues("dv").getValueCount());
TermsEnum expected = terms.iterator();
TermsEnum actual = ar.getSortedSetDocValues("dv").termsEnum();
assertEquals(terms.size(), expected, actual);
}
ir.close();
writer.close();
dir.close();
}
private void assertEquals(long numOrds, TermsEnum expected, TermsEnum actual) throws Exception {
BytesRef ref;
// sequential next() through all terms
while ((ref = expected.next()) != null) {
assertEquals(ref, actual.next());
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
assertNull(actual.next());
// sequential seekExact(ord) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
actual.seekExact(i);
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// sequential seekExact(BytesRef) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
assertTrue(actual.seekExact(expected.term()));
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// sequential seekCeil(BytesRef) through all terms
for (long i = 0; i < numOrds; i++) {
expected.seekExact(i);
assertEquals(SeekStatus.FOUND, actual.seekCeil(expected.term()));
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekExact(ord)
for (long i = 0; i < numOrds; i++) {
long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
expected.seekExact(randomOrd);
actual.seekExact(randomOrd);
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekExact(BytesRef)
for (long i = 0; i < numOrds; i++) {
long randomOrd = TestUtil.nextLong(random(), 0, numOrds - 1);
expected.seekExact(randomOrd);
actual.seekExact(expected.term());
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
// random seekCeil(BytesRef)
for (long i = 0; i < numOrds; i++) {
BytesRef target = new BytesRef(TestUtil.randomUnicodeString(random()));
SeekStatus expectedStatus = expected.seekCeil(target);
assertEquals(expectedStatus, actual.seekCeil(target));
if (expectedStatus != SeekStatus.END) {
assertEquals(expected.ord(), actual.ord());
assertEquals(expected.term(), actual.term());
}
}
}
public void testSparseLongValues() throws IOException {
final int iters = atLeast(5);
for (int iter = 0; iter < iters; ++iter) {
final int numDocs = TestUtil.nextInt(random(), 0, 100);
final int[] docIds = new int[numDocs];
final long[] values = new long[numDocs];
final int maxDoc;
if (numDocs == 0) {
maxDoc = 1 + random().nextInt(10);
} else {
docIds[0] = random().nextInt(10);
for (int i = 1; i < docIds.length; ++i) {
docIds[i] = docIds[i - 1] + 1 + random().nextInt(100);
}
maxDoc = docIds[numDocs - 1] + 1 + random().nextInt(10);
}
for (int i = 0; i < values.length; ++i) {
values[i] = random().nextLong();
}
final long missingValue = random().nextLong();
final LongValues docIdsValues = new LongValues() {
@Override
public long get(long index) {
return docIds[Math.toIntExact(index)];
}
};
final LongValues valuesValues = new LongValues() {
@Override
public long get(long index) {
return values[Math.toIntExact(index)];
}
};
final SparseNumericDocValues sparseValues = new SparseNumericDocValues(numDocs, docIdsValues, valuesValues);
// sequential access
assertEquals(-1, sparseValues.docID());
for (int i = 0; i < docIds.length; ++i) {
assertEquals(docIds[i], sparseValues.nextDoc());
}
assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.nextDoc());
// advance
for (int i = 0; i < 2000; ++i) {
final int target = TestUtil.nextInt(random(), 0, (int) maxDoc);
int index = Arrays.binarySearch(docIds, target);
if (index < 0) {
index = -1 - index;
}
sparseValues.reset();
if (index > 0) {
assertEquals(docIds[index - 1], sparseValues.advance(Math.toIntExact(docIds[index - 1])));
}
if (index == docIds.length) {
assertEquals(DocIdSetIterator.NO_MORE_DOCS, sparseValues.advance(target));
} else {
assertEquals(docIds[index], sparseValues.advance(target));
}
}
final SparseNumericDocValuesRandomAccessWrapper raWrapper = new SparseNumericDocValuesRandomAccessWrapper(sparseValues, missingValue);
// random-access
for (int i = 0; i < 2000; ++i) {
final int docId = TestUtil.nextInt(random(), 0, maxDoc - 1);
final int idx = Arrays.binarySearch(docIds, docId);
final long value = raWrapper.get(docId);
if (idx >= 0) {
assertEquals(values[idx], value);
} else {
assertEquals(missingValue, value);
}
}
// sequential access
for (int docId = 0; docId < maxDoc; docId += random().nextInt(3)) {
final int idx = Arrays.binarySearch(docIds, docId);
final long value = raWrapper.get(docId);
if (idx >= 0) {
assertEquals(values[idx], value);
} else {
assertEquals(missingValue, value);
}
}
}
}
@Slow
public void testSortedSetAroundBlockSize() throws IOException {
final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
final Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
RAMFile buffer = new RAMFile();
RAMOutputStream out = new RAMOutputStream(buffer, false);
Document doc = new Document();
SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef());
doc.add(field1);
SortedSetDocValuesField field2 = new SortedSetDocValuesField("sset", new BytesRef());
doc.add(field2);
for (int i = 0; i < maxDoc; ++i) {
BytesRef s1 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
BytesRef s2 = new BytesRef(TestUtil.randomSimpleString(random(), 2));
field1.setBytesValue(s1);
field2.setBytesValue(s2);
w.addDocument(doc);
Set<BytesRef> set = new TreeSet<>(Arrays.asList(s1, s2));
out.writeVInt(set.size());
for (BytesRef ref : set) {
out.writeVInt(ref.length);
out.writeBytes(ref.bytes, ref.offset, ref.length);
}
}
out.close();
w.forceMerge(1);
DirectoryReader r = DirectoryReader.open(w);
w.close();
LeafReader sr = getOnlyLeafReader(r);
assertEquals(maxDoc, sr.maxDoc());
SortedSetDocValues values = sr.getSortedSetDocValues("sset");
assertNotNull(values);
RAMInputStream in = new RAMInputStream("", buffer);
BytesRefBuilder b = new BytesRefBuilder();
for (int i = 0; i < maxDoc; ++i) {
assertEquals(i, values.nextDoc());
final int numValues = in.readVInt();
for (int j = 0; j < numValues; ++j) {
b.setLength(in.readVInt());
b.grow(b.length());
in.readBytes(b.bytes(), 0, b.length());
assertEquals(b.get(), values.lookupOrd(values.nextOrd()));
}
assertEquals(SortedSetDocValues.NO_MORE_ORDS, values.nextOrd());
}
r.close();
dir.close();
}
}
@Slow
public void testSortedNumericAroundBlockSize() throws IOException {
final int frontier = 1 << Lucene70DocValuesFormat.DIRECT_MONOTONIC_BLOCK_SHIFT;
for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
final Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
RAMFile buffer = new RAMFile();
RAMOutputStream out = new RAMOutputStream(buffer, false);
Document doc = new Document();
SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L);
doc.add(field1);
SortedNumericDocValuesField field2 = new SortedNumericDocValuesField("snum", 0L);
doc.add(field2);
for (int i = 0; i < maxDoc; ++i) {
long s1 = random().nextInt(100);
long s2 = random().nextInt(100);
field1.setLongValue(s1);
field2.setLongValue(s2);
w.addDocument(doc);
out.writeVLong(Math.min(s1, s2));
out.writeVLong(Math.max(s1, s2));
}
out.close();
w.forceMerge(1);
DirectoryReader r = DirectoryReader.open(w);
w.close();
LeafReader sr = getOnlyLeafReader(r);
assertEquals(maxDoc, sr.maxDoc());
SortedNumericDocValues values = sr.getSortedNumericDocValues("snum");
assertNotNull(values);
RAMInputStream in = new RAMInputStream("", buffer);
for (int i = 0; i < maxDoc; ++i) {
assertEquals(i, values.nextDoc());
assertEquals(2, values.docValueCount());
assertEquals(in.readVLong(), values.nextValue());
assertEquals(in.readVLong(), values.nextValue());
}
r.close();
dir.close();
}
}
}

View File

@ -135,6 +135,6 @@ public class Test2BPoints extends LuceneTestCase {
}
private static Codec getCodec() {
return Codec.forName("Lucene62");
return Codec.forName("Lucene70");
}
}

View File

@ -393,11 +393,11 @@ public class TestPointValues extends LuceneTestCase {
dir.close();
}
// Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with SimpleText
// Write point values, one segment with Lucene70, another with SimpleText, then forceMerge with SimpleText
public void testDifferentCodecs1() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(Codec.forName("Lucene62"));
iwc.setCodec(Codec.forName("Lucene70"));
IndexWriter w = new IndexWriter(dir, iwc);
Document doc = new Document();
doc.add(new IntPoint("int", 1));
@ -416,7 +416,7 @@ public class TestPointValues extends LuceneTestCase {
dir.close();
}
// Write point values, one segment with Lucene62, another with SimpleText, then forceMerge with Lucene60
// Write point values, one segment with Lucene70, another with SimpleText, then forceMerge with Lucene70
public void testDifferentCodecs2() throws Exception {
Directory dir = newDirectory();
IndexWriterConfig iwc = new IndexWriterConfig(new MockAnalyzer(random()));
@ -428,7 +428,7 @@ public class TestPointValues extends LuceneTestCase {
w.close();
iwc = new IndexWriterConfig(new MockAnalyzer(random()));
iwc.setCodec(Codec.forName("Lucene62"));
iwc.setCodec(Codec.forName("Lucene70"));
w = new IndexWriter(dir, iwc);
doc = new Document();
doc.add(new IntPoint("int", 1));

View File

@ -96,7 +96,7 @@ public class TestBoolean2 extends LuceneTestCase {
IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random()));
// randomized codecs are sometimes too costly for this test:
iwc.setCodec(Codec.forName("Lucene62"));
iwc.setCodec(Codec.forName("Lucene70"));
iwc.setMergePolicy(newLogMergePolicy());
RandomIndexWriter writer= new RandomIndexWriter(random(), directory, iwc);
// we'll make a ton of docs, disable store/norms/vectors
@ -141,7 +141,7 @@ public class TestBoolean2 extends LuceneTestCase {
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
// we need docID order to be preserved:
// randomized codecs are sometimes too costly for this test:
iwc.setCodec(Codec.forName("Lucene62"));
iwc.setCodec(Codec.forName("Lucene70"));
iwc.setMergePolicy(newLogMergePolicy());
try (IndexWriter w = new IndexWriter(singleSegmentDirectory, iwc)) {
w.forceMerge(1, true);
@ -167,7 +167,7 @@ public class TestBoolean2 extends LuceneTestCase {
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
// randomized codecs are sometimes too costly for this test:
iwc.setCodec(Codec.forName("Lucene62"));
iwc.setCodec(Codec.forName("Lucene70"));
RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
w.addIndexes(copy);
copy.close();
@ -179,7 +179,7 @@ public class TestBoolean2 extends LuceneTestCase {
iwc = newIndexWriterConfig(new MockAnalyzer(random()));
iwc.setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000));
// randomized codecs are sometimes too costly for this test:
iwc.setCodec(Codec.forName("Lucene62"));
iwc.setCodec(Codec.forName("Lucene70"));
RandomIndexWriter w = new RandomIndexWriter(random(), dir2, iwc);
doc = new Document();

View File

@ -1157,14 +1157,14 @@ public class TestPointQueries extends LuceneTestCase {
}
private static Codec getCodec() {
if (Codec.getDefault().getName().equals("Lucene62")) {
if (Codec.getDefault().getName().equals("Lucene70")) {
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
double maxMBSortInHeap = 5.0 + (3*random().nextDouble());
if (VERBOSE) {
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
}
return new FilterCodec("Lucene62", Codec.getDefault()) {
return new FilterCodec("Lucene70", Codec.getDefault()) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {

View File

@ -247,7 +247,7 @@ public class TestNearest extends LuceneTestCase {
private IndexWriterConfig getIndexWriterConfig() {
IndexWriterConfig iwc = newIndexWriterConfig();
iwc.setCodec(Codec.forName("Lucene62"));
iwc.setCodec(Codec.forName("Lucene70"));
return iwc;
}
}

View File

@ -85,14 +85,14 @@ import com.carrotsearch.randomizedtesting.generators.RandomInts;
public class TestGeo3DPoint extends LuceneTestCase {
private static Codec getCodec() {
if (Codec.getDefault().getName().equals("Lucene62")) {
if (Codec.getDefault().getName().equals("Lucene70")) {
int maxPointsInLeafNode = TestUtil.nextInt(random(), 16, 2048);
double maxMBSortInHeap = 3.0 + (3*random().nextDouble());
if (VERBOSE) {
System.out.println("TEST: using Lucene60PointsFormat with maxPointsInLeafNode=" + maxPointsInLeafNode + " and maxMBSortInHeap=" + maxMBSortInHeap);
}
return new FilterCodec("Lucene62", Codec.getDefault()) {
return new FilterCodec("Lucene70", Codec.getDefault()) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {

View File

@ -32,7 +32,7 @@ import org.apache.lucene.analysis.MockAnalyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.lucene.document.IntPoint;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
@ -646,7 +646,7 @@ public class TestSuggestField extends LuceneTestCase {
static IndexWriterConfig iwcWithSuggestField(Analyzer analyzer, final Set<String> suggestFields) {
IndexWriterConfig iwc = newIndexWriterConfig(random(), analyzer);
iwc.setMergePolicy(newLogMergePolicy());
Codec filterCodec = new Lucene62Codec() {
Codec filterCodec = new Lucene70Codec() {
PostingsFormat postingsFormat = new Completion50PostingsFormat();
@Override

View File

@ -1239,7 +1239,7 @@ public abstract class BaseGeoPointTestCase extends LuceneTestCase {
// Else seeds may not reproduce:
iwc.setMergeScheduler(new SerialMergeScheduler());
int pointsInLeaf = 2 + random().nextInt(4);
iwc.setCodec(new FilterCodec("Lucene62", TestUtil.getDefaultCodec()) {
iwc.setCodec(new FilterCodec("Lucene70", TestUtil.getDefaultCodec()) {
@Override
public PointsFormat pointsFormat() {
return new PointsFormat() {

View File

@ -32,7 +32,7 @@ import org.apache.lucene.codecs.asserting.AssertingPostingsFormat;
import org.apache.lucene.codecs.cheapbastard.CheapBastardCodec;
import org.apache.lucene.codecs.compressing.CompressingCodec;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
import org.apache.lucene.index.RandomCodec;
@ -181,8 +181,8 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
codec = new AssertingCodec();
} else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
codec = CompressingCodec.randomInstance(random);
} else if ("Lucene62".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene62"))) {
codec = new Lucene62Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
} else if ("Lucene70".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene70"))) {
codec = new Lucene70Codec(RandomPicks.randomFrom(random, Lucene50StoredFieldsFormat.Mode.values()));
} else if (!"random".equals(TEST_CODEC)) {
codec = Codec.forName(TEST_CODEC);
} else if ("random".equals(TEST_POSTINGSFORMAT)) {

View File

@ -53,8 +53,8 @@ import org.apache.lucene.codecs.asserting.AssertingCodec;
import org.apache.lucene.codecs.blockterms.LuceneFixedGap;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat;
import org.apache.lucene.codecs.lucene54.Lucene54DocValuesFormat;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.lucene.codecs.lucene70.Lucene70DocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.document.BinaryDocValuesField;
@ -913,7 +913,7 @@ public final class TestUtil {
* This may be different than {@link Codec#getDefault()} because that is randomized.
*/
public static Codec getDefaultCodec() {
return new Lucene62Codec();
return new Lucene70Codec();
}
/**
@ -946,7 +946,7 @@ public final class TestUtil {
* Returns the actual default docvalues format (e.g. LuceneMNDocValuesFormat for this version of Lucene.
*/
public static DocValuesFormat getDefaultDocValuesFormat() {
return new Lucene54DocValuesFormat();
return new Lucene70DocValuesFormat();
}
// TODO: generalize all 'test-checks-for-crazy-codecs' to

View File

@ -152,6 +152,10 @@ Bug Fixes
* SOLR-9556: OverseerAutoFailoverReplicaThread was not exiting on interrupt
(Alan Woodward)
* SOLR-9563: Collection creation could fail if an ADDREPLICA subrequest arrived
at a node before its local state had updated with the new collection data
(Alan Woodward)
Optimizations
----------------------

View File

@ -59,31 +59,33 @@ print_error() {
exit 1
}
proc_version=`cat /etc/*-release 2>/dev/null`
if [[ $? -gt 0 ]]; then
if [ -f "/proc/version" ]; then
proc_version=`cat /proc/version`
else
proc_version=`uname -a`
fi
fi
if [[ $proc_version == *"Debian"* ]]; then
distro=Debian
elif [[ $proc_version == *"Red Hat"* ]]; then
distro=RedHat
elif [[ $proc_version == *"CentOS"* ]]; then
distro=CentOS
elif [[ $proc_version == *"Ubuntu"* ]]; then
distro=Ubuntu
elif [[ $proc_version == *"SUSE"* ]]; then
distro=SUSE
elif [[ $proc_version == *"Darwin"* ]]; then
echo "Sorry, this script does not support macOS. You'll need to setup Solr as a service manually using the documentation provided in the Solr Reference Guide."
echo "You could also try installing via Homebrew (http://brew.sh/), e.g. brew install solr"
exit 1
else
echo -e "\nERROR: Your Linux distribution ($proc_version) not supported by this script!\nYou'll need to setup Solr as a service manually using the documentation provided in the Solr Reference Guide.\n" 1>&2
# Locate *NIX distribution by looking for match from various detection strategies
# We start with /etc/os-release, as this will also work for Docker containers
for command in "grep -E \"^NAME=\" /etc/os-release" \
"lsb_release -i" \
"cat /proc/version" \
"uname -a" ; do
distro_string=$(eval $command 2>/dev/null)
unset distro
if [[ ${distro_string,,} == *"debian"* ]]; then
distro=Debian
elif [[ ${distro_string,,} == *"red hat"* ]]; then
distro=RedHat
elif [[ ${distro_string,,} == *"centos"* ]]; then
distro=CentOS
elif [[ ${distro_string,,} == *"ubuntu"* ]]; then
distro=Ubuntu
elif [[ ${distro_string,,} == *"suse"* ]]; then
distro=SUSE
elif [[ ${distro_string,,} == *"darwin"* ]]; then
echo "Sorry, this script does not support macOS. You'll need to setup Solr as a service manually using the documentation provided in the Solr Reference Guide."
echo "You could also try installing via Homebrew (http://brew.sh/), e.g. brew install solr"
exit 1
fi
if [[ $distro ]] ; then break ; fi
done
if [[ ! $distro ]] ; then
echo -e "\nERROR: Unable to auto-detect your *NIX distribution!\nYou'll need to setup Solr as a service manually using the documentation provided in the Solr Reference Guide.\n" 1>&2
exit 1
fi

View File

@ -43,6 +43,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference;
import com.google.common.base.Strings;
import org.apache.commons.lang.StringUtils;
@ -1524,34 +1525,43 @@ public class ZkController {
if (!Overseer.isLegacy(zkStateReader)) {
CloudDescriptor cloudDesc = cd.getCloudDescriptor();
String coreNodeName = cloudDesc.getCoreNodeName();
assert coreNodeName != null : "SolrCore: " + cd.getName() + " has no coreNodeName";
if (coreNodeName == null)
throw new SolrException(ErrorCode.SERVER_ERROR, "No coreNodeName for " + cd);
if (cloudDesc.getShardId() == null) {
throw new SolrException(ErrorCode.SERVER_ERROR, "No shard id for :" + cd);
throw new SolrException(ErrorCode.SERVER_ERROR, "No shard id for " + cd);
}
long endTime = System.nanoTime() + TimeUnit.NANOSECONDS.convert(3, TimeUnit.SECONDS);
String errMessage = null;
while (System.nanoTime() < endTime) {
Slice slice = zkStateReader.getClusterState().getSlice(cd.getCollectionName(), cloudDesc.getShardId());
if (slice == null) {
errMessage = "Invalid slice : " + cloudDesc.getShardId();
continue;
}
if (slice.getReplica(coreNodeName) != null) {
AtomicReference<String> errorMessage = new AtomicReference<>();
AtomicReference<DocCollection> collectionState = new AtomicReference<>();
try {
zkStateReader.waitForState(cd.getCollectionName(), 3, TimeUnit.SECONDS, (n, c) -> {
collectionState.set(c);
if (c == null)
return false;
Slice slice = c.getSlice(cloudDesc.getShardId());
if (slice == null) {
errorMessage.set("Invalid shard: " + cloudDesc.getShardId());
return false;
}
Replica replica = slice.getReplica(coreNodeName);
if (replica == null) {
errorMessage.set("coreNodeName " + coreNodeName + " does not exist in shard " + cloudDesc.getShardId());
return false;
}
String baseUrl = replica.getStr(BASE_URL_PROP);
String coreName = replica.getStr(CORE_NAME_PROP);
if (baseUrl.equals(this.baseURL) && coreName.equals(cd.getName())) {
return;
} else {
errMessage = "replica with coreNodeName " + coreNodeName + " exists but with a different name or base_url";
return true;
}
}
Thread.sleep(100);
errorMessage.set("coreNodeName " + coreNodeName + " exists, but does not match expected node or core name");
return false;
});
} catch (TimeoutException e) {
String error = errorMessage.get();
if (error == null)
error = "Replica " + coreNodeName + " is not present in cluster state";
throw new SolrException(ErrorCode.SERVER_ERROR, error + ": " + collectionState.get());
}
if (errMessage == null) {
errMessage = "replica " + coreNodeName + " is not present in cluster state";
}
throw new SolrException(ErrorCode.SERVER_ERROR, errMessage + ". state : " + zkStateReader.getClusterState().getCollection(cd.getCollectionName()));
}
}

View File

@ -24,7 +24,7 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene62.Lucene62Codec;
import org.apache.lucene.codecs.lucene70.Lucene70Codec;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.util.NamedList;
@ -91,7 +91,7 @@ public class SchemaCodecFactory extends CodecFactory implements SolrCoreAware {
compressionMode = SOLR_DEFAULT_COMPRESSION_MODE;
log.debug("Using default compressionMode: " + compressionMode);
}
codec = new Lucene62Codec(compressionMode) {
codec = new Lucene70Codec(compressionMode) {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
final SchemaField schemaField = core.getLatestSchema().getFieldOrNull(field);

View File

@ -19,7 +19,7 @@
<fieldType name="string_direct" class="solr.StrField" postingsFormat="Direct"/>
<fieldType name="string_standard" class="solr.StrField" postingsFormat="Lucene50"/>
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene54"/>
<fieldType name="string_disk" class="solr.StrField" docValuesFormat="Lucene70"/>
<fieldType name="string_memory" class="solr.StrField" docValuesFormat="Memory"/>
<fieldType name="string" class="solr.StrField"/>

View File

@ -16,127 +16,71 @@
*/
package org.apache.solr.cloud;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.client.solrj.SolrClient;
import java.lang.invoke.MethodHandles;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest.CreateAlias;
import org.apache.solr.client.solrj.request.CollectionAdminRequest.DeleteAlias;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.junit.Test;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.ArrayList;
import java.util.List;
/**
* Test sync phase that occurs when Leader goes down and a new Leader is
* elected.
*/
@Slow
public class AliasIntegrationTest extends AbstractFullDistribZkTestBase {
public class AliasIntegrationTest extends SolrCloudTestCase {
private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public AliasIntegrationTest() {
super();
sliceCount = 1;
fixShardCount(random().nextBoolean() ? 3 : 4);
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(2)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Test
public void test() throws Exception {
handle.clear();
handle.put("timestamp", SKIPVAL);
CollectionAdminRequest.createCollection("collection1", "conf", 2, 1).process(cluster.getSolrClient());
CollectionAdminRequest.createCollection("collection2", "conf", 1, 1).process(cluster.getSolrClient());
waitForState("Expected collection1 to be created with 2 shards and 1 replica", "collection1", clusterShape(2, 1));
waitForState("Expected collection2 to be created with 1 shard and 1 replica", "collection2", clusterShape(1, 1));
waitForThingsToLevelOut(30);
new UpdateRequest()
.add("id", "6", "a_t", "humpty dumpy sat on a wall")
.add("id", "7", "a_t", "humpty dumpy3 sat on a walls")
.add("id", "8", "a_t", "humpty dumpy2 sat on a walled")
.commit(cluster.getSolrClient(), "collection1");
logger.info("### STARTING ACTUAL TEST");
new UpdateRequest()
.add("id", "9", "a_t", "humpty dumpy sat on a wall")
.add("id", "10", "a_t", "humpty dumpy3 sat on a walls")
.commit(cluster.getSolrClient(), "collection2");
del("*:*");
createCollection("collection2", 2, 1, 10);
List<Integer> numShardsNumReplicaList = new ArrayList<>(2);
numShardsNumReplicaList.add(2);
numShardsNumReplicaList.add(1);
checkForCollection("collection2", numShardsNumReplicaList, null);
waitForRecoveriesToFinish("collection2", true);
cloudClient.setDefaultCollection("collection1");
SolrInputDocument doc1 = getDoc(id, 6, i1, -600, tlong, 600, t1,
"humpty dumpy sat on a wall");
SolrInputDocument doc2 = getDoc(id, 7, i1, -600, tlong, 600, t1,
"humpty dumpy3 sat on a walls");
SolrInputDocument doc3 = getDoc(id, 8, i1, -600, tlong, 600, t1,
"humpty dumpy2 sat on a walled");
CollectionAdminRequest.createAlias("testalias", "collection1").process(cluster.getSolrClient());
cloudClient.add(doc1);
cloudClient.add(doc2);
cloudClient.add(doc3);
cloudClient.commit();
SolrInputDocument doc6 = getDoc(id, 9, i1, -600, tlong, 600, t1,
"humpty dumpy sat on a wall");
SolrInputDocument doc7 = getDoc(id, 10, i1, -600, tlong, 600, t1,
"humpty dumpy3 sat on a walls");
cloudClient.setDefaultCollection("collection2");
cloudClient.add(doc6);
cloudClient.add(doc7);
cloudClient.commit();
// create alias
createAlias("testalias", "collection1");
// search for alias
SolrQuery query = new SolrQuery("*:*");
query.set("collection", "testalias");
QueryResponse res = cloudClient.query(query);
QueryResponse res = cluster.getSolrClient().query("testalias", new SolrQuery("*:*"));
assertEquals(3, res.getResults().getNumFound());
// search for alias with random non cloud client
query = new SolrQuery("*:*");
query.set("collection", "testalias");
JettySolrRunner jetty = jettys.get(random().nextInt(jettys.size()));
int port = jetty.getLocalPort();
try (HttpSolrClient client = getHttpSolrClient(buildUrl(port) + "/testalias")) {
res = client.query(query);
JettySolrRunner jetty = cluster.getRandomJetty(random());
try (HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/testalias")) {
res = client.query(new SolrQuery("*:*"));
assertEquals(3, res.getResults().getNumFound());
}
// now without collections param
query = new SolrQuery("*:*");
jetty = jettys.get(random().nextInt(jettys.size()));
port = jetty.getLocalPort();
try (HttpSolrClient client = getHttpSolrClient(buildUrl(port) + "/testalias")) {
res = client.query(query);
assertEquals(3, res.getResults().getNumFound());
}
// create alias, collection2 first because it's not on every node
createAlias("testalias", "collection2,collection1");
CollectionAdminRequest.createAlias("testalias", "collection2,collection1").process(cluster.getSolrClient());
// search with new cloud client
try (CloudSolrClient cloudSolrClient = getCloudSolrClient(zkServer.getZkAddress(), random().nextBoolean())) {
try (CloudSolrClient cloudSolrClient = getCloudSolrClient(cluster.getZkServer().getZkAddress(), random().nextBoolean())) {
cloudSolrClient.setParallelUpdates(random().nextBoolean());
query = new SolrQuery("*:*");
SolrQuery query = new SolrQuery("*:*");
query.set("collection", "testalias");
res = cloudSolrClient.query(query);
assertEquals(5, res.getResults().getNumFound());
@ -149,141 +93,76 @@ public class AliasIntegrationTest extends AbstractFullDistribZkTestBase {
}
// search for alias with random non cloud client
query = new SolrQuery("*:*");
query.set("collection", "testalias");
jetty = jettys.get(random().nextInt(jettys.size()));
port = jetty.getLocalPort();
try (HttpSolrClient client = getHttpSolrClient(buildUrl(port) + "/testalias")) {
jetty = cluster.getRandomJetty(random());
try (HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/testalias")) {
SolrQuery query = new SolrQuery("*:*");
query.set("collection", "testalias");
res = client.query(query);
assertEquals(5, res.getResults().getNumFound());
}
// now without collections param
query = new SolrQuery("*:*");
jetty = jettys.get(random().nextInt(jettys.size()));
port = jetty.getLocalPort();
try (HttpSolrClient client = getHttpSolrClient(buildUrl(port) + "/testalias")) {
// now without collections param
query = new SolrQuery("*:*");
res = client.query(query);
assertEquals(5, res.getResults().getNumFound());
}
// update alias
createAlias("testalias", "collection2");
//checkForAlias("testalias", "collection2");
CollectionAdminRequest.createAlias("testalias", "collection2").process(cluster.getSolrClient());
// search for alias
query = new SolrQuery("*:*");
SolrQuery query = new SolrQuery("*:*");
query.set("collection", "testalias");
res = cloudClient.query(query);
res = cluster.getSolrClient().query(query);
assertEquals(2, res.getResults().getNumFound());
// set alias to two collections
createAlias("testalias", "collection1,collection2");
//checkForAlias("testalias", "collection1,collection2");
CollectionAdminRequest.createAlias("testalias", "collection1,collection2").process(cluster.getSolrClient());
query = new SolrQuery("*:*");
query.set("collection", "testalias");
res = cloudClient.query(query);
res = cluster.getSolrClient().query(query);
assertEquals(5, res.getResults().getNumFound());
// try a std client
// search 1 and 2, but have no collections param
query = new SolrQuery("*:*");
try (HttpSolrClient client = getHttpSolrClient(getBaseUrl((HttpSolrClient) clients.get(0)) + "/testalias")) {
try (HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/testalias")) {
res = client.query(query);
assertEquals(5, res.getResults().getNumFound());
}
createAlias("testalias", "collection2");
CollectionAdminRequest.createAlias("testalias", "collection2").process(cluster.getSolrClient());
// a second alias
createAlias("testalias2", "collection2");
CollectionAdminRequest.createAlias("testalias2", "collection2").process(cluster.getSolrClient());
try (HttpSolrClient client = getHttpSolrClient(getBaseUrl((HttpSolrClient) clients.get(0)) + "/testalias")) {
SolrInputDocument doc8 = getDoc(id, 11, i1, -600, tlong, 600, t1,
"humpty dumpy4 sat on a walls");
client.add(doc8);
client.commit();
try (HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString() + "/testalias")) {
new UpdateRequest()
.add("id", "11", "a_t", "humpty dumpy4 sat on a walls")
.commit(cluster.getSolrClient(), "testalias");
res = client.query(query);
assertEquals(3, res.getResults().getNumFound());
}
createAlias("testalias", "collection2,collection1");
CollectionAdminRequest.createAlias("testalias", "collection2,collection1").process(cluster.getSolrClient());
query = new SolrQuery("*:*");
query.set("collection", "testalias");
res = cloudClient.query(query);
res = cluster.getSolrClient().query(query);
assertEquals(6, res.getResults().getNumFound());
deleteAlias("testalias");
deleteAlias("testalias2");
boolean sawException = false;
try {
res = cloudClient.query(query);
} catch (SolrException e) {
sawException = true;
}
assertTrue(sawException);
CollectionAdminRequest.deleteAlias("testalias").process(cluster.getSolrClient());
CollectionAdminRequest.deleteAlias("testalias2").process(cluster.getSolrClient());
SolrException e = expectThrows(SolrException.class, () -> {
SolrQuery q = new SolrQuery("*:*");
q.set("collection", "testalias");
cluster.getSolrClient().query(q);
});
assertTrue("Unexpected exception message: " + e.getMessage(), e.getMessage().contains("Collection not found: testalias"));
logger.info("### FINISHED ACTUAL TEST");
}
private void createAlias(String alias, String collections)
throws SolrServerException, IOException {
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
if (random().nextBoolean()) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set("collections", collections);
params.set("name", alias);
params.set("action", CollectionAction.CREATEALIAS.toString());
QueryRequest request = new QueryRequest(params);
request.setPath("/admin/collections");
client.request(request);
} else {
CreateAlias request = new CreateAlias();
request.setAliasName(alias);
request.setAliasedCollections(collections);
request.process(client);
}
}
}
private void deleteAlias(String alias) throws SolrServerException,
IOException {
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
if (random().nextBoolean()) {
ModifiableSolrParams params = new ModifiableSolrParams();
params.set("name", alias);
params.set("action", CollectionAction.DELETEALIAS.toString());
QueryRequest request = new QueryRequest(params);
request.setPath("/admin/collections");
client.request(request);
} else {
DeleteAlias request = new DeleteAlias();
request.setAliasName(alias);
request.process(client);
}
}
}
protected void indexDoc(List<CloudJettyRunner> skipServers, Object... fields) throws IOException,
SolrServerException {
SolrInputDocument doc = new SolrInputDocument();
addFields(doc, fields);
addFields(doc, "rnd_b", true);
controlClient.add(doc);
UpdateRequest ureq = new UpdateRequest();
ureq.add(doc);
ModifiableSolrParams params = new ModifiableSolrParams();
for (CloudJettyRunner skip : skipServers) {
params.add("test.distrib.skip.servers", skip.url + "/");
}
ureq.setParams(params);
ureq.process(cloudClient);
}
}

View File

@ -16,29 +16,45 @@
*/
package org.apache.solr.cloud;
import java.util.concurrent.TimeUnit;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.common.util.NamedList;
import org.junit.BeforeClass;
import org.junit.Test;
public class AsyncCallRequestStatusResponseTest extends AbstractFullDistribZkTestBase {
public class AsyncCallRequestStatusResponseTest extends SolrCloudTestCase {
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(2)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@ShardsFixed(num = 2)
@Test
public void testAsyncCallStatusResponse() throws Exception {
CollectionAdminRequest.Create create = new CollectionAdminRequest.Create();
create.setCollectionName("asynccall")
.setNumShards(2)
.setAsyncId("1000")
.setConfigName("conf1")
.process(cloudClient);
waitForCollection(cloudClient.getZkStateReader(), "asynccall", 2);
final RequestStatusState state = getRequestStateAfterCompletion("1000", 30, cloudClient);
assertSame(RequestStatusState.COMPLETED, state);
CollectionAdminRequest.RequestStatus requestStatus = new CollectionAdminRequest.RequestStatus();
requestStatus.setRequestId("1000");
CollectionAdminResponse rsp = requestStatus.process(cloudClient);
String asyncId =
CollectionAdminRequest.createCollection("asynccall", "conf", 2, 1).processAsync(cluster.getSolrClient());
waitForState("Expected collection 'asynccall' to have 2 shards and 1 replica", "asynccall", clusterShape(2, 1));
int tries = 0;
while (true) {
final RequestStatusState state
= CollectionAdminRequest.requestStatus(asyncId).process(cluster.getSolrClient()).getRequestStatus();
if (state == RequestStatusState.COMPLETED)
break;
if (tries++ > 10)
fail("Expected to see RequestStatusState.COMPLETED but was " + state.toString());
TimeUnit.SECONDS.sleep(1);
}
CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus(asyncId);
CollectionAdminResponse rsp = requestStatus.process(cluster.getSolrClient());
NamedList<?> r = rsp.getResponse();
// Check that there's more response than the hardcoded status and states
assertEquals("Assertion Failure" + r.toString(), 5, r.size());

View File

@ -1,121 +0,0 @@
/*
* 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.
*/
package org.apache.solr.cloud;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CommonAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
import org.junit.Test;
import java.io.IOException;
public class AsyncMigrateRouteKeyTest extends MigrateRouteKeyTest {
public AsyncMigrateRouteKeyTest() {
schemaString = "schema15.xml"; // we need a string id
}
private static final int MAX_WAIT_SECONDS = 2 * 60;
@Test
public void test() throws Exception {
waitForThingsToLevelOut(15);
multipleShardMigrateTest();
printLayout();
}
protected void checkAsyncRequestForCompletion(String asyncId) throws SolrServerException, IOException {
ModifiableSolrParams params;
String message;
params = new ModifiableSolrParams();
params.set("action", CollectionParams.CollectionAction.REQUESTSTATUS.toString());
params.set(OverseerCollectionMessageHandler.REQUESTID, asyncId);
// This task takes long enough to run. Also check for the current state of the task to be running.
message = sendStatusRequestWithRetry(params, 5);
assertEquals("found [" + asyncId + "] in running tasks", message);
// Now wait until the task actually completes successfully/fails.
message = sendStatusRequestWithRetry(params, MAX_WAIT_SECONDS);
assertEquals("Task " + asyncId + " not found in completed tasks.",
"found [" + asyncId + "] in completed tasks", message);
}
@Override
protected void invokeMigrateApi(String sourceCollection, String splitKey, String targetCollection) throws SolrServerException, IOException {
ModifiableSolrParams params = new ModifiableSolrParams();
String asyncId = "20140128";
params.set(CollectionParams.ACTION, CollectionParams.CollectionAction.MIGRATE.toString());
params.set("collection", sourceCollection);
params.set("target.collection", targetCollection);
params.set("split.key", splitKey);
params.set("forward.timeout", 45);
params.set(CommonAdminParams.ASYNC, asyncId);
invoke(params);
checkAsyncRequestForCompletion(asyncId);
}
/**
* Helper method to send a status request with specific retry limit and return
* the message/null from the success response.
*/
private String sendStatusRequestWithRetry(ModifiableSolrParams params, int maxCounter)
throws SolrServerException, IOException {
NamedList status = null;
RequestStatusState state = null;
String message = null;
NamedList r;
while (maxCounter-- > 0) {
r = sendRequest(params);
status = (NamedList) r.get("status");
state = RequestStatusState.fromKey((String) status.get("state"));
message = (String) status.get("msg");
if (state == RequestStatusState.COMPLETED || state == RequestStatusState.FAILED) {
return (String) status.get("msg");
}
try {
Thread.sleep(1000);
} catch (InterruptedException e) {
}
}
// Return last state?
return message;
}
protected NamedList sendRequest(ModifiableSolrParams params) throws SolrServerException, IOException {
final SolrRequest request = new QueryRequest(params);
request.setPath("/admin/collections");
String baseUrl = ((HttpSolrClient) shardToJetty.get(SHARD1).get(0).client.solrClient).getBaseURL();
baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
try (HttpSolrClient baseServer = getHttpSolrClient(baseUrl)) {
baseServer.setConnectionTimeout(15000);
return baseServer.request(request);
}
}
}

View File

@ -19,13 +19,11 @@ package org.apache.solr.cloud;
import java.lang.invoke.MethodHandles;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.util.RetryUtil;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -33,68 +31,53 @@ import org.slf4j.LoggerFactory;
/**
* Verifies cluster state remains consistent after collection reload.
*/
@Slow
@SuppressSSL(bugUrl = "https://issues.apache.org/jira/browse/SOLR-5776")
public class CollectionReloadTest extends AbstractFullDistribZkTestBase {
public class CollectionReloadTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public CollectionReloadTest() {
super();
sliceCount = 1;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(1)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Test
public void testReloadedLeaderStateAfterZkSessionLoss() throws Exception {
waitForThingsToLevelOut(30000);
log.info("testReloadedLeaderStateAfterZkSessionLoss initialized OK ... running test logic");
String testCollectionName = "c8n_1x1";
String shardId = "shard1";
createCollectionRetry(testCollectionName, 1, 1, 1);
cloudClient.setDefaultCollection(testCollectionName);
final String testCollectionName = "c8n_1x1";
CollectionAdminRequest.createCollection(testCollectionName, "conf", 1, 1)
.process(cluster.getSolrClient());
Replica leader = getShardLeader(testCollectionName, shardId, 30 /* timeout secs */);
Replica leader
= cluster.getSolrClient().getZkStateReader().getLeaderRetry(testCollectionName, "shard1", DEFAULT_TIMEOUT);
// reload collection and wait to see the core report it has been reloaded
boolean wasReloaded = reloadCollection(leader, testCollectionName);
assertTrue("Collection '"+testCollectionName+"' failed to reload within a reasonable amount of time!",
wasReloaded);
long coreStartTime = getCoreStatus(leader).getCoreStartTime().getTime();
CollectionAdminRequest.reloadCollection(testCollectionName).process(cluster.getSolrClient());
// cause session loss
chaosMonkey.expireSession(getJettyOnPort(getReplicaPort(leader)));
RetryUtil.retryUntil("Timed out waiting for core to reload", 30, 1000, TimeUnit.MILLISECONDS, () -> {
long restartTime = 0;
try {
restartTime = getCoreStatus(leader).getCoreStartTime().getTime();
} catch (Exception e) {
log.warn("Exception getting core start time: {}", e.getMessage());
return false;
}
return restartTime > coreStartTime;
});
// TODO: have to wait a while for the node to get marked down after ZK session loss
// but tests shouldn't be so timing dependent!
Thread.sleep(15000);
final int initialStateVersion = getCollectionState(testCollectionName).getZNodeVersion();
// wait up to 15 seconds to see the replica in the active state
String replicaState = null;
int timeoutSecs = 15;
long timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutSecs, TimeUnit.SECONDS);
while (System.nanoTime() < timeout) {
// state of leader should be active after session loss recovery - see SOLR-7338
cloudClient.getZkStateReader().forceUpdateCollection(testCollectionName);
ClusterState cs = cloudClient.getZkStateReader().getClusterState();
Slice slice = cs.getSlice(testCollectionName, shardId);
replicaState = slice.getReplica(leader.getName()).getStr(ZkStateReader.STATE_PROP);
if ("active".equals(replicaState))
break;
cluster.expireZkSession(cluster.getReplicaJetty(leader));
Thread.sleep(1000);
}
assertEquals("Leader state should be active after recovering from ZK session loss, but after " +
timeoutSecs + " seconds, it is " + replicaState, "active", replicaState);
// try to clean up
try {
new CollectionAdminRequest.Delete()
.setCollectionName(testCollectionName).process(cloudClient);
} catch (Exception e) {
// don't fail the test
log.warn("Could not delete collection {} after test completed", testCollectionName);
}
waitForState("Timed out waiting for core to re-register as ACTIVE after session expiry", testCollectionName, (n, c) -> {
log.info("Collection state: {}", c.toString());
Replica expiredReplica = c.getReplica(leader.getName());
return expiredReplica.getState() == Replica.State.ACTIVE && c.getZNodeVersion() > initialStateVersion;
});
log.info("testReloadedLeaderStateAfterZkSessionLoss succeeded ... shutting down now!");
}

View File

@ -16,74 +16,55 @@
*/
package org.apache.solr.cloud;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.zookeeper.data.Stat;
import org.junit.BeforeClass;
import org.junit.Test;
public class CollectionStateFormat2Test extends AbstractFullDistribZkTestBase {
public class CollectionStateFormat2Test extends SolrCloudTestCase {
protected String getSolrXml() {
return "solr.xml";
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(4)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@Test
@ShardsFixed(num = 4)
public void test() throws Exception {
try (CloudSolrClient client = createCloudClient(null)) {
testZkNodeLocation(client);
testConfNameAndCollectionNameSame(client);
}
}
public void testConfNameAndCollectionNameSame() throws Exception {
@Override
protected String getStateFormat() {
return "2";
}
private void testConfNameAndCollectionNameSame(CloudSolrClient client) throws Exception{
// .system collection precreates the configset
createCollection(".system", client, 2, 1);
waitForRecoveriesToFinish(".system", false);
CollectionAdminRequest.createCollection(".system", 2, 1)
.process(cluster.getSolrClient());
}
private void testZkNodeLocation(CloudSolrClient client) throws Exception{
@Test
public void testZkNodeLocation() throws Exception {
String collectionName = "myExternColl";
CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
.process(cluster.getSolrClient());
createCollection(collectionName, client, 2, 2);
waitForState("Collection not created", collectionName, (n, c) -> DocCollection.isFullyActive(n, c, 2, 2));
assertTrue("State Format 2 collection path does not exist",
zkClient().exists(ZkStateReader.getCollectionPath(collectionName), true));
waitForRecoveriesToFinish(collectionName, false);
assertTrue("does not exist collection state externally",
cloudClient.getZkStateReader().getZkClient().exists(ZkStateReader.getCollectionPath(collectionName), true));
Stat stat = new Stat();
byte[] data = cloudClient.getZkStateReader().getZkClient().getData(ZkStateReader.getCollectionPath(collectionName), null, stat, true);
DocCollection c = ZkStateReader.getCollectionLive(cloudClient.getZkStateReader(), collectionName);
ClusterState clusterState = cloudClient.getZkStateReader().getClusterState();
assertEquals("The zkversion of the nodes must be same zkver:" + stat.getVersion() , stat.getVersion(),clusterState.getCollection(collectionName).getZNodeVersion() );
assertTrue("DocCllection#getStateFormat() must be > 1", cloudClient.getZkStateReader().getClusterState().getCollection(collectionName).getStateFormat() > 1);
zkClient().getData(ZkStateReader.getCollectionPath(collectionName), null, stat, true);
DocCollection c = getCollectionState(collectionName);
assertEquals("DocCollection version should equal the znode version", stat.getVersion(), c.getZNodeVersion() );
assertTrue("DocCollection#getStateFormat() must be > 1", c.getStateFormat() > 1);
// remove collection
ModifiableSolrParams params = new ModifiableSolrParams();
params.set("action", CollectionParams.CollectionAction.DELETE.toString());
params.set("name", collectionName);
QueryRequest request = new QueryRequest(params);
request.setPath("/admin/collections");
if (client == null) {
client = createCloudClient(null);
}
CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
waitForState("Collection not deleted", collectionName, (n, coll) -> coll == null);
client.request(request);
assertCollectionNotExists(collectionName, 45);
assertFalse("collection state should not exist externally", cloudClient.getZkStateReader().getZkClient().exists(ZkStateReader.getCollectionPath(collectionName), true));
assertFalse("collection state should not exist externally",
zkClient().exists(ZkStateReader.getCollectionPath(collectionName), true));
}
}

View File

@ -85,7 +85,7 @@ public class DeleteInactiveReplicaTest extends SolrCloudTestCase {
CoreContainer.CoreLoadFailure loadFailure = cc.getCoreInitFailures().get(replica.getCoreName());
assertNotNull("Deleted core was still loaded!", loadFailure);
assertTrue("Unexpected load failure message: " + loadFailure.exception.getMessage(),
loadFailure.exception.getMessage().contains("not present in cluster state"));
loadFailure.exception.getMessage().contains("does not exist in shard"));
// Check that we can't create a core with no coreNodeName
try (SolrClient queryClient = getHttpSolrClient(jetty.getBaseUrl().toString())) {

View File

@ -16,65 +16,55 @@
*/
package org.apache.solr.cloud;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrRequest;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.response.QueryResponse;
import org.apache.solr.common.SolrInputDocument;
import org.apache.solr.common.cloud.ClusterState;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.RoutingRule;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.Utils;
import org.apache.solr.util.TimeOut;
import org.apache.zookeeper.KeeperException;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
@LuceneTestCase.Slow
public class MigrateRouteKeyTest extends SolrCloudTestCase {
import static org.apache.solr.cloud.OverseerCollectionMessageHandler.NUM_SLICES;
import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(2)
.addConfig("conf", configset("cloud-minimal"))
.configure();
public class MigrateRouteKeyTest extends BasicDistributedZkTest {
public MigrateRouteKeyTest() {
schemaString = "schema15.xml"; // we need a string id
if (usually()) {
CollectionAdminRequest.setClusterProperty("legacyCloud", "false").process(cluster.getSolrClient());
}
}
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@Test
public void test() throws Exception {
waitForThingsToLevelOut(15);
if (usually()) {
log.info("Using legacyCloud=false for cluster");
CollectionsAPIDistributedZkTest.setClusterProp(cloudClient, "legacyCloud", "false");
}
multipleShardMigrateTest();
printLayout();
}
private boolean waitForRuleToExpire(String splitKey, long finishTime) throws KeeperException, InterruptedException, SolrServerException, IOException {
ClusterState state;Slice slice;
private boolean waitForRuleToExpire(String collection, String shard, String splitKey, long finishTime) throws KeeperException, InterruptedException, SolrServerException, IOException {
DocCollection state;
Slice slice;
boolean ruleRemoved = false;
long expiryTime = finishTime + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
while (System.nanoTime() < expiryTime) {
getCommonCloudSolrClient().getZkStateReader().forceUpdateCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
state = getCommonCloudSolrClient().getZkStateReader().getClusterState();
slice = state.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2);
cluster.getSolrClient().getZkStateReader().forceUpdateCollection(collection);
state = getCollectionState(collection);
slice = state.getSlice(shard);
Map<String,RoutingRule> routingRules = slice.getRoutingRules();
if (routingRules == null || routingRules.isEmpty() || !routingRules.containsKey(splitKey)) {
ruleRemoved = true;
@ -82,59 +72,28 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest {
}
SolrInputDocument doc = new SolrInputDocument();
doc.addField("id", splitKey + random().nextInt());
cloudClient.add(doc);
cluster.getSolrClient().add(collection, doc);
Thread.sleep(1000);
}
return ruleRemoved;
}
protected void invokeMigrateApi(String sourceCollection, String splitKey, String targetCollection) throws SolrServerException, IOException {
cloudClient.setDefaultCollection(sourceCollection);
CollectionAdminRequest.Migrate migrateRequest = new CollectionAdminRequest.Migrate();
migrateRequest.setCollectionName(sourceCollection);
migrateRequest.setTargetCollection(targetCollection);
migrateRequest.setSplitKey(splitKey);
migrateRequest.setForwardTimeout(45);
migrateRequest.process(cloudClient);
}
protected void invoke(ModifiableSolrParams params) throws SolrServerException, IOException {
SolrRequest request = new QueryRequest(params);
request.setPath("/admin/collections");
String baseUrl = ((HttpSolrClient) shardToJetty.get(SHARD1).get(0).client.solrClient)
.getBaseURL();
baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
try (HttpSolrClient baseClient = getHttpSolrClient(baseUrl)) {
baseClient.setConnectionTimeout(15000);
baseClient.setSoTimeout(60000 * 5);
baseClient.request(request);
protected void invokeCollectionMigration(CollectionAdminRequest.AsyncCollectionAdminRequest request) throws IOException, SolrServerException, InterruptedException {
if (random().nextBoolean()) {
cluster.getSolrClient().setSoTimeout(60000); // can take a while
request.process(cluster.getSolrClient());
}
else {
request.processAndWait(cluster.getSolrClient(), 60000);
}
}
private void createCollection(String targetCollection) throws Exception {
HashMap<String, List<Integer>> collectionInfos = new HashMap<>();
@Test
public void multipleShardMigrateTest() throws Exception {
try (CloudSolrClient client = createCloudClient(null)) {
Map<String, Object> props = Utils.makeMap(
REPLICATION_FACTOR, 1,
MAX_SHARDS_PER_NODE, 5,
NUM_SLICES, 1);
CollectionAdminRequest.createCollection("sourceCollection", "conf", 2, 1).process(cluster.getSolrClient());
cluster.getSolrClient().setDefaultCollection("sourceCollection");
createCollection(collectionInfos, targetCollection, props, client);
}
List<Integer> list = collectionInfos.get(targetCollection);
checkForCollection(targetCollection, list, null);
waitForRecoveriesToFinish(targetCollection, false);
}
protected void multipleShardMigrateTest() throws Exception {
del("*:*");
commit();
assertTrue(cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound() == 0);
final String splitKey = "a";
final int BIT_SEP = 1;
final int[] splitKeyCount = new int[1];
@ -147,38 +106,41 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest {
SolrInputDocument doc = new SolrInputDocument();
doc.addField("id", key + "!" + id);
doc.addField("n_ti", id);
cloudClient.add(doc);
cluster.getSolrClient().add("sourceCollection", doc);
if (splitKey.equals(shardKey))
splitKeyCount[0]++;
}
assertTrue(splitKeyCount[0] > 0);
String targetCollection = "migrate_multipleshardtest_targetCollection";
createCollection(targetCollection);
CollectionAdminRequest.createCollection(targetCollection, "conf", 1, 1).process(cluster.getSolrClient());
Indexer indexer = new Indexer(cloudClient, splitKey, 1, 30);
Indexer indexer = new Indexer(cluster.getSolrClient(), splitKey, 1, 30);
indexer.start();
String url = getUrlFromZk(getCommonCloudSolrClient().getZkStateReader().getClusterState(), targetCollection);
try (HttpSolrClient collectionClient = getHttpSolrClient(url)) {
DocCollection state = getCollectionState(targetCollection);
Replica replica = state.getReplicas().get(0);
try (HttpSolrClient collectionClient = getHttpSolrClient(replica.getCoreUrl())) {
SolrQuery solrQuery = new SolrQuery("*:*");
assertEquals("DocCount on target collection does not match", 0, collectionClient.query(solrQuery).getResults().getNumFound());
invokeMigrateApi(AbstractDistribZkTestBase.DEFAULT_COLLECTION, splitKey + "/" + BIT_SEP + "!", targetCollection);
invokeCollectionMigration(
CollectionAdminRequest.migrateData("sourceCollection", targetCollection, splitKey + "/" + BIT_SEP + "!")
.setForwardTimeout(45));
long finishTime = System.nanoTime();
indexer.join();
splitKeyCount[0] += indexer.getSplitKeyCount();
try {
cloudClient.deleteById("a/" + BIT_SEP + "!104");
cluster.getSolrClient().deleteById("a/" + BIT_SEP + "!104");
splitKeyCount[0]--;
} catch (Exception e) {
log.warn("Error deleting document a/" + BIT_SEP + "!104", e);
}
cloudClient.commit();
cluster.getSolrClient().commit();
collectionClient.commit();
solrQuery = new SolrQuery("*:*").setRows(1000);
@ -186,14 +148,20 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest {
log.info("Response from target collection: " + response);
assertEquals("DocCount on target collection does not match", splitKeyCount[0], response.getResults().getNumFound());
getCommonCloudSolrClient().getZkStateReader().forceUpdateCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
ClusterState state = getCommonCloudSolrClient().getZkStateReader().getClusterState();
Slice slice = state.getSlice(AbstractDistribZkTestBase.DEFAULT_COLLECTION, SHARD2);
assertNotNull("Routing rule map is null", slice.getRoutingRules());
assertFalse("Routing rule map is empty", slice.getRoutingRules().isEmpty());
assertNotNull("No routing rule exists for route key: " + splitKey, slice.getRoutingRules().get(splitKey + "!"));
waitForState("Expected to find routing rule for split key " + splitKey, "sourceCollection", (n, c) -> {
if (c == null)
return false;
Slice shard = c.getSlice("shard2");
if (shard == null)
return false;
if (shard.getRoutingRules() == null || shard.getRoutingRules().isEmpty())
return false;
if (shard.getRoutingRules().get(splitKey + "!") == null)
return false;
return true;
});
boolean ruleRemoved = waitForRuleToExpire(splitKey, finishTime);
boolean ruleRemoved = waitForRuleToExpire("sourceCollection", "shard2", splitKey, finishTime);
assertTrue("Routing rule was not expired", ruleRemoved);
}
}
@ -230,7 +198,7 @@ public class MigrateRouteKeyTest extends BasicDistributedZkTest {
try {
Thread.sleep(50);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
return;
}
}
}

View File

@ -17,23 +17,19 @@
package org.apache.solr.cloud.rule;
import java.lang.invoke.MethodHandles;
import java.nio.file.Paths;
import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.solr.client.solrj.SolrClient;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.HttpSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.GenericSolrRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.cloud.AbstractFullDistribZkTestBase;
import org.apache.solr.cloud.SolrCloudTestCase;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import org.slf4j.Logger;
@ -43,34 +39,35 @@ import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
import static org.apache.solr.common.params.CommonParams.COLLECTIONS_HANDLER_PATH;
import static org.junit.matchers.JUnitMatchers.containsString;
public class RulesTest extends AbstractFullDistribZkTestBase {
@LuceneTestCase.Slow
public class RulesTest extends SolrCloudTestCase {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
@BeforeClass
public static void setupCluster() throws Exception {
configureCluster(5)
.addConfig("conf", configset("cloud-minimal"))
.configure();
}
@org.junit.Rule
public ExpectedException expectedException = ExpectedException.none();
@Test
@ShardsFixed(num = 5)
public void doIntegrationTest() throws Exception {
final long minGB = (random().nextBoolean() ? 1 : 0);
assumeTrue("doIntegrationTest needs minGB="+minGB+" usable disk space", ImplicitSnitch.getUsableSpaceInGB(Paths.get("/")) > minGB);
assumeTrue("doIntegrationTest needs minGB="+minGB+" usable disk space",
ImplicitSnitch.getUsableSpaceInGB(Paths.get("/")) > minGB);
String rulesColl = "rulesColl";
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
CollectionAdminResponse rsp;
CollectionAdminRequest.Create create = new CollectionAdminRequest.Create()
.setCollectionName(rulesColl)
.setShards("shard1")
.setRouterName(ImplicitDocRouter.NAME)
.setReplicationFactor(2)
.setRule("cores:<4", "node:*,replica:<2", "freedisk:>"+minGB)
.setSnitch("class:ImplicitSnitch");
rsp = create.process(client);
assertEquals(0, rsp.getStatus());
assertTrue(rsp.isSuccess());
CollectionAdminRequest.createCollectionWithImplicitRouter(rulesColl, "conf", "shard1", 2)
.setRule("cores:<4", "node:*,replica:<2", "freedisk:>"+minGB)
.setSnitch("class:ImplicitSnitch")
.process(cluster.getSolrClient());
}
DocCollection rulesCollection = getCollectionState(rulesColl);
DocCollection rulesCollection = cloudClient.getZkStateReader().getClusterState().getCollection(rulesColl);
List list = (List) rulesCollection.get("rule");
assertEquals(3, list.size());
assertEquals ( "<4", ((Map)list.get(0)).get("cores"));
@ -80,51 +77,25 @@ public class RulesTest extends AbstractFullDistribZkTestBase {
assertEquals(1, list.size());
assertEquals ( "ImplicitSnitch", ((Map)list.get(0)).get("class"));
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
CollectionAdminResponse rsp;
CollectionAdminRequest.CreateShard createShard = new CollectionAdminRequest.CreateShard()
.setCollectionName(rulesColl)
.setShardName("shard2");
rsp = createShard.process(client);
assertEquals(0, rsp.getStatus());
assertTrue(rsp.isSuccess());
CollectionAdminRequest.AddReplica addReplica = new CollectionAdminRequest.AddReplica()
.setCollectionName(rulesColl)
.setShardName("shard2");
rsp = addReplica.process(client);
assertEquals(0, rsp.getStatus());
assertTrue(rsp.isSuccess());
}
CollectionAdminRequest.createShard(rulesColl, "shard2").process(cluster.getSolrClient());
CollectionAdminRequest.addReplicaToShard(rulesColl, "shard2").process(cluster.getSolrClient());
}
@Test
public void testPortRule() throws Exception {
JettySolrRunner jetty = cluster.getRandomJetty(random());
String port = Integer.toString(jetty.getLocalPort());
String rulesColl = "portRuleColl";
String baseUrl = getBaseUrl((HttpSolrClient) clients.get(0));
String port = "-1";
Matcher hostAndPortMatcher = Pattern.compile("(?:https?://)?([^:]+):(\\d+)").matcher(baseUrl);
if (hostAndPortMatcher.find()) {
port = hostAndPortMatcher.group(2);
}
try (SolrClient client = createNewSolrClient("", baseUrl)) {
CollectionAdminResponse rsp;
CollectionAdminRequest.Create create = new CollectionAdminRequest.Create();
create.setCollectionName(rulesColl);
create.setShards("shard1");
create.setRouterName(ImplicitDocRouter.NAME);
create.setReplicationFactor(2);
create.setRule("port:" + port);
create.setSnitch("class:ImplicitSnitch");
rsp = create.process(client);
assertEquals(0, rsp.getStatus());
assertTrue(rsp.isSuccess());
CollectionAdminRequest.createCollectionWithImplicitRouter(rulesColl, "conf", "shard1", 2)
.setRule("port:" + port)
.setSnitch("class:ImplicitSnitch")
.process(cluster.getSolrClient());
}
DocCollection rulesCollection = getCollectionState(rulesColl);
DocCollection rulesCollection = cloudClient.getZkStateReader().getClusterState().getCollection(rulesColl);
List list = (List) rulesCollection.get("rule");
assertEquals(1, list.size());
assertEquals(port, ((Map) list.get(0)).get("port"));
@ -135,33 +106,21 @@ public class RulesTest extends AbstractFullDistribZkTestBase {
@Test
public void testHostFragmentRule() throws Exception {
String rulesColl = "ipRuleColl";
String baseUrl = getBaseUrl((HttpSolrClient) clients.get(0));
String ip_1 = "-1";
String ip_2 = "-1";
Matcher hostAndPortMatcher = Pattern.compile("(?:https?://)?([^:]+):(\\d+)").matcher(baseUrl);
if (hostAndPortMatcher.find()) {
String[] ipFragments = hostAndPortMatcher.group(1).split("\\.");
ip_1 = ipFragments[ipFragments.length - 1];
ip_2 = ipFragments[ipFragments.length - 2];
}
try (SolrClient client = createNewSolrClient("", baseUrl)) {
CollectionAdminResponse rsp;
CollectionAdminRequest.Create create = new CollectionAdminRequest.Create();
create.setCollectionName(rulesColl);
create.setShards("shard1");
create.setRouterName(ImplicitDocRouter.NAME);
create.setReplicationFactor(2);
create.setRule("ip_2:" + ip_2, "ip_1:" + ip_1);
create.setSnitch("class:ImplicitSnitch");
rsp = create.process(client);
assertEquals(0, rsp.getStatus());
assertTrue(rsp.isSuccess());
String rulesColl = "hostFragment";
}
JettySolrRunner jetty = cluster.getRandomJetty(random());
String host = jetty.getBaseUrl().getHost();
String[] ipFragments = host.split("\\.");
String ip_1 = ipFragments[ipFragments.length - 1];
String ip_2 = ipFragments[ipFragments.length - 2];
DocCollection rulesCollection = cloudClient.getZkStateReader().getClusterState().getCollection(rulesColl);
CollectionAdminRequest.createCollectionWithImplicitRouter(rulesColl, "conf", "shard1", 2)
.setRule("ip_2:" + ip_2, "ip_1:" + ip_1)
.setSnitch("class:ImplicitSnitch")
.process(cluster.getSolrClient());
DocCollection rulesCollection = getCollectionState(rulesColl);
List<Map> list = (List<Map>) rulesCollection.get("rule");
assertEquals(2, list.size());
assertEquals(ip_2, list.get(0).get("ip_2"));
@ -175,66 +134,54 @@ public class RulesTest extends AbstractFullDistribZkTestBase {
@Test
public void testHostFragmentRuleThrowsExceptionWhenIpDoesNotMatch() throws Exception {
String rulesColl = "ipRuleColl";
String baseUrl = getBaseUrl((HttpSolrClient) clients.get(0));
String ip_1 = "-1";
String ip_2 = "-1";
Matcher hostAndPortMatcher = Pattern.compile("(?:https?://)?([^:]+):(\\d+)").matcher(baseUrl);
if (hostAndPortMatcher.find()) {
String[] ipFragments = hostAndPortMatcher.group(1).split("\\.");
ip_1 = ipFragments[ipFragments.length - 1];
ip_2 = ipFragments[ipFragments.length - 2];
}
try (SolrClient client = createNewSolrClient("", baseUrl)) {
CollectionAdminRequest.Create create = new CollectionAdminRequest.Create();
create.setCollectionName(rulesColl);
create.setShards("shard1");
create.setRouterName(ImplicitDocRouter.NAME);
create.setReplicationFactor(2);
JettySolrRunner jetty = cluster.getRandomJetty(random());
String host = jetty.getBaseUrl().getHost();
String[] ipFragments = host.split("\\.");
String ip_1 = ipFragments[ipFragments.length - 1];
String ip_2 = ipFragments[ipFragments.length - 2];
create.setRule("ip_2:" + ip_2, "ip_1:" + ip_1 + "9999");
create.setSnitch("class:ImplicitSnitch");
expectedException.expect(HttpSolrClient.RemoteSolrException.class);
expectedException.expectMessage(containsString("ip_1"));
expectedException.expect(HttpSolrClient.RemoteSolrException.class);
expectedException.expectMessage(containsString("ip_1"));
create.process(client);
}
CollectionAdminRequest.createCollectionWithImplicitRouter(rulesColl, "conf", "shard1", 2)
.setRule("ip_2:" + ip_2, "ip_1:" + ip_1 + "9999")
.setSnitch("class:ImplicitSnitch")
.process(cluster.getSolrClient());
}
@Test
public void testModifyColl() throws Exception {
final long minGB1 = (random().nextBoolean() ? 1 : 0);
final long minGB2 = 5;
assumeTrue("testModifyColl needs minGB1="+minGB1+" usable disk space", ImplicitSnitch.getUsableSpaceInGB(Paths.get("/")) > minGB1);
assumeTrue("testModifyColl needs minGB2="+minGB2+" usable disk space", ImplicitSnitch.getUsableSpaceInGB(Paths.get("/")) > minGB2);
assumeTrue("testModifyColl needs minGB1="+minGB1+" usable disk space",
ImplicitSnitch.getUsableSpaceInGB(Paths.get("/")) > minGB1);
assumeTrue("testModifyColl needs minGB2="+minGB2+" usable disk space",
ImplicitSnitch.getUsableSpaceInGB(Paths.get("/")) > minGB2);
String rulesColl = "modifyColl";
try (SolrClient client = createNewSolrClient("", getBaseUrl((HttpSolrClient) clients.get(0)))) {
CollectionAdminResponse rsp;
CollectionAdminRequest.Create create = new CollectionAdminRequest.Create()
.setCollectionName(rulesColl)
.setNumShards(1)
.setReplicationFactor(2)
.setRule("cores:<4", "node:*,replica:1", "freedisk:>"+minGB1)
.setSnitch("class:ImplicitSnitch");
rsp = create.process(client);
assertEquals(0, rsp.getStatus());
assertTrue(rsp.isSuccess());
ModifiableSolrParams p = new ModifiableSolrParams();
p.add("collection", rulesColl);
p.add("action", "MODIFYCOLLECTION");
p.add("rule", "cores:<5");
p.add("rule", "node:*,replica:1");
p.add("rule", "freedisk:>"+minGB2);
p.add("autoAddReplicas", "true");
client.request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p));
}
CollectionAdminRequest.createCollection(rulesColl, "conf", 1, 2)
.setRule("cores:<4", "node:*,replica:1", "freedisk:>" + minGB1)
.setSnitch("class:ImplicitSnitch")
.process(cluster.getSolrClient());
DocCollection rulesCollection = ZkStateReader.getCollectionLive(cloudClient.getZkStateReader(), rulesColl);
// TODO: Make a MODIFYCOLLECTION SolrJ class
ModifiableSolrParams p = new ModifiableSolrParams();
p.add("collection", rulesColl);
p.add("action", "MODIFYCOLLECTION");
p.add("rule", "cores:<5");
p.add("rule", "node:*,replica:1");
p.add("rule", "freedisk:>"+minGB2);
p.add("autoAddReplicas", "true");
cluster.getSolrClient().request(new GenericSolrRequest(POST, COLLECTIONS_HANDLER_PATH, p));
DocCollection rulesCollection = getCollectionState(rulesColl);
log.info("version_of_coll {} ", rulesCollection.getZNodeVersion());
List list = (List) rulesCollection.get("rule");
assertEquals(3, list.size());

View File

@ -32,6 +32,7 @@ import org.apache.solr.client.solrj.response.RequestStatusState;
import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.ImplicitDocRouter;
import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CollectionAdminParams;
import org.apache.solr.common.params.CollectionParams.CollectionAction;
@ -319,6 +320,31 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
return new Create(collection, config, numShards, numReplicas);
}
/**
* Returns a SolrRequest for creating a collection using a default configSet
*
* This requires that there is either a single configset configured in the cluster, or
* that there is a configset with the same name as the collection
*
* @param collection the collection name
* @param numShards the number of shards in the collection
* @param numReplicas the replication factor of the collection
*/
public static Create createCollection(String collection, int numShards, int numReplicas) {
return new Create(collection, numShards, numReplicas);
}
/**
* Returns a SolrRequest for creating a collection with the implicit router
* @param collection the collection name
* @param config the collection config
* @param shards a shard definition string
* @param numReplicas the replication factor of the collection
*/
public static Create createCollectionWithImplicitRouter(String collection, String config, String shards, int numReplicas) {
return new Create(collection, config, shards, numReplicas);
}
// CREATE request
public static class Create extends AsyncCollectionSpecificAdminRequest {
@ -351,6 +377,20 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
this.replicationFactor = numReplicas;
}
private Create(String collection, int numShards, int numReplicas) {
super(CollectionAction.CREATE, SolrIdentifierValidator.validateCollectionName(collection));
this.numShards = numShards;
this.replicationFactor = numReplicas;
}
private Create(String collection, String config, String shards, int numReplicas) {
super(CollectionAction.CREATE, SolrIdentifierValidator.validateCollectionName(collection));
this.configName = config;
this.replicationFactor = numReplicas;
this.shards = shards;
this.routerName = ImplicitDocRouter.NAME;
}
@Deprecated
public Create setConfigName(String config) { this.configName = config; return this; }
public Create setCreateNodeSet(String nodeSet) { this.createNodeSet = nodeSet; return this; }
@ -424,16 +464,20 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
public SolrParams getParams() {
ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
params.set("collection.configName", configName);
params.set("createNodeSet", createNodeSet);
if (configName != null)
params.set("collection.configName", configName);
if (createNodeSet != null)
params.set("createNodeSet", createNodeSet);
if (numShards != null) {
params.set( ZkStateReader.NUM_SHARDS_PROP, numShards);
}
if (maxShardsPerNode != null) {
params.set( "maxShardsPerNode", maxShardsPerNode);
}
params.set( "router.name", routerName);
params.set("shards", shards);
if (routerName != null)
params.set( "router.name", routerName);
if (shards != null)
params.set("shards", shards);
if (routerField != null) {
params.set("router.field", routerField);
}

View File

@ -17,6 +17,8 @@
package org.apache.solr.client.solrj.request;
import java.util.Date;
import org.apache.solr.common.util.NamedList;
public class CoreStatus {
@ -39,4 +41,8 @@ public class CoreStatus {
public String toString() {
return response.toString();
}
public Date getCoreStartTime() {
return (Date) response.get("startTime");
}
}

View File

@ -19,7 +19,6 @@ package org.apache.solr.common.util;
import java.lang.invoke.MethodHandles;
import java.util.Collections;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import org.apache.solr.common.SolrException;
@ -68,6 +67,16 @@ public class RetryUtil {
}
return false;
}
public static void retryUntil(String errorMessage, int retries, long pauseTime, TimeUnit pauseUnit, BooleanRetryCmd cmd)
throws InterruptedException {
while (retries-- > 0) {
if (cmd.execute())
return;
pauseUnit.sleep(pauseTime);
}
throw new SolrException(ErrorCode.SERVER_ERROR, errorMessage);
}
public static void retryOnBoolean(long timeoutms, long intervalms, BooleanRetryCmd cmd) {
long timeout = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeoutms, TimeUnit.MILLISECONDS);

View File

@ -19,6 +19,7 @@ package org.apache.solr.cloud;
import javax.servlet.Filter;
import java.io.File;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.charset.Charset;
import java.nio.file.Files;
import java.nio.file.Path;
@ -57,13 +58,18 @@ import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.ExecutorUtil;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.SolrjNamedThreadFactory;
import org.apache.solr.core.CoreContainer;
import org.apache.zookeeper.KeeperException;
import org.eclipse.jetty.servlet.ServletHolder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* "Mini" SolrCloud cluster to be used for testing
*/
public class MiniSolrCloudCluster {
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
public static final String DEFAULT_CLOUD_SOLR_XML = "<solr>\n" +
"\n" +
@ -187,6 +193,8 @@ public class MiniSolrCloudCluster {
this.baseDir = Objects.requireNonNull(baseDir);
this.jettyConfig = Objects.requireNonNull(jettyConfig);
log.info("Starting cluster of {} servers in {}", numServers, baseDir);
Files.createDirectories(baseDir);
this.externalZkServer = zkTestServer != null;
@ -513,4 +521,18 @@ public class MiniSolrCloudCluster {
}
throw new IllegalArgumentException("Cannot find Jetty for a replica with core url " + replica.getCoreUrl());
}
/**
* Make the zookeeper session on a particular jetty expire
*/
public void expireZkSession(JettySolrRunner jetty) {
CoreContainer cores = jetty.getCoreContainer();
if (cores != null) {
SolrZkClient zkClient = cores.getZkController().getZkClient();
zkClient.getSolrZooKeeper().closeCnxn();
long sessionId = zkClient.getSolrZooKeeper().getSessionId();
zkServer.expire(sessionId);
log.info("Expired zookeeper session {} from node {}", sessionId, jetty.getBaseUrl());
}
}
}

View File

@ -229,7 +229,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
return predicate.matches(n, c);
});
} catch (Exception e) {
fail(message + "\nLast available state: " + state.get());
fail(message + "\n" + e.getMessage() + "\nLast available state: " + state.get());
}
}
@ -239,6 +239,8 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
*/
public static CollectionStatePredicate clusterShape(int expectedShards, int expectedReplicas) {
return (liveNodes, collectionState) -> {
if (collectionState == null)
return false;
if (collectionState.getSlices().size() != expectedShards)
return false;
for (Slice slice : collectionState) {
@ -257,7 +259,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
/**
* Get a (reproducibly) random shard from a {@link DocCollection}
*/
protected Slice getRandomShard(DocCollection collection) {
protected static Slice getRandomShard(DocCollection collection) {
List<Slice> shards = new ArrayList<>(collection.getActiveSlices());
if (shards.size() == 0)
fail("Couldn't get random shard for collection as it has no shards!\n" + collection.toString());
@ -268,7 +270,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
/**
* Get a (reproducibly) random replica from a {@link Slice}
*/
protected Replica getRandomReplica(Slice slice) {
protected static Replica getRandomReplica(Slice slice) {
List<Replica> replicas = new ArrayList<>(slice.getReplicas());
if (replicas.size() == 0)
fail("Couldn't get random replica from shard as it has no replicas!\n" + slice.toString());
@ -279,7 +281,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
/**
* Get a (reproducibly) random replica from a {@link Slice} matching a predicate
*/
protected Replica getRandomReplica(Slice slice, Predicate<Replica> matchPredicate) {
protected static Replica getRandomReplica(Slice slice, Predicate<Replica> matchPredicate) {
List<Replica> replicas = new ArrayList<>(slice.getReplicas());
if (replicas.size() == 0)
fail("Couldn't get random replica from shard as it has no replicas!\n" + slice.toString());
@ -297,7 +299,7 @@ public class SolrCloudTestCase extends SolrTestCaseJ4 {
*
* This assumes that the replica is hosted on a live node.
*/
protected CoreStatus getCoreStatus(Replica replica) throws IOException, SolrServerException {
protected static CoreStatus getCoreStatus(Replica replica) throws IOException, SolrServerException {
JettySolrRunner jetty = cluster.getReplicaJetty(replica);
try (HttpSolrClient client = getHttpSolrClient(jetty.getBaseUrl().toString(), cluster.getSolrClient().getHttpClient())) {
return CoreAdminRequest.getCoreStatus(replica.getCoreName(), client);