Core: remove built-in support for Lucene's experimental codecs
Lucene's experimental codecs (from the codecs module) do not provide backwards compatibility and are free to change from release to release. When they do change, they typically cannot in general read older indices and the resulting exceptions look like index corruption. So, we are removing built-in support for them to prevent applications from choosing one and then seeing strange exceptions on upgrade. Closes #7566 Closes #7604
This commit is contained in:
parent
c7d0c3ea18
commit
130fdef367
6
pom.xml
6
pom.xml
|
@ -89,12 +89,6 @@
|
|||
<version>${lucene.version}</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.lucene</groupId>
|
||||
<artifactId>lucene-codecs</artifactId>
|
||||
<version>${lucene.version}</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.lucene</groupId>
|
||||
<artifactId>lucene-queries</artifactId>
|
||||
|
|
|
@ -1,46 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.index.codec.docvaluesformat;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
/**
|
||||
* A disk-based doc values format. This format takes no parameter.
|
||||
*/
|
||||
public class DiskDocValuesFormatProvider extends AbstractDocValuesFormatProvider {
|
||||
|
||||
private final DocValuesFormat docValuesFormat;
|
||||
|
||||
@Inject
|
||||
public DiskDocValuesFormatProvider(@Assisted String name, @Assisted Settings docValuesFormatSettings) {
|
||||
super(name);
|
||||
// TODO: log a warning if someone chooses this? just remove this together and map it to the 4.9 provider?
|
||||
this.docValuesFormat = new Lucene410DocValuesFormat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat get() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
}
|
|
@ -39,9 +39,6 @@ public class DocValuesFormats {
|
|||
}
|
||||
// LUCENE UPGRADE: update those DVF if necessary
|
||||
builtInDocValuesFormatsX.put(DocValuesFormatService.DEFAULT_FORMAT, new PreBuiltDocValuesFormatProvider.Factory(DocValuesFormatService.DEFAULT_FORMAT, DocValuesFormat.forName("Lucene410")));
|
||||
builtInDocValuesFormatsX.put("memory", new PreBuiltDocValuesFormatProvider.Factory("memory", DocValuesFormat.forName("Memory")));
|
||||
builtInDocValuesFormatsX.put("disk", new PreBuiltDocValuesFormatProvider.Factory("disk", DocValuesFormat.forName("Lucene410")));
|
||||
builtInDocValuesFormatsX.put("Disk", new PreBuiltDocValuesFormatProvider.Factory("Disk", DocValuesFormat.forName("Lucene410")));
|
||||
builtInDocValuesFormats = builtInDocValuesFormatsX.immutableMap();
|
||||
}
|
||||
|
||||
|
|
|
@ -1,45 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.index.codec.docvaluesformat;
|
||||
|
||||
import org.apache.lucene.codecs.DocValuesFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
/**
|
||||
* A memory-based doc values format. This format takes no parameter.
|
||||
*/
|
||||
public class MemoryDocValuesFormatProvider extends AbstractDocValuesFormatProvider {
|
||||
|
||||
private final DocValuesFormat docValuesFormat;
|
||||
|
||||
@Inject
|
||||
public MemoryDocValuesFormatProvider(@Assisted String name, @Assisted Settings docValuesFormatSettings) {
|
||||
super(name);
|
||||
this.docValuesFormat = new MemoryDocValuesFormat();
|
||||
}
|
||||
|
||||
@Override
|
||||
public DocValuesFormat get() {
|
||||
return docValuesFormat;
|
||||
}
|
||||
}
|
|
@ -1,106 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.index.codec.postingsformat;
|
||||
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.bloom.BloomFilterFactory;
|
||||
import org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat;
|
||||
import org.apache.lucene.codecs.bloom.FuzzySet;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.SegmentWriteState;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class BloomFilterLucenePostingsFormatProvider extends AbstractPostingsFormatProvider {
|
||||
|
||||
public static final class Defaults {
|
||||
public static final float MAX_SATURATION = 0.1f;
|
||||
public static final float SATURATION_LIMIT = 0.9f;
|
||||
}
|
||||
|
||||
private final float desiredMaxSaturation;
|
||||
private final float saturationLimit;
|
||||
private final PostingsFormatProvider delegate;
|
||||
private final BloomFilteringPostingsFormat postingsFormat;
|
||||
|
||||
@Inject
|
||||
public BloomFilterLucenePostingsFormatProvider(@IndexSettings Settings indexSettings, @Nullable Map<String, Factory> postingFormatFactories, @Assisted String name, @Assisted Settings postingsFormatSettings) {
|
||||
super(name);
|
||||
this.desiredMaxSaturation = postingsFormatSettings.getAsFloat("desired_max_saturation", Defaults.MAX_SATURATION);
|
||||
this.saturationLimit = postingsFormatSettings.getAsFloat("saturation_limit", Defaults.SATURATION_LIMIT);
|
||||
this.delegate = Helper.lookup(indexSettings, postingsFormatSettings.get("delegate"), postingFormatFactories);
|
||||
this.postingsFormat = new BloomFilteringPostingsFormat(
|
||||
delegate.get(),
|
||||
new CustomBloomFilterFactory(desiredMaxSaturation, saturationLimit)
|
||||
);
|
||||
}
|
||||
|
||||
public float desiredMaxSaturation() {
|
||||
return desiredMaxSaturation;
|
||||
}
|
||||
|
||||
public float saturationLimit() {
|
||||
return saturationLimit;
|
||||
}
|
||||
|
||||
public PostingsFormatProvider delegate() {
|
||||
return delegate;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsFormat get() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
public static class CustomBloomFilterFactory extends BloomFilterFactory {
|
||||
|
||||
private final float desiredMaxSaturation;
|
||||
private final float saturationLimit;
|
||||
|
||||
public CustomBloomFilterFactory() {
|
||||
this(Defaults.MAX_SATURATION, Defaults.SATURATION_LIMIT);
|
||||
}
|
||||
|
||||
CustomBloomFilterFactory(float desiredMaxSaturation, float saturationLimit) {
|
||||
this.desiredMaxSaturation = desiredMaxSaturation;
|
||||
this.saturationLimit = saturationLimit;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FuzzySet getSetForField(SegmentWriteState state, FieldInfo info) {
|
||||
//Assume all of the docs have a unique term (e.g. a primary key) and we hope to maintain a set with desiredMaxSaturation% of bits set
|
||||
return FuzzySet.createSetBasedOnQuality(state.segmentInfo.getDocCount(), desiredMaxSaturation);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSaturated(FuzzySet bloomFilter, FieldInfo fieldInfo) {
|
||||
// Don't bother saving bitsets if > saturationLimit % of bits are set - we don't want to
|
||||
// throw any more memory at this problem.
|
||||
return bloomFilter.getSaturation() > saturationLimit;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,72 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.index.codec.postingsformat;
|
||||
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
/**
|
||||
* A {@link PostingsFormatProvider} for {@link DirectPostingsFormat}. This
|
||||
* postings format uses an on-disk storage for its terms and posting lists and
|
||||
* streams its data during segment merges but loads its entire postings, terms
|
||||
* and positions into memory for faster search performance. This format has a
|
||||
* significant memory footprint and should be used with care. <b> This postings
|
||||
* format offers the following parameters:
|
||||
* <ul>
|
||||
* <li><tt>min_skip_count</tt>: the minimum number terms with a shared prefix to
|
||||
* allow a skip pointer to be written. the default is <tt>8</tt></li>
|
||||
*
|
||||
* <li><tt>low_freq_cutoff</tt>: terms with a lower document frequency use a
|
||||
* single array object representation for postings and positions.</li>
|
||||
* </ul>
|
||||
*
|
||||
* @see DirectPostingsFormat
|
||||
*
|
||||
*/
|
||||
public class DirectPostingsFormatProvider extends AbstractPostingsFormatProvider {
|
||||
|
||||
private final int minSkipCount;
|
||||
private final int lowFreqCutoff;
|
||||
private final DirectPostingsFormat postingsFormat;
|
||||
|
||||
@Inject
|
||||
public DirectPostingsFormatProvider(@Assisted String name, @Assisted Settings postingsFormatSettings) {
|
||||
super(name);
|
||||
this.minSkipCount = postingsFormatSettings.getAsInt("min_skip_count", 8); // See DirectPostingsFormat#DEFAULT_MIN_SKIP_COUNT
|
||||
this.lowFreqCutoff = postingsFormatSettings.getAsInt("low_freq_cutoff", 32); // See DirectPostingsFormat#DEFAULT_LOW_FREQ_CUTOFF
|
||||
this.postingsFormat = new DirectPostingsFormat(minSkipCount, lowFreqCutoff);
|
||||
}
|
||||
|
||||
public int minSkipCount() {
|
||||
return minSkipCount;
|
||||
}
|
||||
|
||||
public int lowFreqCutoff() {
|
||||
return lowFreqCutoff;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsFormat get() {
|
||||
return postingsFormat;
|
||||
}
|
||||
}
|
|
@ -1,68 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.index.codec.postingsformat;
|
||||
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.util.packed.PackedInts;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
/**
|
||||
* A {@link PostingsFormatProvider} for Lucenes {@link MemoryPostingsFormat}.
|
||||
* This postings format offers the following parameters:
|
||||
* <ul>
|
||||
* <li><tt>pack_fst</tt>: <code>true</code> iff the in memory structure should
|
||||
* be packed once its build. Packed will reduce the size for the data-structure
|
||||
* in memory but requires more memory during building. Default is <code>false</code></li>
|
||||
*
|
||||
* <li><tt>acceptable_overhead_ratio</tt>: the compression overhead used to
|
||||
* compress internal structures. See {@link PackedInts} for details. Default is {@value PackedInts#DEFAULT}</li>
|
||||
* </ul>
|
||||
*/
|
||||
public class MemoryPostingsFormatProvider extends AbstractPostingsFormatProvider {
|
||||
|
||||
private final boolean packFst;
|
||||
private final float acceptableOverheadRatio;
|
||||
private final MemoryPostingsFormat postingsFormat;
|
||||
|
||||
@Inject
|
||||
public MemoryPostingsFormatProvider(@Assisted String name, @Assisted Settings postingsFormatSettings) {
|
||||
super(name);
|
||||
this.packFst = postingsFormatSettings.getAsBoolean("pack_fst", false);
|
||||
this.acceptableOverheadRatio = postingsFormatSettings.getAsFloat("acceptable_overhead_ratio", PackedInts.DEFAULT);
|
||||
// TODO this should really be an ENUM?
|
||||
this.postingsFormat = new MemoryPostingsFormat(packFst, acceptableOverheadRatio);
|
||||
}
|
||||
|
||||
public boolean packFst() {
|
||||
return packFst;
|
||||
}
|
||||
|
||||
public float acceptableOverheadRatio() {
|
||||
return acceptableOverheadRatio;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsFormat get() {
|
||||
return postingsFormat;
|
||||
}
|
||||
}
|
|
@ -22,39 +22,20 @@ package org.elasticsearch.index.codec.postingsformat;
|
|||
import com.google.common.collect.ImmutableCollection;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.util.BloomFilter;
|
||||
|
||||
/**
|
||||
* This class represents the set of Elasticsearch "build-in"
|
||||
* This class represents the set of Elasticsearch "built-in"
|
||||
* {@link PostingsFormatProvider.Factory postings format factories}
|
||||
* <ul>
|
||||
* <li><b>direct</b>: a postings format that uses disk-based storage but loads
|
||||
* its terms and postings directly into memory. Note this postings format is
|
||||
* very memory intensive and has certain limitation that don't allow segments to
|
||||
* grow beyond 2.1GB see {@link DirectPostingsFormat} for details.</li>
|
||||
* <p/>
|
||||
* <li><b>memory</b>: a postings format that stores its entire terms, postings,
|
||||
* positions and payloads in a finite state transducer. This format should only
|
||||
* be used for primary keys or with fields where each term is contained in a
|
||||
* very low number of documents.</li>
|
||||
* <p/>
|
||||
* <li><b>pulsing</b>: a postings format in-lines the posting lists for very low
|
||||
* frequent terms in the term dictionary. This is useful to improve lookup
|
||||
* performance for low-frequent terms.</li>
|
||||
* <p/>
|
||||
* <li><b>bloom_default</b>: a postings format that uses a bloom filter to
|
||||
* improve term lookup performance. This is useful for primarily keys or fields
|
||||
* that are used as a delete key</li>
|
||||
* <p/>
|
||||
* <li><b>bloom_pulsing</b>: a postings format that combines the advantages of
|
||||
* <b>bloom</b> and <b>pulsing</b> to further improve lookup performance</li>
|
||||
* <p/>
|
||||
* <li><b>default</b>: the default Elasticsearch postings format offering best
|
||||
* general purpose performance. This format is used if no postings format is
|
||||
* specified in the field mapping.</li>
|
||||
* <li><b>***</b>: other formats from Lucene core (e.g. Lucene41 as of Lucene 4.10)
|
||||
* </ul>
|
||||
*/
|
||||
public class PostingFormats {
|
||||
|
@ -62,30 +43,25 @@ public class PostingFormats {
|
|||
private static final ImmutableMap<String, PreBuiltPostingsFormatProvider.Factory> builtInPostingFormats;
|
||||
|
||||
static {
|
||||
MapBuilder<String, PreBuiltPostingsFormatProvider.Factory> buildInPostingFormatsX = MapBuilder.newMapBuilder();
|
||||
// add defaults ones
|
||||
MapBuilder<String, PreBuiltPostingsFormatProvider.Factory> builtInPostingFormatsX = MapBuilder.newMapBuilder();
|
||||
// Add any PostingsFormat visible in the CLASSPATH (from Lucene core or via user's plugins). Note that we no longer include
|
||||
// lucene codecs module since those codecs have no backwards compatibility between releases and can easily cause exceptions that
|
||||
// look like index corruption on upgrade:
|
||||
for (String luceneName : PostingsFormat.availablePostingsFormats()) {
|
||||
buildInPostingFormatsX.put(luceneName, new PreBuiltPostingsFormatProvider.Factory(PostingsFormat.forName(luceneName)));
|
||||
builtInPostingFormatsX.put(luceneName, new PreBuiltPostingsFormatProvider.Factory(PostingsFormat.forName(luceneName)));
|
||||
}
|
||||
final PostingsFormat defaultFormat = new Elasticsearch090PostingsFormat();
|
||||
buildInPostingFormatsX.put("direct", new PreBuiltPostingsFormatProvider.Factory("direct", PostingsFormat.forName("Direct")));
|
||||
buildInPostingFormatsX.put("memory", new PreBuiltPostingsFormatProvider.Factory("memory", PostingsFormat.forName("Memory")));
|
||||
// LUCENE UPGRADE: Need to change this to the relevant ones on a lucene upgrade
|
||||
buildInPostingFormatsX.put("pulsing", new PreBuiltPostingsFormatProvider.Factory("pulsing", PostingsFormat.forName("Pulsing41")));
|
||||
buildInPostingFormatsX.put(PostingsFormatService.DEFAULT_FORMAT, new PreBuiltPostingsFormatProvider.Factory(PostingsFormatService.DEFAULT_FORMAT, defaultFormat));
|
||||
builtInPostingFormatsX.put(PostingsFormatService.DEFAULT_FORMAT,
|
||||
new PreBuiltPostingsFormatProvider.Factory(PostingsFormatService.DEFAULT_FORMAT, defaultFormat));
|
||||
|
||||
buildInPostingFormatsX.put("bloom_pulsing", new PreBuiltPostingsFormatProvider.Factory("bloom_pulsing", wrapInBloom(PostingsFormat.forName("Pulsing41"))));
|
||||
buildInPostingFormatsX.put("bloom_default", new PreBuiltPostingsFormatProvider.Factory("bloom_default", wrapInBloom(PostingsFormat.forName("Lucene41"))));
|
||||
builtInPostingFormatsX.put("bloom_default", new PreBuiltPostingsFormatProvider.Factory("bloom_default", wrapInBloom(PostingsFormat.forName("Lucene41"))));
|
||||
|
||||
builtInPostingFormats = buildInPostingFormatsX.immutableMap();
|
||||
builtInPostingFormats = builtInPostingFormatsX.immutableMap();
|
||||
}
|
||||
|
||||
public static final boolean luceneBloomFilter = false;
|
||||
|
||||
static PostingsFormat wrapInBloom(PostingsFormat delegate) {
|
||||
if (luceneBloomFilter) {
|
||||
return new BloomFilteringPostingsFormat(delegate, new BloomFilterLucenePostingsFormatProvider.CustomBloomFilterFactory());
|
||||
}
|
||||
return new BloomFilterPostingsFormat(delegate, BloomFilter.Factory.DEFAULT);
|
||||
}
|
||||
|
||||
|
|
|
@ -101,9 +101,7 @@ public interface PostingsFormatProvider {
|
|||
|
||||
/**
|
||||
* A simple factory used to create {@link PostingsFormatProvider} used by
|
||||
* delegating providers like {@link BloomFilterLucenePostingsFormatProvider} or
|
||||
* {@link PulsingPostingsFormatProvider}. Those providers wrap other
|
||||
* postings formats to enrich their capabilities.
|
||||
* delegating providers.
|
||||
*/
|
||||
public interface Factory {
|
||||
PostingsFormatProvider create(String name, Settings settings);
|
||||
|
|
|
@ -1,81 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch 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.elasticsearch.index.codec.postingsformat;
|
||||
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.blocktree.BlockTreeTermsWriter;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.inject.assistedinject.Assisted;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
/**
|
||||
* A {@link PostingsFormatProvider} for Lucenes {@link Pulsing41PostingsFormat}.
|
||||
* The pulsing implementation in-lines the posting lists for very low frequent
|
||||
* terms in the term dictionary. This is useful to improve lookup performance
|
||||
* for low-frequent terms. This postings format offers the following parameters:
|
||||
* <ul>
|
||||
* <li><tt>min_block_size</tt>: the minimum block size the default Lucene term
|
||||
* dictionary uses to encode on-disk blocks.</li>
|
||||
*
|
||||
* <li><tt>max_block_size</tt>: the maximum block size the default Lucene term
|
||||
* dictionary uses to encode on-disk blocks.</li>
|
||||
*
|
||||
* <li><tt>freq_cut_off</tt>: the document frequency cut off where pulsing
|
||||
* in-lines posting lists into the term dictionary. Terms with a document
|
||||
* frequency less or equal to the cutoff will be in-lined. The default is
|
||||
* <tt>1</tt></li>
|
||||
* </ul>
|
||||
*/
|
||||
// LUCENE UPGRADE: Check if type of field postingsFormat needs to be updated!
|
||||
public class PulsingPostingsFormatProvider extends AbstractPostingsFormatProvider {
|
||||
|
||||
private final int freqCutOff;
|
||||
private final int minBlockSize;
|
||||
private final int maxBlockSize;
|
||||
private final Pulsing41PostingsFormat postingsFormat;
|
||||
|
||||
@Inject
|
||||
public PulsingPostingsFormatProvider(@Assisted String name, @Assisted Settings postingsFormatSettings) {
|
||||
super(name);
|
||||
this.freqCutOff = postingsFormatSettings.getAsInt("freq_cut_off", 1);
|
||||
this.minBlockSize = postingsFormatSettings.getAsInt("min_block_size", BlockTreeTermsWriter.DEFAULT_MIN_BLOCK_SIZE);
|
||||
this.maxBlockSize = postingsFormatSettings.getAsInt("max_block_size", BlockTreeTermsWriter.DEFAULT_MAX_BLOCK_SIZE);
|
||||
this.postingsFormat = new Pulsing41PostingsFormat(freqCutOff, minBlockSize, maxBlockSize);
|
||||
}
|
||||
|
||||
public int freqCutOff() {
|
||||
return freqCutOff;
|
||||
}
|
||||
|
||||
public int minBlockSize() {
|
||||
return minBlockSize;
|
||||
}
|
||||
|
||||
public int maxBlockSize() {
|
||||
return maxBlockSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PostingsFormat get() {
|
||||
return postingsFormat;
|
||||
}
|
||||
|
||||
}
|
|
@ -62,32 +62,7 @@ public class CodecTests extends ElasticsearchIntegrationTest {
|
|||
.field("postings_format", "test1").field("index_options", "docs").field("type", "string").endObject().endObject().endObject().endObject())
|
||||
.setSettings(ImmutableSettings.settingsBuilder()
|
||||
.put(indexSettings())
|
||||
.put("index.codec.postings_format.test1.type", "pulsing")));
|
||||
|
||||
client().prepareIndex("test", "type1", "1").setSource("field1", "quick brown fox", "field2", "quick brown fox").execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "2").setSource("field1", "quick lazy huge brown fox", "field2", "quick lazy huge brown fox").setRefresh(true).execute().actionGet();
|
||||
|
||||
SearchResponse searchResponse = client().prepareSearch().setQuery(QueryBuilders.matchQuery("field2", "quick brown").type(MatchQueryBuilder.Type.PHRASE).slop(0)).execute().actionGet();
|
||||
assertThat(searchResponse.getHits().totalHits(), equalTo(1l));
|
||||
try {
|
||||
client().prepareSearch().setQuery(QueryBuilders.matchQuery("field1", "quick brown").type(MatchQueryBuilder.Type.PHRASE).slop(0)).execute().actionGet();
|
||||
} catch (SearchPhaseExecutionException e) {
|
||||
assertThat(e.getMessage(), endsWith("IllegalStateException[field \"field1\" was indexed without position data; cannot run PhraseQuery (term=quick)]; }"));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIndexingWithSimpleTextCodec() throws Exception {
|
||||
try {
|
||||
client().admin().indices().prepareDelete("test").execute().actionGet();
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
|
||||
assertAcked(prepareCreate("test")
|
||||
.setSettings(ImmutableSettings.settingsBuilder()
|
||||
.put(indexSettings())
|
||||
.put("index.codec", "SimpleText")));
|
||||
.put("index.codec.postings_format.test1.type", "default")));
|
||||
|
||||
client().prepareIndex("test", "type1", "1").setSource("field1", "quick brown fox", "field2", "quick brown fox").execute().actionGet();
|
||||
client().prepareIndex("test", "type1", "2").setSource("field1", "quick lazy huge brown fox", "field2", "quick lazy huge brown fox").setRefresh(true).execute().actionGet();
|
||||
|
@ -111,12 +86,12 @@ public class CodecTests extends ElasticsearchIntegrationTest {
|
|||
|
||||
assertAcked(prepareCreate("test")
|
||||
.addMapping("test", jsonBuilder().startObject().startObject("test")
|
||||
.startObject("_version").field("doc_values_format", "disk").endObject()
|
||||
.startObject("_version").field("doc_values_format", "Lucene410").endObject()
|
||||
.startObject("properties").startObject("field").field("type", "long").field("doc_values_format", "dvf").endObject().endObject()
|
||||
.endObject().endObject())
|
||||
.setSettings(ImmutableSettings.settingsBuilder()
|
||||
.put(indexSettings())
|
||||
.put("index.codec.doc_values_format.dvf.type", "disk")));
|
||||
.put("index.codec.doc_values_format.dvf.type", "default")));
|
||||
|
||||
for (int i = 10; i >= 0; --i) {
|
||||
client().prepareIndex("test", "test", Integer.toString(i)).setSource("field", randomLong()).setRefresh(i == 0 || rarely()).execute().actionGet();
|
||||
|
|
|
@ -19,29 +19,26 @@
|
|||
|
||||
package org.elasticsearch.index.codec;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41Codec;
|
||||
import org.apache.lucene.codecs.lucene41.Lucene41PostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene42.Lucene42Codec;
|
||||
import org.apache.lucene.codecs.lucene45.Lucene45Codec;
|
||||
import org.apache.lucene.codecs.lucene46.Lucene46Codec;
|
||||
import org.apache.lucene.codecs.lucene49.Lucene49Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410Codec;
|
||||
import org.apache.lucene.codecs.lucene410.Lucene410DocValuesFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryDocValuesFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.codecs.perfield.PerFieldPostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing41PostingsFormat;
|
||||
import org.apache.lucene.codecs.simpletext.SimpleTextCodec;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
import org.elasticsearch.index.codec.docvaluesformat.*;
|
||||
import org.elasticsearch.index.codec.postingsformat.*;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.internal.IdFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.VersionFieldMapper;
|
||||
import org.elasticsearch.index.service.IndexService;
|
||||
|
@ -49,9 +46,6 @@ import org.elasticsearch.test.ElasticsearchSingleNodeLuceneTestCase;
|
|||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
|
||||
|
@ -76,7 +70,6 @@ public class CodecTests extends ElasticsearchSingleNodeLuceneTestCase {
|
|||
assertThat(codecService.codec("Lucene40"), instanceOf(Lucene40Codec.class));
|
||||
assertThat(codecService.codec("Lucene41"), instanceOf(Lucene41Codec.class));
|
||||
assertThat(codecService.codec("Lucene42"), instanceOf(Lucene42Codec.class));
|
||||
assertThat(codecService.codec("SimpleText"), instanceOf(SimpleTextCodec.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -102,39 +95,15 @@ public class CodecTests extends ElasticsearchSingleNodeLuceneTestCase {
|
|||
|
||||
assertThat(postingsFormatService.get("XBloomFilter"), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(postingsFormatService.get("XBloomFilter").get(), instanceOf(BloomFilterPostingsFormat.class));
|
||||
|
||||
if (PostingFormats.luceneBloomFilter) {
|
||||
assertThat(postingsFormatService.get("bloom_pulsing").get(), instanceOf(BloomFilteringPostingsFormat.class));
|
||||
} else {
|
||||
assertThat(postingsFormatService.get("bloom_pulsing").get(), instanceOf(BloomFilterPostingsFormat.class));
|
||||
}
|
||||
|
||||
assertThat(postingsFormatService.get("pulsing"), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(postingsFormatService.get("pulsing").get(), instanceOf(Pulsing41PostingsFormat.class));
|
||||
assertThat(postingsFormatService.get("Pulsing41"), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(postingsFormatService.get("Pulsing41").get(), instanceOf(Pulsing41PostingsFormat.class));
|
||||
|
||||
assertThat(postingsFormatService.get("memory"), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(postingsFormatService.get("memory").get(), instanceOf(MemoryPostingsFormat.class));
|
||||
assertThat(postingsFormatService.get("Memory"), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(postingsFormatService.get("Memory").get(), instanceOf(MemoryPostingsFormat.class));
|
||||
|
||||
assertThat(postingsFormatService.get("direct"), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(postingsFormatService.get("direct").get(), instanceOf(DirectPostingsFormat.class));
|
||||
assertThat(postingsFormatService.get("Direct"), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(postingsFormatService.get("Direct").get(), instanceOf(DirectPostingsFormat.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResolveDefaultDocValuesFormats() throws Exception {
|
||||
DocValuesFormatService docValuesFormatService = createCodecService().docValuesFormatService();
|
||||
|
||||
for (String dvf : Arrays.asList("memory", "disk", "Disk", "default")) {
|
||||
for (String dvf : Arrays.asList("default")) {
|
||||
assertThat(docValuesFormatService.get(dvf), instanceOf(PreBuiltDocValuesFormatProvider.class));
|
||||
}
|
||||
assertThat(docValuesFormatService.get("memory").get(), instanceOf(MemoryDocValuesFormat.class));
|
||||
assertThat(docValuesFormatService.get("disk").get(), instanceOf(Lucene410DocValuesFormat.class));
|
||||
assertThat(docValuesFormatService.get("Disk").get(), instanceOf(Lucene410DocValuesFormat.class));
|
||||
assertThat(docValuesFormatService.get("default").get(), instanceOf(Lucene410DocValuesFormat.class));
|
||||
}
|
||||
|
||||
|
@ -163,164 +132,16 @@ public class CodecTests extends ElasticsearchSingleNodeLuceneTestCase {
|
|||
assertThat(provider.maxBlockSize(), equalTo(64));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResolvePostingFormatsFromMapping_memory() throws Exception {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties")
|
||||
.startObject("field1").field("type", "string").field("postings_format", "memory").endObject()
|
||||
.startObject("field2").field("type", "string").field("postings_format", "my_format1").endObject()
|
||||
.endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
Settings indexSettings = ImmutableSettings.settingsBuilder()
|
||||
.put("index.codec.postings_format.my_format1.type", "memory")
|
||||
.put("index.codec.postings_format.my_format1.pack_fst", true)
|
||||
.put("index.codec.postings_format.my_format1.acceptable_overhead_ratio", 0.3f)
|
||||
.build();
|
||||
CodecService codecService = createCodecService(indexSettings);
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider(), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider().get(), instanceOf(MemoryPostingsFormat.class));
|
||||
|
||||
assertThat(documentMapper.mappers().name("field2").mapper().postingsFormatProvider(), instanceOf(MemoryPostingsFormatProvider.class));
|
||||
MemoryPostingsFormatProvider provider = (MemoryPostingsFormatProvider) documentMapper.mappers().name("field2").mapper().postingsFormatProvider();
|
||||
assertThat(provider.packFst(), equalTo(true));
|
||||
assertThat(provider.acceptableOverheadRatio(), equalTo(0.3f));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResolvePostingFormatsFromMapping_direct() throws Exception {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties")
|
||||
.startObject("field1").field("type", "string").field("postings_format", "direct").endObject()
|
||||
.startObject("field2").field("type", "string").field("postings_format", "my_format1").endObject()
|
||||
.endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
Settings indexSettings = ImmutableSettings.settingsBuilder()
|
||||
.put("index.codec.postings_format.my_format1.type", "direct")
|
||||
.put("index.codec.postings_format.my_format1.min_skip_count", 16)
|
||||
.put("index.codec.postings_format.my_format1.low_freq_cutoff", 64)
|
||||
.build();
|
||||
CodecService codecService = createCodecService(indexSettings);
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider(), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider().get(), instanceOf(DirectPostingsFormat.class));
|
||||
|
||||
assertThat(documentMapper.mappers().name("field2").mapper().postingsFormatProvider(), instanceOf(DirectPostingsFormatProvider.class));
|
||||
DirectPostingsFormatProvider provider = (DirectPostingsFormatProvider) documentMapper.mappers().name("field2").mapper().postingsFormatProvider();
|
||||
assertThat(provider.minSkipCount(), equalTo(16));
|
||||
assertThat(provider.lowFreqCutoff(), equalTo(64));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResolvePostingFormatsFromMapping_pulsing() throws Exception {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties")
|
||||
.startObject("field1").field("type", "string").field("postings_format", "pulsing").endObject()
|
||||
.startObject("field2").field("type", "string").field("postings_format", "my_format1").endObject()
|
||||
.endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
Settings indexSettings = ImmutableSettings.settingsBuilder()
|
||||
.put("index.codec.postings_format.my_format1.type", "pulsing")
|
||||
.put("index.codec.postings_format.my_format1.freq_cut_off", 2)
|
||||
.put("index.codec.postings_format.my_format1.min_block_size", 32)
|
||||
.put("index.codec.postings_format.my_format1.max_block_size", 64)
|
||||
.build();
|
||||
CodecService codecService = createCodecService(indexSettings);
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider(), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider().get(), instanceOf(Pulsing41PostingsFormat.class));
|
||||
|
||||
assertThat(documentMapper.mappers().name("field2").mapper().postingsFormatProvider(), instanceOf(PulsingPostingsFormatProvider.class));
|
||||
PulsingPostingsFormatProvider provider = (PulsingPostingsFormatProvider) documentMapper.mappers().name("field2").mapper().postingsFormatProvider();
|
||||
assertThat(provider.freqCutOff(), equalTo(2));
|
||||
assertThat(provider.minBlockSize(), equalTo(32));
|
||||
assertThat(provider.maxBlockSize(), equalTo(64));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResolvePostingFormatsFromMappingLuceneBloom() throws Exception {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties")
|
||||
.startObject("field1").field("type", "string").field("postings_format", "bloom_default").endObject()
|
||||
.startObject("field2").field("type", "string").field("postings_format", "bloom_pulsing").endObject()
|
||||
.startObject("field3").field("type", "string").field("postings_format", "my_format1").endObject()
|
||||
.endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
Settings indexSettings = ImmutableSettings.settingsBuilder()
|
||||
.put("index.codec.postings_format.my_format1.type", "bloom_filter_lucene")
|
||||
.put("index.codec.postings_format.my_format1.desired_max_saturation", 0.2f)
|
||||
.put("index.codec.postings_format.my_format1.saturation_limit", 0.8f)
|
||||
.put("index.codec.postings_format.my_format1.delegate", "delegate1")
|
||||
.put("index.codec.postings_format.delegate1.type", "direct")
|
||||
.put("index.codec.postings_format.delegate1.min_skip_count", 16)
|
||||
.put("index.codec.postings_format.delegate1.low_freq_cutoff", 64)
|
||||
.build();
|
||||
CodecService codecService = createCodecService(indexSettings);
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider(), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
if (PostingFormats.luceneBloomFilter) {
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider().get(), instanceOf(BloomFilteringPostingsFormat.class));
|
||||
} else {
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().postingsFormatProvider().get(), instanceOf(BloomFilterPostingsFormat.class));
|
||||
}
|
||||
|
||||
assertThat(documentMapper.mappers().name("field2").mapper().postingsFormatProvider(), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
if (PostingFormats.luceneBloomFilter) {
|
||||
assertThat(documentMapper.mappers().name("field2").mapper().postingsFormatProvider().get(), instanceOf(BloomFilteringPostingsFormat.class));
|
||||
} else {
|
||||
assertThat(documentMapper.mappers().name("field2").mapper().postingsFormatProvider().get(), instanceOf(BloomFilterPostingsFormat.class));
|
||||
}
|
||||
|
||||
assertThat(documentMapper.mappers().name("field3").mapper().postingsFormatProvider(), instanceOf(BloomFilterLucenePostingsFormatProvider.class));
|
||||
BloomFilterLucenePostingsFormatProvider provider = (BloomFilterLucenePostingsFormatProvider) documentMapper.mappers().name("field3").mapper().postingsFormatProvider();
|
||||
assertThat(provider.desiredMaxSaturation(), equalTo(0.2f));
|
||||
assertThat(provider.saturationLimit(), equalTo(0.8f));
|
||||
assertThat(provider.delegate(), instanceOf(DirectPostingsFormatProvider.class));
|
||||
DirectPostingsFormatProvider delegate = (DirectPostingsFormatProvider) provider.delegate();
|
||||
assertThat(delegate.minSkipCount(), equalTo(16));
|
||||
assertThat(delegate.lowFreqCutoff(), equalTo(64));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangeUidPostingsFormat() throws Exception {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("_uid").field("postings_format", "memory").endObject()
|
||||
.startObject("_uid").field("postings_format", "Lucene41").endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
CodecService codecService = createCodecService();
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.rootMapper(UidFieldMapper.class).postingsFormatProvider(), instanceOf(PreBuiltPostingsFormatProvider.class));
|
||||
assertThat(documentMapper.rootMapper(UidFieldMapper.class).postingsFormatProvider().get(), instanceOf(MemoryPostingsFormat.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangeUidDocValuesFormat() throws IOException {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("_uid").startObject("fielddata").field("format", "doc_values").endObject().field("doc_values_format", "disk").endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
CodecService codecService = createCodecService();
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.rootMapper(UidFieldMapper.class).hasDocValues(), equalTo(true));
|
||||
assertThat(documentMapper.rootMapper(UidFieldMapper.class).docValuesFormatProvider(), instanceOf(PreBuiltDocValuesFormatProvider.class));
|
||||
assertThat(documentMapper.rootMapper(UidFieldMapper.class).docValuesFormatProvider().get(), instanceOf(Lucene410DocValuesFormat.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangeIdDocValuesFormat() throws IOException {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("_id").startObject("fielddata").field("format", "doc_values").endObject().field("doc_values_format", "disk").endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
CodecService codecService = createCodecService();
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.rootMapper(IdFieldMapper.class).hasDocValues(), equalTo(true));
|
||||
assertThat(documentMapper.rootMapper(IdFieldMapper.class).docValuesFormatProvider(), instanceOf(PreBuiltDocValuesFormatProvider.class));
|
||||
assertThat(documentMapper.rootMapper(IdFieldMapper.class).docValuesFormatProvider().get(), instanceOf(Lucene410DocValuesFormat.class));
|
||||
assertThat(documentMapper.rootMapper(UidFieldMapper.class).postingsFormatProvider().get(), instanceOf(Lucene41PostingsFormat.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -343,50 +164,10 @@ public class CodecTests extends ElasticsearchSingleNodeLuceneTestCase {
|
|||
assertThat(documentMapper.mappers().name("field2").mapper().docValuesFormatProvider(), instanceOf(DefaultDocValuesFormatProvider.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResolveDocValuesFormatsFromMapping_memory() throws Exception {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties")
|
||||
.startObject("field1").field("type", "integer").field("doc_values_format", "memory").endObject()
|
||||
.startObject("field2").field("type", "double").field("doc_values_format", "my_format1").endObject()
|
||||
.endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
Settings indexSettings = ImmutableSettings.settingsBuilder()
|
||||
.put("index.codec.doc_values_format.my_format1.type", "memory")
|
||||
.build();
|
||||
CodecService codecService = createCodecService(indexSettings);
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().docValuesFormatProvider(), instanceOf(PreBuiltDocValuesFormatProvider.class));
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().docValuesFormatProvider().get(), instanceOf(MemoryDocValuesFormat.class));
|
||||
|
||||
assertThat(documentMapper.mappers().name("field2").mapper().docValuesFormatProvider(), instanceOf(MemoryDocValuesFormatProvider.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testResolveDocValuesFormatsFromMapping_disk() throws Exception {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties")
|
||||
.startObject("field1").field("type", "integer").field("doc_values_format", "disk").endObject()
|
||||
.startObject("field2").field("type", "double").field("doc_values_format", "my_format1").endObject()
|
||||
.endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
Settings indexSettings = ImmutableSettings.settingsBuilder()
|
||||
.put("index.codec.doc_values_format.my_format1.type", "disk")
|
||||
.build();
|
||||
CodecService codecService = createCodecService(indexSettings);
|
||||
DocumentMapper documentMapper = codecService.mapperService().documentMapperParser().parse(mapping);
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().docValuesFormatProvider(), instanceOf(PreBuiltDocValuesFormatProvider.class));
|
||||
assertThat(documentMapper.mappers().name("field1").mapper().docValuesFormatProvider().get(), instanceOf(Lucene410DocValuesFormat.class));
|
||||
|
||||
assertThat(documentMapper.mappers().name("field2").mapper().docValuesFormatProvider(), instanceOf(DiskDocValuesFormatProvider.class));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testChangeVersionFormat() throws Exception {
|
||||
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("_version").field("doc_values_format", "disk").endObject()
|
||||
.startObject("_version").field("doc_values_format", "Lucene410").endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
CodecService codecService = createCodecService();
|
||||
|
|
|
@ -129,7 +129,7 @@ public class TestMergeMapperTests extends ElasticsearchSingleNodeTest {
|
|||
.startObject("properties").startObject("field").field("type", "string").field("search_analyzer", "whitespace").endObject().endObject()
|
||||
.endObject().endObject().string();
|
||||
String mapping2 = XContentFactory.jsonBuilder().startObject().startObject("type")
|
||||
.startObject("properties").startObject("field").field("type", "string").field("postings_format", "direct").endObject().endObject()
|
||||
.startObject("properties").startObject("field").field("type", "string").field("postings_format", "Lucene41").endObject().endObject()
|
||||
.endObject().endObject().string();
|
||||
|
||||
DocumentMapper existing = parser.parse(mapping1);
|
||||
|
@ -140,7 +140,7 @@ public class TestMergeMapperTests extends ElasticsearchSingleNodeTest {
|
|||
|
||||
assertThat(mergeResult.hasConflicts(), equalTo(false));
|
||||
assertThat(((NamedAnalyzer) existing.mappers().name("field").mapper().searchAnalyzer()).name(), equalTo("whitespace"));
|
||||
assertThat((existing.mappers().name("field").mapper().postingsFormatProvider()).name(), equalTo("direct"));
|
||||
assertThat((existing.mappers().name("field").mapper().postingsFormatProvider()).name(), equalTo("Lucene41"));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue