mirror of
https://github.com/honeymoose/OpenSearch.git
synced 2025-03-02 17:09:18 +00:00
uid to use bloom filter posting by default
This commit is contained in:
parent
3577d826f2
commit
e2e25ffea3
@ -19,51 +19,24 @@
|
||||
|
||||
package org.elasticsearch.index.codec;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import org.apache.lucene.codecs.PostingsFormat;
|
||||
import org.apache.lucene.codecs.bloom.BloomFilteringPostingsFormat;
|
||||
import org.apache.lucene.codecs.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
|
||||
import org.elasticsearch.ElasticSearchIllegalArgumentException;
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.inject.Scopes;
|
||||
import org.elasticsearch.common.inject.assistedinject.FactoryProvider;
|
||||
import org.elasticsearch.common.inject.multibindings.MapBinder;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.codec.postingsformat.PostingFormats;
|
||||
import org.elasticsearch.index.codec.postingsformat.PostingsFormatProvider;
|
||||
import org.elasticsearch.index.codec.postingsformat.PostingsFormatService;
|
||||
import org.elasticsearch.index.codec.postingsformat.PreBuiltPostingsFormatProvider;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class CodecModule extends AbstractModule {
|
||||
|
||||
public static final ImmutableList<PreBuiltPostingsFormatProvider.Factory> preConfiguredPostingFormats;
|
||||
|
||||
static {
|
||||
List<PreBuiltPostingsFormatProvider.Factory> preConfiguredPostingFormatsX = Lists.newArrayList();
|
||||
// add defaults ones
|
||||
for (String luceneName : PostingsFormat.availablePostingsFormats()) {
|
||||
preConfiguredPostingFormatsX.add(new PreBuiltPostingsFormatProvider.Factory(PostingsFormat.forName(luceneName)));
|
||||
}
|
||||
preConfiguredPostingFormatsX.add(new PreBuiltPostingsFormatProvider.Factory("direct", new DirectPostingsFormat()));
|
||||
preConfiguredPostingFormatsX.add(new PreBuiltPostingsFormatProvider.Factory("memory", new MemoryPostingsFormat()));
|
||||
// LUCENE UPGRADE: Need to change this to the relevant ones on a lucene upgrade
|
||||
preConfiguredPostingFormatsX.add(new PreBuiltPostingsFormatProvider.Factory("pulsing", new Pulsing40PostingsFormat()));
|
||||
preConfiguredPostingFormatsX.add(new PreBuiltPostingsFormatProvider.Factory("bloom_pulsing", new BloomFilteringPostingsFormat(new Pulsing40PostingsFormat())));
|
||||
preConfiguredPostingFormatsX.add(new PreBuiltPostingsFormatProvider.Factory("default", new Lucene40PostingsFormat()));
|
||||
preConfiguredPostingFormatsX.add(new PreBuiltPostingsFormatProvider.Factory("bloom_default", new BloomFilteringPostingsFormat(new Lucene40PostingsFormat())));
|
||||
|
||||
preConfiguredPostingFormats = ImmutableList.copyOf(preConfiguredPostingFormatsX);
|
||||
}
|
||||
|
||||
private final Settings indexSettings;
|
||||
|
||||
private Map<String, Class<? extends PostingsFormatProvider>> customProviders = Maps.newHashMap();
|
||||
@ -105,7 +78,7 @@ public class CodecModule extends AbstractModule {
|
||||
postingFormatFactoryBinder.addBinding(entry.getKey()).toProvider(FactoryProvider.newFactory(PostingsFormatProvider.Factory.class, entry.getValue())).in(Scopes.SINGLETON);
|
||||
}
|
||||
|
||||
for (PreBuiltPostingsFormatProvider.Factory factory : preConfiguredPostingFormats) {
|
||||
for (PreBuiltPostingsFormatProvider.Factory factory : PostingFormats.listFactories()) {
|
||||
if (postingFormatProviders.containsKey(factory.name())) {
|
||||
continue;
|
||||
}
|
||||
|
@ -0,0 +1,66 @@
|
||||
/*
|
||||
* Licensed to ElasticSearch and Shay Banon 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 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.lucene40.Lucene40PostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.DirectPostingsFormat;
|
||||
import org.apache.lucene.codecs.memory.MemoryPostingsFormat;
|
||||
import org.apache.lucene.codecs.pulsing.Pulsing40PostingsFormat;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class PostingFormats {
|
||||
|
||||
private static final ImmutableMap<String, PreBuiltPostingsFormatProvider.Factory> builtInPostingFormats;
|
||||
|
||||
static {
|
||||
MapBuilder<String, PreBuiltPostingsFormatProvider.Factory> buildInPostingFormatsX = MapBuilder.newMapBuilder();
|
||||
// add defaults ones
|
||||
for (String luceneName : PostingsFormat.availablePostingsFormats()) {
|
||||
buildInPostingFormatsX.put(luceneName, new PreBuiltPostingsFormatProvider.Factory(PostingsFormat.forName(luceneName)));
|
||||
}
|
||||
buildInPostingFormatsX.put("direct", new PreBuiltPostingsFormatProvider.Factory("direct", new DirectPostingsFormat()));
|
||||
buildInPostingFormatsX.put("memory", new PreBuiltPostingsFormatProvider.Factory("memory", new MemoryPostingsFormat()));
|
||||
// LUCENE UPGRADE: Need to change this to the relevant ones on a lucene upgrade
|
||||
buildInPostingFormatsX.put("pulsing", new PreBuiltPostingsFormatProvider.Factory("pulsing", new Pulsing40PostingsFormat()));
|
||||
buildInPostingFormatsX.put("bloom_pulsing", new PreBuiltPostingsFormatProvider.Factory("bloom_pulsing", new BloomFilteringPostingsFormat(new Pulsing40PostingsFormat())));
|
||||
buildInPostingFormatsX.put("default", new PreBuiltPostingsFormatProvider.Factory("default", new Lucene40PostingsFormat()));
|
||||
buildInPostingFormatsX.put("bloom_default", new PreBuiltPostingsFormatProvider.Factory("bloom_default", new BloomFilteringPostingsFormat(new Lucene40PostingsFormat())));
|
||||
|
||||
builtInPostingFormats = buildInPostingFormatsX.immutableMap();
|
||||
}
|
||||
|
||||
public static PostingsFormatProvider.Factory getAsFactory(String name) {
|
||||
return builtInPostingFormats.get(name);
|
||||
}
|
||||
|
||||
public static PostingsFormatProvider getAsProvider(String name) {
|
||||
return builtInPostingFormats.get(name).get();
|
||||
}
|
||||
|
||||
public static ImmutableCollection<PreBuiltPostingsFormatProvider.Factory> listFactories() {
|
||||
return builtInPostingFormats.values();
|
||||
}
|
||||
}
|
@ -27,7 +27,6 @@ import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.codec.CodecModule;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
import java.util.Map;
|
||||
@ -66,7 +65,7 @@ public class PostingsFormatService extends AbstractIndexComponent {
|
||||
|
||||
// even though we have this logic in the cache module (where it should be, so posting format with delegates will work properly wiht the pre initialized map)
|
||||
// we do it here as well so we can use just this instance for tests
|
||||
for (PreBuiltPostingsFormatProvider.Factory factory : CodecModule.preConfiguredPostingFormats) {
|
||||
for (PreBuiltPostingsFormatProvider.Factory factory : PostingFormats.listFactories()) {
|
||||
if (providers.containsKey(factory.name())) {
|
||||
continue;
|
||||
}
|
||||
|
@ -32,6 +32,7 @@ import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.lucene.search.TermFilter;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.analysis.NamedAnalyzer;
|
||||
import org.elasticsearch.index.codec.postingsformat.PostingFormats;
|
||||
import org.elasticsearch.index.codec.postingsformat.PostingsFormatProvider;
|
||||
import org.elasticsearch.index.field.data.FieldDataType;
|
||||
import org.elasticsearch.index.mapper.*;
|
||||
@ -269,9 +270,19 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T>, Mapper {
|
||||
} else {
|
||||
this.searchAnalyzer = searchAnalyzer;
|
||||
}
|
||||
if (postingsFormat == null) {
|
||||
if (defaultPostingFormat() != null) {
|
||||
postingsFormat = PostingFormats.getAsProvider(defaultPostingFormat());
|
||||
}
|
||||
}
|
||||
this.postingsFormat = postingsFormat;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
protected String defaultPostingFormat() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String name() {
|
||||
return names.name();
|
||||
@ -583,7 +594,9 @@ public abstract class AbstractFieldMapper<T> implements FieldMapper<T>, Mapper {
|
||||
}
|
||||
}
|
||||
if (postingsFormat != null) {
|
||||
builder.field("postings_format", postingsFormat.name());
|
||||
if (!postingsFormat.name().equals(defaultPostingFormat())) {
|
||||
builder.field("postings_format", postingsFormat.name());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -23,9 +23,11 @@ import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.FieldType;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
import org.elasticsearch.common.lucene.uid.UidField;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.codec.postingsformat.PostingsFormatProvider;
|
||||
import org.elasticsearch.index.mapper.*;
|
||||
import org.elasticsearch.index.mapper.core.AbstractFieldMapper;
|
||||
|
||||
@ -61,6 +63,7 @@ public class UidFieldMapper extends AbstractFieldMapper<Uid> implements Internal
|
||||
public static class Builder extends Mapper.Builder<Builder, UidFieldMapper> {
|
||||
|
||||
protected String indexName;
|
||||
protected PostingsFormatProvider postingsFormat;
|
||||
|
||||
public Builder() {
|
||||
super(Defaults.NAME);
|
||||
@ -69,14 +72,23 @@ public class UidFieldMapper extends AbstractFieldMapper<Uid> implements Internal
|
||||
|
||||
@Override
|
||||
public UidFieldMapper build(BuilderContext context) {
|
||||
return new UidFieldMapper(name, indexName);
|
||||
return new UidFieldMapper(name, indexName, postingsFormat);
|
||||
}
|
||||
}
|
||||
|
||||
public static class TypeParser implements Mapper.TypeParser {
|
||||
@Override
|
||||
public Mapper.Builder parse(String name, Map<String, Object> node, ParserContext parserContext) throws MapperParsingException {
|
||||
return uid();
|
||||
Builder builder = uid();
|
||||
for (Map.Entry<String, Object> entry : node.entrySet()) {
|
||||
String fieldName = Strings.toUnderscoreCase(entry.getKey());
|
||||
Object fieldNode = entry.getValue();
|
||||
if (fieldName.equals("postings_format")) {
|
||||
String postingFormatName = fieldNode.toString();
|
||||
builder.postingsFormat = parserContext.postingFormatService().get(postingFormatName);
|
||||
}
|
||||
}
|
||||
return builder;
|
||||
}
|
||||
}
|
||||
|
||||
@ -92,12 +104,17 @@ public class UidFieldMapper extends AbstractFieldMapper<Uid> implements Internal
|
||||
}
|
||||
|
||||
protected UidFieldMapper(String name) {
|
||||
this(name, name);
|
||||
this(name, name, null);
|
||||
}
|
||||
|
||||
protected UidFieldMapper(String name, String indexName) {
|
||||
protected UidFieldMapper(String name, String indexName, PostingsFormatProvider postingsFormat) {
|
||||
super(new Names(name, indexName, indexName, name), Defaults.BOOST, new FieldType(Defaults.UID_FIELD_TYPE),
|
||||
Lucene.KEYWORD_ANALYZER, Lucene.KEYWORD_ANALYZER, null);
|
||||
Lucene.KEYWORD_ANALYZER, Lucene.KEYWORD_ANALYZER, postingsFormat);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String defaultPostingFormat() {
|
||||
return "bloom_default";
|
||||
}
|
||||
|
||||
@Override
|
||||
|
Loading…
x
Reference in New Issue
Block a user