mirror of https://github.com/apache/lucene.git
LUCENE-10291: Only read/write postings when there is at least one indexed field (#539)
This commit is contained in:
parent
65296e5f84
commit
8fa7412dec
|
@ -79,9 +79,11 @@ public abstract class FieldsConsumer implements Closeable {
|
||||||
final FieldsProducer f = mergeState.fieldsProducers[readerIndex];
|
final FieldsProducer f = mergeState.fieldsProducers[readerIndex];
|
||||||
|
|
||||||
final int maxDoc = mergeState.maxDocs[readerIndex];
|
final int maxDoc = mergeState.maxDocs[readerIndex];
|
||||||
f.checkIntegrity();
|
if (f != null) {
|
||||||
slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
|
f.checkIntegrity();
|
||||||
fields.add(f);
|
slices.add(new ReaderSlice(docBase, maxDoc, readerIndex));
|
||||||
|
fields.add(f);
|
||||||
|
}
|
||||||
docBase += maxDoc;
|
docBase += maxDoc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -79,7 +79,10 @@ final class PerFieldMergeState {
|
||||||
in.fieldInfos[i] = new FilterFieldInfos(orgFieldInfos[i], fields);
|
in.fieldInfos[i] = new FilterFieldInfos(orgFieldInfos[i], fields);
|
||||||
}
|
}
|
||||||
for (int i = 0; i < orgFieldsProducers.length; i++) {
|
for (int i = 0; i < orgFieldsProducers.length; i++) {
|
||||||
in.fieldsProducers[i] = new FilterFieldsProducer(orgFieldsProducers[i], fields);
|
in.fieldsProducers[i] =
|
||||||
|
orgFieldsProducers[i] == null
|
||||||
|
? null
|
||||||
|
: new FilterFieldsProducer(orgFieldsProducers[i], fields);
|
||||||
}
|
}
|
||||||
return in;
|
return in;
|
||||||
}
|
}
|
||||||
|
@ -103,6 +106,7 @@ final class PerFieldMergeState {
|
||||||
// Copy of the private fields from FieldInfos
|
// Copy of the private fields from FieldInfos
|
||||||
// Renamed so as to be less confusing about which fields we're referring to
|
// Renamed so as to be less confusing about which fields we're referring to
|
||||||
private final boolean filteredHasVectors;
|
private final boolean filteredHasVectors;
|
||||||
|
private final boolean filteredHasPostings;
|
||||||
private final boolean filteredHasProx;
|
private final boolean filteredHasProx;
|
||||||
private final boolean filteredHasPayloads;
|
private final boolean filteredHasPayloads;
|
||||||
private final boolean filteredHasOffsets;
|
private final boolean filteredHasOffsets;
|
||||||
|
@ -116,6 +120,7 @@ final class PerFieldMergeState {
|
||||||
super(toArray(src));
|
super(toArray(src));
|
||||||
|
|
||||||
boolean hasVectors = false;
|
boolean hasVectors = false;
|
||||||
|
boolean hasPostings = false;
|
||||||
boolean hasProx = false;
|
boolean hasProx = false;
|
||||||
boolean hasPayloads = false;
|
boolean hasPayloads = false;
|
||||||
boolean hasOffsets = false;
|
boolean hasOffsets = false;
|
||||||
|
@ -130,6 +135,7 @@ final class PerFieldMergeState {
|
||||||
if (this.filteredNames.contains(fi.name)) {
|
if (this.filteredNames.contains(fi.name)) {
|
||||||
this.filtered.add(fi);
|
this.filtered.add(fi);
|
||||||
hasVectors |= fi.hasVectors();
|
hasVectors |= fi.hasVectors();
|
||||||
|
hasPostings |= fi.getIndexOptions() != IndexOptions.NONE;
|
||||||
hasProx |= fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
|
hasProx |= fi.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
|
||||||
hasFreq |= fi.getIndexOptions() != IndexOptions.DOCS;
|
hasFreq |= fi.getIndexOptions() != IndexOptions.DOCS;
|
||||||
hasOffsets |=
|
hasOffsets |=
|
||||||
|
@ -143,6 +149,7 @@ final class PerFieldMergeState {
|
||||||
}
|
}
|
||||||
|
|
||||||
this.filteredHasVectors = hasVectors;
|
this.filteredHasVectors = hasVectors;
|
||||||
|
this.filteredHasPostings = hasPostings;
|
||||||
this.filteredHasProx = hasProx;
|
this.filteredHasProx = hasProx;
|
||||||
this.filteredHasPayloads = hasPayloads;
|
this.filteredHasPayloads = hasPayloads;
|
||||||
this.filteredHasOffsets = hasOffsets;
|
this.filteredHasOffsets = hasOffsets;
|
||||||
|
@ -171,6 +178,11 @@ final class PerFieldMergeState {
|
||||||
return filteredHasFreq;
|
return filteredHasFreq;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean hasPostings() {
|
||||||
|
return filteredHasPostings;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean hasProx() {
|
public boolean hasProx() {
|
||||||
return filteredHasProx;
|
return filteredHasProx;
|
||||||
|
|
|
@ -27,6 +27,7 @@ import java.util.IdentityHashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.ServiceLoader;
|
import java.util.ServiceLoader;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.TreeMap;
|
import java.util.TreeMap;
|
||||||
|
@ -185,6 +186,7 @@ public abstract class PerFieldPostingsFormat extends PostingsFormat {
|
||||||
new MergedIterator<>(
|
new MergedIterator<>(
|
||||||
true,
|
true,
|
||||||
Arrays.stream(mergeState.fieldsProducers)
|
Arrays.stream(mergeState.fieldsProducers)
|
||||||
|
.filter(Objects::nonNull)
|
||||||
.map(FieldsProducer::iterator)
|
.map(FieldsProducer::iterator)
|
||||||
.toArray(Iterator[]::new));
|
.toArray(Iterator[]::new));
|
||||||
Map<PostingsFormat, FieldsGroup> formatToGroups = buildFieldsGroupMapping(indexedFieldNames);
|
Map<PostingsFormat, FieldsGroup> formatToGroups = buildFieldsGroupMapping(indexedFieldNames);
|
||||||
|
|
|
@ -44,6 +44,7 @@ import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import org.apache.lucene.codecs.Codec;
|
import org.apache.lucene.codecs.Codec;
|
||||||
import org.apache.lucene.codecs.DocValuesProducer;
|
import org.apache.lucene.codecs.DocValuesProducer;
|
||||||
|
import org.apache.lucene.codecs.FieldsProducer;
|
||||||
import org.apache.lucene.codecs.NormsProducer;
|
import org.apache.lucene.codecs.NormsProducer;
|
||||||
import org.apache.lucene.codecs.PointsReader;
|
import org.apache.lucene.codecs.PointsReader;
|
||||||
import org.apache.lucene.codecs.PostingsFormat;
|
import org.apache.lucene.codecs.PostingsFormat;
|
||||||
|
@ -2407,7 +2408,12 @@ public final class CheckIndex implements Closeable {
|
||||||
infoStream.print(" test: terms, freq, prox...");
|
infoStream.print(" test: terms, freq, prox...");
|
||||||
}
|
}
|
||||||
|
|
||||||
final Fields fields = reader.getPostingsReader().getMergeInstance();
|
FieldsProducer fields = reader.getPostingsReader();
|
||||||
|
if (fields != null) {
|
||||||
|
fields = fields.getMergeInstance();
|
||||||
|
} else {
|
||||||
|
return new Status.TermIndexStatus();
|
||||||
|
}
|
||||||
final FieldInfos fieldInfos = reader.getFieldInfos();
|
final FieldInfos fieldInfos = reader.getFieldInfos();
|
||||||
NormsProducer normsProducer = reader.getNormsReader();
|
NormsProducer normsProducer = reader.getNormsReader();
|
||||||
if (normsProducer != null) {
|
if (normsProducer != null) {
|
||||||
|
@ -3540,10 +3546,13 @@ public final class CheckIndex implements Closeable {
|
||||||
|
|
||||||
final Bits liveDocs = reader.getLiveDocs();
|
final Bits liveDocs = reader.getLiveDocs();
|
||||||
|
|
||||||
final Fields postingsFields;
|
FieldsProducer postingsFields;
|
||||||
// TODO: testTermsIndex
|
// TODO: testTermsIndex
|
||||||
if (doSlowChecks) {
|
if (doSlowChecks) {
|
||||||
postingsFields = reader.getPostingsReader().getMergeInstance();
|
postingsFields = reader.getPostingsReader();
|
||||||
|
if (postingsFields != null) {
|
||||||
|
postingsFields = postingsFields.getMergeInstance();
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
postingsFields = null;
|
postingsFields = null;
|
||||||
}
|
}
|
||||||
|
@ -3597,6 +3606,10 @@ public final class CheckIndex implements Closeable {
|
||||||
final boolean postingsHasPayload = fieldInfo.hasPayloads();
|
final boolean postingsHasPayload = fieldInfo.hasPayloads();
|
||||||
final boolean vectorsHasPayload = terms.hasPayloads();
|
final boolean vectorsHasPayload = terms.hasPayloads();
|
||||||
|
|
||||||
|
if (postingsFields == null) {
|
||||||
|
throw new CheckIndexException(
|
||||||
|
"vector field=" + field + " does not exist in postings; doc=" + j);
|
||||||
|
}
|
||||||
Terms postingsTerms = postingsFields.terms(field);
|
Terms postingsTerms = postingsFields.terms(field);
|
||||||
if (postingsTerms == null) {
|
if (postingsTerms == null) {
|
||||||
throw new CheckIndexException(
|
throw new CheckIndexException(
|
||||||
|
|
|
@ -106,10 +106,14 @@ public abstract class CodecReader extends LeafReader {
|
||||||
@Override
|
@Override
|
||||||
public final Terms terms(String field) throws IOException {
|
public final Terms terms(String field) throws IOException {
|
||||||
// ensureOpen(); no; getPostingsReader calls this
|
// ensureOpen(); no; getPostingsReader calls this
|
||||||
|
FieldsProducer fieldsProducer = getPostingsReader();
|
||||||
|
if (fieldsProducer == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
// We could check the FieldInfo IndexOptions but there's no point since
|
// We could check the FieldInfo IndexOptions but there's no point since
|
||||||
// PostingsReader will simply return null for fields that don't exist or that have no terms
|
// PostingsReader will simply return null for fields that don't exist or that have no terms
|
||||||
// index.
|
// index.
|
||||||
return getPostingsReader().terms(field);
|
return fieldsProducer.terms(field);
|
||||||
}
|
}
|
||||||
|
|
||||||
// returns the FieldInfo that corresponds to the given field and type, or
|
// returns the FieldInfo that corresponds to the given field and type, or
|
||||||
|
@ -241,7 +245,9 @@ public abstract class CodecReader extends LeafReader {
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
|
|
||||||
// terms/postings
|
// terms/postings
|
||||||
getPostingsReader().checkIntegrity();
|
if (getPostingsReader() != null) {
|
||||||
|
getPostingsReader().checkIntegrity();
|
||||||
|
}
|
||||||
|
|
||||||
// norms
|
// norms
|
||||||
if (getNormsReader() != null) {
|
if (getNormsReader() != null) {
|
||||||
|
|
|
@ -48,6 +48,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
||||||
public static final FieldInfos EMPTY = new FieldInfos(new FieldInfo[0]);
|
public static final FieldInfos EMPTY = new FieldInfos(new FieldInfo[0]);
|
||||||
|
|
||||||
private final boolean hasFreq;
|
private final boolean hasFreq;
|
||||||
|
private final boolean hasPostings;
|
||||||
private final boolean hasProx;
|
private final boolean hasProx;
|
||||||
private final boolean hasPayloads;
|
private final boolean hasPayloads;
|
||||||
private final boolean hasOffsets;
|
private final boolean hasOffsets;
|
||||||
|
@ -67,6 +68,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
||||||
/** Constructs a new FieldInfos from an array of FieldInfo objects */
|
/** Constructs a new FieldInfos from an array of FieldInfo objects */
|
||||||
public FieldInfos(FieldInfo[] infos) {
|
public FieldInfos(FieldInfo[] infos) {
|
||||||
boolean hasVectors = false;
|
boolean hasVectors = false;
|
||||||
|
boolean hasPostings = false;
|
||||||
boolean hasProx = false;
|
boolean hasProx = false;
|
||||||
boolean hasPayloads = false;
|
boolean hasPayloads = false;
|
||||||
boolean hasOffsets = false;
|
boolean hasOffsets = false;
|
||||||
|
@ -112,6 +114,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
||||||
}
|
}
|
||||||
|
|
||||||
hasVectors |= info.hasVectors();
|
hasVectors |= info.hasVectors();
|
||||||
|
hasPostings |= info.getIndexOptions() != IndexOptions.NONE;
|
||||||
hasProx |= info.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
|
hasProx |= info.getIndexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) >= 0;
|
||||||
hasFreq |= info.getIndexOptions() != IndexOptions.DOCS;
|
hasFreq |= info.getIndexOptions() != IndexOptions.DOCS;
|
||||||
hasOffsets |=
|
hasOffsets |=
|
||||||
|
@ -132,6 +135,7 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
||||||
}
|
}
|
||||||
|
|
||||||
this.hasVectors = hasVectors;
|
this.hasVectors = hasVectors;
|
||||||
|
this.hasPostings = hasPostings;
|
||||||
this.hasProx = hasProx;
|
this.hasProx = hasProx;
|
||||||
this.hasPayloads = hasPayloads;
|
this.hasPayloads = hasPayloads;
|
||||||
this.hasOffsets = hasOffsets;
|
this.hasOffsets = hasOffsets;
|
||||||
|
@ -200,6 +204,11 @@ public class FieldInfos implements Iterable<FieldInfo> {
|
||||||
return hasFreq;
|
return hasFreq;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Returns true if any fields have postings */
|
||||||
|
public boolean hasPostings() {
|
||||||
|
return hasPostings;
|
||||||
|
}
|
||||||
|
|
||||||
/** Returns true if any fields have positions */
|
/** Returns true if any fields have positions */
|
||||||
public boolean hasProx() {
|
public boolean hasProx() {
|
||||||
return hasProx;
|
return hasProx;
|
||||||
|
|
|
@ -99,6 +99,11 @@ final class FreqProxTermsWriter extends TermsHash {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (!state.fieldInfos.hasPostings()) {
|
||||||
|
assert allFields.isEmpty();
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
// Sort by field name
|
// Sort by field name
|
||||||
CollectionUtil.introSort(allFields);
|
CollectionUtil.introSort(allFields);
|
||||||
|
|
||||||
|
|
|
@ -132,7 +132,11 @@ public class MergeState {
|
||||||
termVectorsReaders[i] = termVectorsReaders[i].getMergeInstance();
|
termVectorsReaders[i] = termVectorsReaders[i].getMergeInstance();
|
||||||
}
|
}
|
||||||
|
|
||||||
fieldsProducers[i] = reader.getPostingsReader().getMergeInstance();
|
fieldsProducers[i] = reader.getPostingsReader();
|
||||||
|
if (fieldsProducers[i] != null) {
|
||||||
|
fieldsProducers[i] = fieldsProducers[i].getMergeInstance();
|
||||||
|
}
|
||||||
|
|
||||||
pointsReaders[i] = reader.getPointsReader();
|
pointsReaders[i] = reader.getPointsReader();
|
||||||
if (pointsReaders[i] != null) {
|
if (pointsReaders[i] != null) {
|
||||||
pointsReaders[i] = pointsReaders[i].getMergeInstance();
|
pointsReaders[i] = pointsReaders[i].getMergeInstance();
|
||||||
|
|
|
@ -244,7 +244,9 @@ public class SegmentCommitInfo {
|
||||||
// updates) and then maybe even be able to remove LiveDocsFormat.files().
|
// updates) and then maybe even be able to remove LiveDocsFormat.files().
|
||||||
|
|
||||||
// Must separately add any live docs files:
|
// Must separately add any live docs files:
|
||||||
info.getCodec().liveDocsFormat().files(this, files);
|
if (hasDeletions()) {
|
||||||
|
info.getCodec().liveDocsFormat().files(this, files);
|
||||||
|
}
|
||||||
|
|
||||||
// must separately add any field updates files
|
// must separately add any field updates files
|
||||||
for (Set<String> updatefiles : dvUpdatesFiles.values()) {
|
for (Set<String> updatefiles : dvUpdatesFiles.values()) {
|
||||||
|
|
|
@ -112,10 +112,14 @@ final class SegmentCoreReaders {
|
||||||
|
|
||||||
final SegmentReadState segmentReadState =
|
final SegmentReadState segmentReadState =
|
||||||
new SegmentReadState(cfsDir, si.info, coreFieldInfos, context);
|
new SegmentReadState(cfsDir, si.info, coreFieldInfos, context);
|
||||||
final PostingsFormat format = codec.postingsFormat();
|
if (coreFieldInfos.hasPostings()) {
|
||||||
// Ask codec for its Fields
|
final PostingsFormat format = codec.postingsFormat();
|
||||||
fields = format.fieldsProducer(segmentReadState);
|
// Ask codec for its Fields
|
||||||
assert fields != null;
|
fields = format.fieldsProducer(segmentReadState);
|
||||||
|
assert fields != null;
|
||||||
|
} else {
|
||||||
|
fields = null;
|
||||||
|
}
|
||||||
// ask codec for its Norms:
|
// ask codec for its Norms:
|
||||||
// TODO: since we don't write any norms file if there are no norms,
|
// TODO: since we don't write any norms file if there are no norms,
|
||||||
// kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
|
// kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
|
||||||
|
|
|
@ -203,8 +203,10 @@ final class SegmentMerger {
|
||||||
// Use the merge instance in order to reuse the same IndexInput for all terms
|
// Use the merge instance in order to reuse the same IndexInput for all terms
|
||||||
normsMergeInstance = norms.getMergeInstance();
|
normsMergeInstance = norms.getMergeInstance();
|
||||||
}
|
}
|
||||||
try (FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState)) {
|
if (mergeState.mergeFieldInfos.hasPostings()) {
|
||||||
consumer.merge(mergeState, normsMergeInstance);
|
try (FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState)) {
|
||||||
|
consumer.merge(mergeState, normsMergeInstance);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,17 @@
|
||||||
|
# Licensed to the Apache Software Foundation (ASF) under one or more
|
||||||
|
# contributor license agreements. See the NOTICE file distributed with
|
||||||
|
# this work for additional information regarding copyright ownership.
|
||||||
|
# The ASF licenses this file to You under the Apache License, Version 2.0
|
||||||
|
# (the "License"); you may not use this file except in compliance with
|
||||||
|
# the License. You may obtain a copy of the License at
|
||||||
|
#
|
||||||
|
# http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
#
|
||||||
|
# Unless required by applicable law or agreed to in writing, software
|
||||||
|
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||||
|
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||||
|
# See the License for the specific language governing permissions and
|
||||||
|
# limitations under the License.
|
||||||
|
|
||||||
|
org.apache.lucene.codecs.TestMinimalCodec$MinimalCodec
|
||||||
|
org.apache.lucene.codecs.TestMinimalCodec$MinimalCompoundCodec
|
|
@ -0,0 +1,162 @@
|
||||||
|
/*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import org.apache.lucene.document.Document;
|
||||||
|
import org.apache.lucene.index.DirectoryReader;
|
||||||
|
import org.apache.lucene.index.IndexWriter;
|
||||||
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
|
import org.apache.lucene.tests.analysis.MockAnalyzer;
|
||||||
|
import org.apache.lucene.tests.store.BaseDirectoryWrapper;
|
||||||
|
import org.apache.lucene.tests.util.LuceneTestCase;
|
||||||
|
import org.apache.lucene.tests.util.TestUtil;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Tests to ensure that {@link Codec}s won't need to implement all formats in case where only a
|
||||||
|
* small subset of Lucene's functionality is used.
|
||||||
|
*/
|
||||||
|
public class TestMinimalCodec extends LuceneTestCase {
|
||||||
|
|
||||||
|
public void testMinimalCodec() throws IOException {
|
||||||
|
runMinimalCodecTest(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testMinimalCompoundCodec() throws IOException {
|
||||||
|
runMinimalCodecTest(true);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void runMinimalCodecTest(boolean useCompoundFile) throws IOException {
|
||||||
|
try (BaseDirectoryWrapper dir = newDirectory()) {
|
||||||
|
IndexWriterConfig writerConfig =
|
||||||
|
newIndexWriterConfig(new MockAnalyzer(random()))
|
||||||
|
.setCodec(useCompoundFile ? new MinimalCompoundCodec() : new MinimalCodec())
|
||||||
|
.setUseCompoundFile(useCompoundFile);
|
||||||
|
if (!useCompoundFile) {
|
||||||
|
writerConfig.getMergePolicy().setNoCFSRatio(0.0);
|
||||||
|
writerConfig.getMergePolicy().setMaxCFSSegmentSizeMB(Double.POSITIVE_INFINITY);
|
||||||
|
}
|
||||||
|
|
||||||
|
try (IndexWriter writer = new IndexWriter(dir, writerConfig)) {
|
||||||
|
writer.addDocument(basicDocument());
|
||||||
|
writer.flush();
|
||||||
|
// create second segment
|
||||||
|
writer.addDocument(basicDocument());
|
||||||
|
writer.forceMerge(1); // test merges
|
||||||
|
if (randomBoolean()) {
|
||||||
|
writer.commit();
|
||||||
|
}
|
||||||
|
|
||||||
|
try (DirectoryReader reader = DirectoryReader.open(writer)) {
|
||||||
|
assertEquals(2, reader.numDocs());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** returns a basic document with no indexed fields */
|
||||||
|
private static Document basicDocument() {
|
||||||
|
return new Document();
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Minimal codec implementation for working with the most basic documents */
|
||||||
|
public static class MinimalCodec extends Codec {
|
||||||
|
|
||||||
|
protected final Codec wrappedCodec = TestUtil.getDefaultCodec();
|
||||||
|
|
||||||
|
public MinimalCodec() {
|
||||||
|
this("MinimalCodec");
|
||||||
|
}
|
||||||
|
|
||||||
|
protected MinimalCodec(String name) {
|
||||||
|
super(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public FieldInfosFormat fieldInfosFormat() {
|
||||||
|
return wrappedCodec.fieldInfosFormat();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public SegmentInfoFormat segmentInfoFormat() {
|
||||||
|
return wrappedCodec.segmentInfoFormat();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CompoundFormat compoundFormat() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public LiveDocsFormat liveDocsFormat() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public StoredFieldsFormat storedFieldsFormat() {
|
||||||
|
// TODO: avoid calling this when no stored fields are written or read
|
||||||
|
return wrappedCodec.storedFieldsFormat();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PostingsFormat postingsFormat() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DocValuesFormat docValuesFormat() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public TermVectorsFormat termVectorsFormat() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public NormsFormat normsFormat() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public PointsFormat pointsFormat() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public KnnVectorsFormat knnVectorsFormat() {
|
||||||
|
throw new UnsupportedOperationException();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Minimal codec implementation for working with the most basic documents, supporting compound
|
||||||
|
* formats
|
||||||
|
*/
|
||||||
|
public static class MinimalCompoundCodec extends MinimalCodec {
|
||||||
|
public MinimalCompoundCodec() {
|
||||||
|
super("MinimalCompoundCodec");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public CompoundFormat compoundFormat() {
|
||||||
|
return wrappedCodec.compoundFormat();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue