LUCENE-9378: Make it possible to configure how to trade speed for compression on doc values. (#2069)

This adds a switch to `Lucene80DocValuesFormat` which allows to
configure whether to prioritize retrieval speed over compression ratio
or the other way around. When prioritizing retrieval speed, binary doc
values are written using the exact same format as before more aggressive
compression got introduced.
This commit is contained in:
Adrien Grand 2020-11-12 16:10:00 +01:00 committed by GitHub
parent d1297e52d9
commit 06877b2c6e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 248 additions and 37 deletions

View File

@ -216,6 +216,9 @@ New Features
* LUCENE-9553: New XYPoint query that accepts an array of XYGeometries. (Ignacio Vera)
* LUCENE-9378: Doc values now allow configuring how to trade compression for
retrieval speed. (Adrien Grand)
Improvements
---------------------

View File

@ -17,6 +17,8 @@
package org.apache.lucene.backward_codecs.lucene87;
import java.util.Objects;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.CompoundFormat;
import org.apache.lucene.codecs.DocValuesFormat;
@ -34,6 +36,7 @@ import org.apache.lucene.codecs.lucene50.Lucene50CompoundFormat;
import org.apache.lucene.codecs.lucene50.Lucene50LiveDocsFormat;
import org.apache.lucene.codecs.lucene50.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene60.Lucene60FieldInfosFormat;
import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat;
import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
@ -54,6 +57,23 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
* @lucene.experimental
*/
public class Lucene87Codec extends Codec {
/** Configuration option for the codec. */
public static enum Mode {
/** Trade compression ratio for retrieval speed. */
BEST_SPEED(Lucene87StoredFieldsFormat.Mode.BEST_SPEED, Lucene80DocValuesFormat.Mode.BEST_SPEED),
/** Trade retrieval speed for compression ratio. */
BEST_COMPRESSION(Lucene87StoredFieldsFormat.Mode.BEST_COMPRESSION, Lucene80DocValuesFormat.Mode.BEST_COMPRESSION);
private final Lucene87StoredFieldsFormat.Mode storedMode;
private final Lucene80DocValuesFormat.Mode dvMode;
private Mode(Lucene87StoredFieldsFormat.Mode storedMode, Lucene80DocValuesFormat.Mode dvMode) {
this.storedMode = Objects.requireNonNull(storedMode);
this.dvMode = Objects.requireNonNull(dvMode);
}
}
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene60FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene86SegmentInfoFormat();
@ -85,6 +105,7 @@ public class Lucene87Codec extends Codec {
super("Lucene87");
this.storedFieldsFormat = new Lucene87StoredFieldsFormat();
this.defaultFormat = new Lucene84PostingsFormat();
this.defaultDVFormat = new Lucene80DocValuesFormat();
}
@Override
@ -161,7 +182,7 @@ public class Lucene87Codec extends Codec {
return docValuesFormat;
}
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene80");
private final DocValuesFormat defaultDVFormat;
private final NormsFormat normsFormat = new Lucene80NormsFormat();

View File

@ -64,12 +64,14 @@ import static org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat.NUMERIC_
/** writer for {@link Lucene80DocValuesFormat} */
final class Lucene80DocValuesConsumer extends DocValuesConsumer implements Closeable {
final Lucene80DocValuesFormat.Mode mode;
IndexOutput data, meta;
final int maxDoc;
private final SegmentWriteState state;
/** expert: Creates a new writer */
public Lucene80DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension) throws IOException {
public Lucene80DocValuesConsumer(SegmentWriteState state, String dataCodec, String dataExtension, String metaCodec, String metaExtension, Lucene80DocValuesFormat.Mode mode) throws IOException {
this.mode = mode;
boolean success = false;
try {
this.state = state;
@ -490,13 +492,86 @@ final class Lucene80DocValuesConsumer extends DocValuesConsumer implements Close
}
}
@Override
public void addBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
meta.writeInt(field.number);
meta.writeByte(Lucene80DocValuesFormat.BINARY);
switch (mode) {
case BEST_SPEED:
meta.writeByte((byte) 0);
doAddUncompressedBinaryField(field, valuesProducer);
break;
case BEST_COMPRESSION:
meta.writeByte((byte) 1);
doAddCompressedBinaryField(field, valuesProducer);
break;
default:
throw new AssertionError();
}
}
private void doAddUncompressedBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
BinaryDocValues values = valuesProducer.getBinary(field);
long start = data.getFilePointer();
meta.writeLong(start); // dataOffset
int numDocsWithField = 0;
int minLength = Integer.MAX_VALUE;
int maxLength = 0;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
numDocsWithField++;
BytesRef v = values.binaryValue();
int length = v.length;
data.writeBytes(v.bytes, v.offset, v.length);
minLength = Math.min(length, minLength);
maxLength = Math.max(length, maxLength);
}
assert numDocsWithField <= maxDoc;
meta.writeLong(data.getFilePointer() - start); // dataLength
if (numDocsWithField == 0) {
meta.writeLong(-2); // docsWithFieldOffset
meta.writeLong(0L); // docsWithFieldLength
meta.writeShort((short) -1); // jumpTableEntryCount
meta.writeByte((byte) -1); // denseRankPower
} else if (numDocsWithField == maxDoc) {
meta.writeLong(-1); // docsWithFieldOffset
meta.writeLong(0L); // docsWithFieldLength
meta.writeShort((short) -1); // jumpTableEntryCount
meta.writeByte((byte) -1); // denseRankPower
} else {
long offset = data.getFilePointer();
meta.writeLong(offset); // docsWithFieldOffset
values = valuesProducer.getBinary(field);
final short jumpTableEntryCount = IndexedDISI.writeBitSet(values, data, IndexedDISI.DEFAULT_DENSE_RANK_POWER);
meta.writeLong(data.getFilePointer() - offset); // docsWithFieldLength
meta.writeShort(jumpTableEntryCount);
meta.writeByte(IndexedDISI.DEFAULT_DENSE_RANK_POWER);
}
meta.writeInt(numDocsWithField);
meta.writeInt(minLength);
meta.writeInt(maxLength);
if (maxLength > minLength) {
start = data.getFilePointer();
meta.writeLong(start);
meta.writeVInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
final DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, data, numDocsWithField + 1, DIRECT_MONOTONIC_BLOCK_SHIFT);
long addr = 0;
writer.add(addr);
values = valuesProducer.getBinary(field);
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
addr += values.binaryValue().length;
writer.add(addr);
}
writer.finish();
meta.writeLong(data.getFilePointer() - start);
}
}
private void doAddCompressedBinaryField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
try (CompressedBinaryBlockWriter blockWriter = new CompressedBinaryBlockWriter()){
BinaryDocValues values = valuesProducer.getBinary(field);
long start = data.getFilePointer();
@ -542,7 +617,6 @@ final class Lucene80DocValuesConsumer extends DocValuesConsumer implements Close
meta.writeInt(maxLength);
blockWriter.writeMetaData();
}
}

View File

@ -18,6 +18,7 @@ package org.apache.lucene.codecs.lucene80;
import java.io.IOException;
import java.util.Objects;
import org.apache.lucene.codecs.DocValuesConsumer;
import org.apache.lucene.codecs.DocValuesFormat;
@ -131,14 +132,30 @@ import org.apache.lucene.util.packed.DirectWriter;
*/
public final class Lucene80DocValuesFormat extends DocValuesFormat {
/** Sole Constructor */
/** Configuration option for doc values. */
public static enum Mode {
/** Trade compression ratio for retrieval speed. */
BEST_SPEED,
/** Trade retrieval speed for compression ratio. */
BEST_COMPRESSION
}
private final Mode mode;
/** Default constructor. */
public Lucene80DocValuesFormat() {
this(Mode.BEST_SPEED);
}
/** Constructor */
public Lucene80DocValuesFormat(Mode mode) {
super("Lucene80");
this.mode = Objects.requireNonNull(mode);
}
@Override
public DocValuesConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
return new Lucene80DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION);
return new Lucene80DocValuesConsumer(state, DATA_CODEC, DATA_EXTENSION, META_CODEC, META_EXTENSION, mode);
}
@Override
@ -152,7 +169,8 @@ public final class Lucene80DocValuesFormat extends DocValuesFormat {
static final String META_EXTENSION = "dvm";
static final int VERSION_START = 0;
static final int VERSION_BIN_COMPRESSED = 1;
static final int VERSION_CURRENT = VERSION_BIN_COMPRESSED;
static final int VERSION_CONFIGURABLE_COMPRESSION = 2;
static final int VERSION_CURRENT = VERSION_CONFIGURABLE_COMPRESSION;
// indicates docvalues type
static final byte NUMERIC = 0;

View File

@ -174,6 +174,20 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
private BinaryEntry readBinary(ChecksumIndexInput meta) throws IOException {
BinaryEntry entry = new BinaryEntry();
if (version >= Lucene80DocValuesFormat.VERSION_CONFIGURABLE_COMPRESSION) {
int b = meta.readByte();
switch (b) {
case 0:
case 1:
// valid
break;
default:
throw new CorruptIndexException("Unexpected byte: " + b + ", expected 0 or 1", meta);
}
entry.compressed = b != 0;
} else {
entry.compressed = version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED;
}
entry.dataOffset = meta.readLong();
entry.dataLength = meta.readLong();
entry.docsWithFieldOffset = meta.readLong();
@ -183,19 +197,19 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
entry.numDocsWithField = meta.readInt();
entry.minLength = meta.readInt();
entry.maxLength = meta.readInt();
if ((version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED && entry.numDocsWithField > 0) || entry.minLength < entry.maxLength) {
if ((entry.compressed && entry.numDocsWithField > 0) || entry.minLength < entry.maxLength) {
entry.addressesOffset = meta.readLong();
// Old count of uncompressed addresses
long numAddresses = entry.numDocsWithField + 1L;
// New count of compressed addresses - the number of compresseed blocks
if (version >= Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED) {
if (entry.compressed) {
entry.numCompressedChunks = meta.readVInt();
entry.docsPerChunkShift = meta.readVInt();
entry.maxUncompressedChunkSize = meta.readVInt();
numAddresses = entry.numCompressedChunks;
}
final int blockShift = meta.readVInt();
entry.addressesMeta = DirectMonotonicReader.loadMeta(meta, numAddresses, blockShift);
ramBytesUsed += entry.addressesMeta.ramBytesUsed();
@ -303,6 +317,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
}
private static class BinaryEntry {
boolean compressed;
long dataOffset;
long dataLength;
long docsWithFieldOffset;
@ -680,9 +695,7 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
}
}
// BWC - old binary format
private BinaryDocValues getUncompressedBinary(FieldInfo field) throws IOException {
BinaryEntry entry = binaries.get(field.name);
private BinaryDocValues getUncompressedBinary(BinaryEntry entry) throws IOException {
if (entry.docsWithFieldOffset == -2) {
return DocValues.emptyBinary();
}
@ -844,11 +857,16 @@ final class Lucene80DocValuesProducer extends DocValuesProducer implements Close
@Override
public BinaryDocValues getBinary(FieldInfo field) throws IOException {
if (version < Lucene80DocValuesFormat.VERSION_BIN_COMPRESSED) {
return getUncompressedBinary(field);
}
BinaryEntry entry = binaries.get(field.name);
if (entry.compressed) {
return getCompressedBinary(entry);
} else {
return getUncompressedBinary(entry);
}
}
private BinaryDocValues getCompressedBinary(BinaryEntry entry) throws IOException {
if (entry.docsWithFieldOffset == -2) {
return DocValues.emptyBinary();
}

View File

@ -34,6 +34,7 @@ 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.Lucene50TermVectorsFormat;
import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat;
import org.apache.lucene.codecs.lucene80.Lucene80NormsFormat;
import org.apache.lucene.codecs.lucene84.Lucene84PostingsFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsFormat;
@ -53,6 +54,23 @@ import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
* @lucene.experimental
*/
public class Lucene90Codec extends Codec {
/** Configuration option for the codec. */
public static enum Mode {
/** Trade compression ratio for retrieval speed. */
BEST_SPEED(Lucene87StoredFieldsFormat.Mode.BEST_SPEED, Lucene80DocValuesFormat.Mode.BEST_SPEED),
/** Trade retrieval speed for compression ratio. */
BEST_COMPRESSION(Lucene87StoredFieldsFormat.Mode.BEST_COMPRESSION, Lucene80DocValuesFormat.Mode.BEST_COMPRESSION);
private final Lucene87StoredFieldsFormat.Mode storedMode;
private final Lucene80DocValuesFormat.Mode dvMode;
private Mode(Lucene87StoredFieldsFormat.Mode storedMode, Lucene80DocValuesFormat.Mode dvMode) {
this.storedMode = Objects.requireNonNull(storedMode);
this.dvMode = Objects.requireNonNull(dvMode);
}
}
private final TermVectorsFormat vectorsFormat = new Lucene50TermVectorsFormat();
private final FieldInfosFormat fieldInfosFormat = new Lucene90FieldInfosFormat();
private final SegmentInfoFormat segmentInfosFormat = new Lucene86SegmentInfoFormat();
@ -82,7 +100,7 @@ public class Lucene90Codec extends Codec {
* Instantiates a new codec.
*/
public Lucene90Codec() {
this(Lucene87StoredFieldsFormat.Mode.BEST_SPEED);
this(Mode.BEST_SPEED);
}
/**
@ -91,10 +109,11 @@ public class Lucene90Codec extends Codec {
* @param mode stored fields compression mode to use for newly
* flushed/merged segments.
*/
public Lucene90Codec(Lucene87StoredFieldsFormat.Mode mode) {
public Lucene90Codec(Mode mode) {
super("Lucene90");
this.storedFieldsFormat = new Lucene87StoredFieldsFormat(Objects.requireNonNull(mode));
this.storedFieldsFormat = new Lucene87StoredFieldsFormat(Objects.requireNonNull(mode).storedMode);
this.defaultFormat = new Lucene84PostingsFormat();
this.defaultDVFormat = new Lucene80DocValuesFormat(mode.dvMode);
}
@Override
@ -172,7 +191,7 @@ public class Lucene90Codec extends Codec {
return docValuesFormat;
}
private final DocValuesFormat defaultDVFormat = DocValuesFormat.forName("Lucene80");
private final DocValuesFormat defaultDVFormat;
private final NormsFormat normsFormat = new Lucene80NormsFormat();

View File

@ -28,7 +28,6 @@ import java.util.function.LongSupplier;
import java.util.function.Supplier;
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;
@ -70,15 +69,8 @@ import org.apache.lucene.util.TestUtil;
/**
* Tests Lucene80DocValuesFormat
* Copied directly from the lucene70 package for separation of codec-code
*/
public class TestLucene80DocValuesFormat extends BaseCompressingDocValuesFormatTestCase {
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene80DocValuesFormat());
@Override
protected Codec getCodec() {
return codec;
}
public abstract class BaseLucene80DocValuesFormatTestCase extends BaseCompressingDocValuesFormatTestCase {
// 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.
@ -286,7 +278,7 @@ public class TestLucene80DocValuesFormat extends BaseCompressingDocValuesFormatT
conf.setMergeScheduler(new SerialMergeScheduler());
// set to duel against a codec which has ordinals:
final PostingsFormat pf = TestUtil.getPostingsFormatWithOrds(random());
final DocValuesFormat dv = new Lucene80DocValuesFormat();
final DocValuesFormat dv = getCodec().docValuesFormat();
conf.setCodec(new AssertingCodec() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {

View File

@ -0,0 +1,33 @@
/*
* 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.lucene80;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.util.TestUtil;
/**
* Tests Lucene80DocValuesFormat
*/
public class TestBestCompressionLucene80DocValuesFormat extends BaseLucene80DocValuesFormatTestCase {
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene80DocValuesFormat(Lucene80DocValuesFormat.Mode.BEST_COMPRESSION));
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -0,0 +1,33 @@
/*
* 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.lucene80;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.util.TestUtil;
/**
* Tests Lucene80DocValuesFormat
*/
public class TestBestSpeedLucene80DocValuesFormat extends BaseLucene80DocValuesFormatTestCase {
private final Codec codec = TestUtil.alwaysDocValuesFormat(new Lucene80DocValuesFormat(Lucene80DocValuesFormat.Mode.BEST_SPEED));
@Override
protected Codec getCodec() {
return codec;
}
}

View File

@ -18,8 +18,8 @@ package org.apache.lucene.codecs.lucene87;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene90.Lucene90Codec;
import org.apache.lucene.codecs.lucene90.Lucene90Codec.Mode;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.BaseStoredFieldsFormatTestCase;

View File

@ -40,6 +40,7 @@ import org.apache.lucene.codecs.blockterms.LuceneVarGapDocFreqInterval;
import org.apache.lucene.codecs.blockterms.LuceneVarGapFixedInterval;
import org.apache.lucene.codecs.blocktreeords.BlockTreeOrdsPostingsFormat;
import org.apache.lucene.codecs.bloom.TestBloomFilteredLucenePostings;
import org.apache.lucene.codecs.lucene80.Lucene80DocValuesFormat;
import org.apache.lucene.codecs.lucene86.Lucene86PointsReader;
import org.apache.lucene.codecs.lucene86.Lucene86PointsWriter;
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
@ -209,7 +210,7 @@ public class RandomCodec extends AssertingCodec {
addDocValues(avoidCodecs,
TestUtil.getDefaultDocValuesFormat(),
new Lucene80DocValuesFormat(Lucene80DocValuesFormat.Mode.BEST_COMPRESSION),
new AssertingDocValuesFormat());
Collections.shuffle(formats, random);

View File

@ -33,7 +33,6 @@ import org.apache.lucene.codecs.asserting.AssertingDocValuesFormat;
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.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.codecs.lucene90.Lucene90Codec;
import org.apache.lucene.codecs.mockrandom.MockRandomPostingsFormat;
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
@ -188,7 +187,7 @@ final class TestRuleSetupAndRestoreClassEnv extends AbstractBeforeAfterRule {
} else if ("Compressing".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 6 && !shouldAvoidCodec("Compressing"))) {
codec = CompressingCodec.randomInstance(random);
} else if ("Lucene90".equals(TEST_CODEC) || ("random".equals(TEST_CODEC) && randomVal == 5 && !shouldAvoidCodec("Lucene90"))) {
codec = new Lucene90Codec(RandomPicks.randomFrom(random, Lucene87StoredFieldsFormat.Mode.values())
codec = new Lucene90Codec(RandomPicks.randomFrom(random, Lucene90Codec.Mode.values())
);
} else if (!"random".equals(TEST_CODEC)) {
codec = Codec.forName(TEST_CODEC);

View File

@ -23,8 +23,8 @@ import java.util.Locale;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene90.Lucene90Codec;
import org.apache.lucene.codecs.lucene90.Lucene90Codec.Mode;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.SolrException.ErrorCode;
import org.apache.solr.common.util.NamedList;

View File

@ -21,7 +21,7 @@ import java.util.Map;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat;
import org.apache.lucene.codecs.lucene87.Lucene87StoredFieldsFormat.Mode;
import org.apache.lucene.codecs.lucene90.Lucene90Codec.Mode;
import org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat;
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
import org.apache.lucene.index.SegmentInfo;