Improve performance for many new fields introduction in mapping
When we have many new fields keep being introduced, the immutable open map we used becomes more and more expensive because of its clone characteristics, and we use it in several places. The usage semantics of it allows us to also use a CHM if we want to, but it would be nice to still maintain the concurrency aspects of volatile immutable map when the number of fields is sane. Introduce a new map like data structure, that can switch internally to CHM when a certain threshold is met. Also add a benchmark class to exploit the many new field mappings use case, which shows significant gains by using this change, to a level where mapping introduction is no longer a significant bottleneck. closes #6707
This commit is contained in:
parent
a9abf18235
commit
c8e553054b
|
@ -50,6 +50,7 @@ import org.elasticsearch.transport.TransportService;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
|
@ -181,7 +182,7 @@ public class TransportGetFieldMappingsIndexAction extends TransportSingleCustomO
|
|||
|
||||
private ImmutableMap<String, FieldMappingMetaData> findFieldMappingsByType(DocumentMapper documentMapper, GetFieldMappingsIndexRequest request) throws ElasticsearchException {
|
||||
MapBuilder<String, FieldMappingMetaData> fieldMappings = new MapBuilder<>();
|
||||
ImmutableList<FieldMapper> allFieldMappers = documentMapper.mappers().mappers();
|
||||
final List<FieldMapper> allFieldMappers = documentMapper.mappers().mappers();
|
||||
for (String field : request.fields()) {
|
||||
if (Regex.isMatchAllPattern(field)) {
|
||||
for (FieldMapper fieldMapper : allFieldMappers) {
|
||||
|
|
|
@ -0,0 +1,177 @@
|
|||
/*
|
||||
* 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.common.collect;
|
||||
|
||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.elasticsearch.ElasticsearchIllegalStateException;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
* A map that exposes only read only methods, and can be mutated using a {@link #mutator()}. It
|
||||
* allows for a cutoff switch between {@link ImmutableOpenMap} and {@link ConcurrentMap}, based on size, since as
|
||||
* the size grows bigger, cloning the immutable map cost gets bigger and bigger, and might as well move to CHM.
|
||||
* <p/>
|
||||
* Note, its important to understand the semantics of the class and its mutator, its not an update in place, when
|
||||
* CHM is used, changes to the mutator will be reflected in the existing maps!. This class should be used as if
|
||||
* its a regular, mutable concurrent map, mutation can affect the existing map.
|
||||
* <p/>
|
||||
* This class only allows for a single concurrent mutator to execute at the same time.
|
||||
*/
|
||||
public final class UpdateInPlaceMap<K, V> {
|
||||
|
||||
final int switchSize;
|
||||
final AtomicBoolean mutating = new AtomicBoolean();
|
||||
volatile ImmutableOpenMap<K, V> immutableMap;
|
||||
volatile ConcurrentMap<K, V> concurrentMap;
|
||||
|
||||
UpdateInPlaceMap(int switchSize) {
|
||||
this.switchSize = switchSize;
|
||||
if (switchSize == 0) {
|
||||
this.concurrentMap = ConcurrentCollections.newConcurrentMap();
|
||||
this.immutableMap = null;
|
||||
} else {
|
||||
this.concurrentMap = null;
|
||||
this.immutableMap = ImmutableOpenMap.of();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns if the map is empty or not.
|
||||
*/
|
||||
public boolean isEmpty() {
|
||||
final ImmutableOpenMap<K, V> immutableMap = this.immutableMap;
|
||||
final ConcurrentMap<K, V> concurrentMap = this.concurrentMap;
|
||||
return immutableMap != null ? immutableMap.isEmpty() : concurrentMap.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the value matching a key, or null if not matched.
|
||||
*/
|
||||
public V get(K key) {
|
||||
final ImmutableOpenMap<K, V> immutableMap = this.immutableMap;
|
||||
final ConcurrentMap<K, V> concurrentMap = this.concurrentMap;
|
||||
return immutableMap != null ? immutableMap.get(key) : concurrentMap.get(key);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all the values in the map, on going mutator changes might or might not be reflected
|
||||
* in the values.
|
||||
*/
|
||||
public Iterable<V> values() {
|
||||
return new Iterable<V>() {
|
||||
@Override
|
||||
public Iterator<V> iterator() {
|
||||
final ImmutableOpenMap<K, V> immutableMap = UpdateInPlaceMap.this.immutableMap;
|
||||
final ConcurrentMap<K, V> concurrentMap = UpdateInPlaceMap.this.concurrentMap;
|
||||
if (immutableMap != null) {
|
||||
return immutableMap.valuesIt();
|
||||
} else {
|
||||
return Iterables.unmodifiableIterable(concurrentMap.values()).iterator();
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Opens a mutator allowing to mutate this map. Note, only one mutator is allowed to execute.
|
||||
*/
|
||||
public Mutator mutator() {
|
||||
if (!mutating.compareAndSet(false, true)) {
|
||||
throw new ElasticsearchIllegalStateException("map is already mutating, can't have another mutator on it");
|
||||
}
|
||||
return new Mutator();
|
||||
}
|
||||
|
||||
public static <K, V> UpdateInPlaceMap<K, V> of(int switchSize) {
|
||||
return new UpdateInPlaceMap<>(switchSize);
|
||||
}
|
||||
|
||||
public final class Mutator implements Releasable {
|
||||
|
||||
private ImmutableOpenMap.Builder<K, V> immutableBuilder;
|
||||
|
||||
private Mutator() {
|
||||
if (immutableMap != null) {
|
||||
immutableBuilder = ImmutableOpenMap.builder(immutableMap);
|
||||
} else {
|
||||
immutableBuilder = null;
|
||||
}
|
||||
}
|
||||
|
||||
public V get(K key) {
|
||||
if (immutableBuilder != null) {
|
||||
return immutableBuilder.get(key);
|
||||
}
|
||||
return concurrentMap.get(key);
|
||||
}
|
||||
|
||||
public V put(K key, V value) {
|
||||
if (immutableBuilder != null) {
|
||||
V v = immutableBuilder.put(key, value);
|
||||
switchIfNeeded();
|
||||
return v;
|
||||
} else {
|
||||
return concurrentMap.put(key, value);
|
||||
}
|
||||
}
|
||||
|
||||
public Mutator putAll(Map<K, V> map) {
|
||||
for (Map.Entry<K, V> entry : map.entrySet()) {
|
||||
put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public V remove(K key) {
|
||||
return immutableBuilder != null ? immutableBuilder.remove(key) : concurrentMap.remove(key);
|
||||
}
|
||||
|
||||
private void switchIfNeeded() {
|
||||
if (concurrentMap != null) {
|
||||
assert immutableBuilder == null;
|
||||
return;
|
||||
}
|
||||
if (immutableBuilder.size() <= switchSize) {
|
||||
return;
|
||||
}
|
||||
concurrentMap = ConcurrentCollections.newConcurrentMap();
|
||||
for (ObjectObjectCursor<K, V> cursor : immutableBuilder) {
|
||||
concurrentMap.put(cursor.key, cursor.value);
|
||||
}
|
||||
immutableBuilder = null;
|
||||
immutableMap = null;
|
||||
}
|
||||
|
||||
public void close() {
|
||||
if (immutableBuilder != null) {
|
||||
immutableMap = immutableBuilder.build();
|
||||
}
|
||||
assert (immutableBuilder != null && concurrentMap == null) || (immutableBuilder == null && concurrentMap != null);
|
||||
mutating.set(false);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -21,23 +21,22 @@ package org.elasticsearch.index.analysis;
|
|||
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.apache.lucene.analysis.SimpleAnalyzerWrapper;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.collect.UpdateInPlaceMap;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public final class FieldNameAnalyzer extends SimpleAnalyzerWrapper {
|
||||
|
||||
private final ImmutableOpenMap<String, Analyzer> analyzers;
|
||||
|
||||
private final UpdateInPlaceMap<String, Analyzer> analyzers;
|
||||
private final Analyzer defaultAnalyzer;
|
||||
|
||||
public FieldNameAnalyzer(ImmutableOpenMap<String, Analyzer> analyzers, Analyzer defaultAnalyzer) {
|
||||
public FieldNameAnalyzer(UpdateInPlaceMap<String, Analyzer> analyzers, Analyzer defaultAnalyzer) {
|
||||
this.analyzers = analyzers;
|
||||
this.defaultAnalyzer = defaultAnalyzer;
|
||||
}
|
||||
|
||||
public ImmutableOpenMap<String, Analyzer> analyzers() {
|
||||
public UpdateInPlaceMap<String, Analyzer> analyzers() {
|
||||
return analyzers;
|
||||
}
|
||||
|
||||
|
|
|
@ -22,9 +22,14 @@ package org.elasticsearch.index.mapper;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.UnmodifiableIterator;
|
||||
import org.apache.lucene.analysis.Analyzer;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.collect.UpdateInPlaceMap;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.analysis.FieldNameAnalyzer;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
|
@ -35,40 +40,40 @@ public class DocumentFieldMappers implements Iterable<FieldMapper> {
|
|||
private final DocumentMapper docMapper;
|
||||
private final FieldMappersLookup fieldMappers;
|
||||
|
||||
private volatile FieldNameAnalyzer indexAnalyzer;
|
||||
private volatile FieldNameAnalyzer searchAnalyzer;
|
||||
private volatile FieldNameAnalyzer searchQuoteAnalyzer;
|
||||
private final FieldNameAnalyzer indexAnalyzer;
|
||||
private final FieldNameAnalyzer searchAnalyzer;
|
||||
private final FieldNameAnalyzer searchQuoteAnalyzer;
|
||||
|
||||
public DocumentFieldMappers(DocumentMapper docMapper) {
|
||||
public DocumentFieldMappers(@Nullable @IndexSettings Settings settings, DocumentMapper docMapper) {
|
||||
this.docMapper = docMapper;
|
||||
this.fieldMappers = new FieldMappersLookup();
|
||||
this.indexAnalyzer = new FieldNameAnalyzer(ImmutableOpenMap.<String, Analyzer>of(), docMapper.indexAnalyzer());
|
||||
this.searchAnalyzer = new FieldNameAnalyzer(ImmutableOpenMap.<String, Analyzer>of(), docMapper.searchAnalyzer());
|
||||
this.searchQuoteAnalyzer = new FieldNameAnalyzer(ImmutableOpenMap.<String, Analyzer>of(), docMapper.searchQuotedAnalyzer());
|
||||
this.fieldMappers = new FieldMappersLookup(settings);
|
||||
this.indexAnalyzer = new FieldNameAnalyzer(UpdateInPlaceMap.<String, Analyzer>of(MapperService.getFieldMappersCollectionSwitch(settings)), docMapper.indexAnalyzer());
|
||||
this.searchAnalyzer = new FieldNameAnalyzer(UpdateInPlaceMap.<String, Analyzer>of(MapperService.getFieldMappersCollectionSwitch(settings)), docMapper.searchAnalyzer());
|
||||
this.searchQuoteAnalyzer = new FieldNameAnalyzer(UpdateInPlaceMap.<String, Analyzer>of(MapperService.getFieldMappersCollectionSwitch(settings)), docMapper.searchQuotedAnalyzer());
|
||||
}
|
||||
|
||||
public void addNewMappers(Iterable<FieldMapper> newMappers) {
|
||||
public void addNewMappers(List<FieldMapper> newMappers) {
|
||||
fieldMappers.addNewMappers(newMappers);
|
||||
|
||||
final ImmutableOpenMap.Builder<String, Analyzer> indexAnalyzers = ImmutableOpenMap.builder(this.indexAnalyzer.analyzers());
|
||||
final ImmutableOpenMap.Builder<String, Analyzer> searchAnalyzers = ImmutableOpenMap.builder(this.searchAnalyzer.analyzers());
|
||||
final ImmutableOpenMap.Builder<String, Analyzer> searchQuoteAnalyzers = ImmutableOpenMap.builder(this.searchQuoteAnalyzer.analyzers());
|
||||
final UpdateInPlaceMap<String, Analyzer>.Mutator indexAnalyzersMutator = this.indexAnalyzer.analyzers().mutator();
|
||||
final UpdateInPlaceMap<String, Analyzer>.Mutator searchAnalyzersMutator = this.searchAnalyzer.analyzers().mutator();
|
||||
final UpdateInPlaceMap<String, Analyzer>.Mutator searchQuoteAnalyzersMutator = this.searchQuoteAnalyzer.analyzers().mutator();
|
||||
|
||||
for (FieldMapper fieldMapper : newMappers) {
|
||||
if (fieldMapper.indexAnalyzer() != null) {
|
||||
indexAnalyzers.put(fieldMapper.names().indexName(), fieldMapper.indexAnalyzer());
|
||||
indexAnalyzersMutator.put(fieldMapper.names().indexName(), fieldMapper.indexAnalyzer());
|
||||
}
|
||||
if (fieldMapper.searchAnalyzer() != null) {
|
||||
searchAnalyzers.put(fieldMapper.names().indexName(), fieldMapper.searchAnalyzer());
|
||||
searchAnalyzersMutator.put(fieldMapper.names().indexName(), fieldMapper.searchAnalyzer());
|
||||
}
|
||||
if (fieldMapper.searchQuoteAnalyzer() != null) {
|
||||
searchQuoteAnalyzers.put(fieldMapper.names().indexName(), fieldMapper.searchQuoteAnalyzer());
|
||||
searchQuoteAnalyzersMutator.put(fieldMapper.names().indexName(), fieldMapper.searchQuoteAnalyzer());
|
||||
}
|
||||
}
|
||||
|
||||
this.indexAnalyzer = new FieldNameAnalyzer(indexAnalyzers.build(), docMapper.indexAnalyzer());
|
||||
this.searchAnalyzer = new FieldNameAnalyzer(searchAnalyzers.build(), docMapper.searchAnalyzer());
|
||||
this.searchQuoteAnalyzer = new FieldNameAnalyzer(searchQuoteAnalyzers.build(), docMapper.searchQuotedAnalyzer());
|
||||
indexAnalyzersMutator.close();
|
||||
searchAnalyzersMutator.close();
|
||||
searchQuoteAnalyzersMutator.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -76,7 +81,7 @@ public class DocumentFieldMappers implements Iterable<FieldMapper> {
|
|||
return fieldMappers.iterator();
|
||||
}
|
||||
|
||||
public ImmutableList<FieldMapper> mappers() {
|
||||
public List<FieldMapper> mappers() {
|
||||
return this.fieldMappers.mappers();
|
||||
}
|
||||
|
||||
|
|
|
@ -329,7 +329,7 @@ public class DocumentMapper implements ToXContent {
|
|||
// now traverse and get all the statically defined ones
|
||||
rootObjectMapper.traverse(fieldMappersAgg);
|
||||
|
||||
this.fieldMappers = new DocumentFieldMappers(this);
|
||||
this.fieldMappers = new DocumentFieldMappers(indexSettings, this);
|
||||
this.fieldMappers.addNewMappers(fieldMappersAgg.mappers);
|
||||
|
||||
final Map<String, ObjectMapper> objectMappers = Maps.newHashMap();
|
||||
|
@ -566,7 +566,7 @@ public class DocumentMapper implements ToXContent {
|
|||
return doc;
|
||||
}
|
||||
|
||||
public void addFieldMappers(Iterable<FieldMapper> fieldMappers) {
|
||||
public void addFieldMappers(List<FieldMapper> fieldMappers) {
|
||||
synchronized (mappersMutex) {
|
||||
this.fieldMappers.addNewMappers(fieldMappers);
|
||||
}
|
||||
|
|
|
@ -38,7 +38,7 @@ public abstract class FieldMapperListener {
|
|||
|
||||
public abstract void fieldMapper(FieldMapper fieldMapper);
|
||||
|
||||
public void fieldMappers(Iterable<FieldMapper> fieldMappers) {
|
||||
public void fieldMappers(List<FieldMapper> fieldMappers) {
|
||||
for (FieldMapper mapper : fieldMappers) {
|
||||
fieldMapper(mapper);
|
||||
}
|
||||
|
|
|
@ -19,14 +19,15 @@
|
|||
|
||||
package org.elasticsearch.index.mapper;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.common.collect.UnmodifiableIterator;
|
||||
import com.google.common.collect.*;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.collect.UpdateInPlaceMap;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -35,94 +36,104 @@ import java.util.Set;
|
|||
*/
|
||||
public class FieldMappersLookup implements Iterable<FieldMapper> {
|
||||
|
||||
private volatile ImmutableList<FieldMapper> mappers;
|
||||
private volatile ImmutableOpenMap<String, FieldMappers> name;
|
||||
private volatile ImmutableOpenMap<String, FieldMappers> indexName;
|
||||
private volatile ImmutableOpenMap<String, FieldMappers> fullName;
|
||||
private volatile FieldMapper[] mappers;
|
||||
private volatile List<FieldMapper> mappersAsList;
|
||||
private final UpdateInPlaceMap<String, FieldMappers> name;
|
||||
private final UpdateInPlaceMap<String, FieldMappers> indexName;
|
||||
private final UpdateInPlaceMap<String, FieldMappers> fullName;
|
||||
|
||||
public FieldMappersLookup() {
|
||||
this.mappers = ImmutableList.of();
|
||||
this.fullName = ImmutableOpenMap.of();
|
||||
this.name = ImmutableOpenMap.of();
|
||||
this.indexName = ImmutableOpenMap.of();
|
||||
public FieldMappersLookup(Settings settings) {
|
||||
this.mappers = new FieldMapper[0];
|
||||
this.mappersAsList = ImmutableList.of();
|
||||
this.fullName = UpdateInPlaceMap.of(MapperService.getFieldMappersCollectionSwitch(settings));
|
||||
this.name = UpdateInPlaceMap.of(MapperService.getFieldMappersCollectionSwitch(settings));
|
||||
this.indexName = UpdateInPlaceMap.of(MapperService.getFieldMappersCollectionSwitch(settings));
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new set of mappers.
|
||||
*/
|
||||
public void addNewMappers(Iterable<FieldMapper> newMappers) {
|
||||
final ImmutableOpenMap.Builder<String, FieldMappers> tempName = ImmutableOpenMap.builder(name);
|
||||
final ImmutableOpenMap.Builder<String, FieldMappers> tempIndexName = ImmutableOpenMap.builder(indexName);
|
||||
final ImmutableOpenMap.Builder<String, FieldMappers> tempFullName = ImmutableOpenMap.builder(fullName);
|
||||
public void addNewMappers(List<FieldMapper> newMappers) {
|
||||
final UpdateInPlaceMap<String, FieldMappers>.Mutator mutatorName = name.mutator();
|
||||
final UpdateInPlaceMap<String, FieldMappers>.Mutator mutatorIndexName = indexName.mutator();
|
||||
final UpdateInPlaceMap<String, FieldMappers>.Mutator mutatorFullName = fullName.mutator();
|
||||
|
||||
for (FieldMapper fieldMapper : newMappers) {
|
||||
FieldMappers mappers = tempName.get(fieldMapper.names().name());
|
||||
FieldMappers mappers = mutatorName.get(fieldMapper.names().name());
|
||||
if (mappers == null) {
|
||||
mappers = new FieldMappers(fieldMapper);
|
||||
} else {
|
||||
mappers = mappers.concat(fieldMapper);
|
||||
}
|
||||
tempName.put(fieldMapper.names().name(), mappers);
|
||||
mutatorName.put(fieldMapper.names().name(), mappers);
|
||||
|
||||
mappers = tempIndexName.get(fieldMapper.names().indexName());
|
||||
mappers = mutatorIndexName.get(fieldMapper.names().indexName());
|
||||
if (mappers == null) {
|
||||
mappers = new FieldMappers(fieldMapper);
|
||||
} else {
|
||||
mappers = mappers.concat(fieldMapper);
|
||||
}
|
||||
tempIndexName.put(fieldMapper.names().indexName(), mappers);
|
||||
mutatorIndexName.put(fieldMapper.names().indexName(), mappers);
|
||||
|
||||
mappers = tempFullName.get(fieldMapper.names().fullName());
|
||||
mappers = mutatorFullName.get(fieldMapper.names().fullName());
|
||||
if (mappers == null) {
|
||||
mappers = new FieldMappers(fieldMapper);
|
||||
} else {
|
||||
mappers = mappers.concat(fieldMapper);
|
||||
}
|
||||
tempFullName.put(fieldMapper.names().fullName(), mappers);
|
||||
mutatorFullName.put(fieldMapper.names().fullName(), mappers);
|
||||
}
|
||||
this.mappers = ImmutableList.<FieldMapper>builder().addAll(this.mappers).addAll(newMappers).build();
|
||||
this.name = tempName.build();
|
||||
this.indexName = tempIndexName.build();
|
||||
this.fullName = tempFullName.build();
|
||||
FieldMapper[] tempMappers = new FieldMapper[this.mappers.length + newMappers.size()];
|
||||
System.arraycopy(mappers, 0, tempMappers, 0, mappers.length);
|
||||
int counter = 0;
|
||||
for (int i = mappers.length; i < tempMappers.length; i++) {
|
||||
tempMappers[i] = newMappers.get(counter++);
|
||||
}
|
||||
this.mappers = tempMappers;
|
||||
this.mappersAsList = Arrays.asList(this.mappers);
|
||||
|
||||
mutatorName.close();
|
||||
mutatorIndexName.close();
|
||||
mutatorFullName.close();
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the set of mappers.
|
||||
*/
|
||||
public void removeMappers(Iterable<FieldMapper> mappersToRemove) {
|
||||
List<FieldMapper> tempMappers = new ArrayList<>(this.mappers);
|
||||
ImmutableOpenMap.Builder<String, FieldMappers> tempName = ImmutableOpenMap.builder(this.name);
|
||||
ImmutableOpenMap.Builder<String, FieldMappers> tempIndexName = ImmutableOpenMap.builder(this.indexName);
|
||||
ImmutableOpenMap.Builder<String, FieldMappers> tempFullName = ImmutableOpenMap.builder(this.fullName);
|
||||
List<FieldMapper> tempMappers = Lists.newArrayList(this.mappers);
|
||||
final UpdateInPlaceMap<String, FieldMappers>.Mutator mutatorName = name.mutator();
|
||||
final UpdateInPlaceMap<String, FieldMappers>.Mutator mutatorIndexName = indexName.mutator();
|
||||
final UpdateInPlaceMap<String, FieldMappers>.Mutator mutatorFullName = fullName.mutator();
|
||||
|
||||
for (FieldMapper mapper : mappersToRemove) {
|
||||
FieldMappers mappers = tempName.get(mapper.names().name());
|
||||
FieldMappers mappers = mutatorName.get(mapper.names().name());
|
||||
if (mappers != null) {
|
||||
mappers = mappers.remove(mapper);
|
||||
if (mappers.isEmpty()) {
|
||||
tempName.remove(mapper.names().name());
|
||||
mutatorName.remove(mapper.names().name());
|
||||
} else {
|
||||
tempName.put(mapper.names().name(), mappers);
|
||||
mutatorName.put(mapper.names().name(), mappers);
|
||||
}
|
||||
}
|
||||
|
||||
mappers = tempIndexName.get(mapper.names().indexName());
|
||||
mappers = mutatorIndexName.get(mapper.names().indexName());
|
||||
if (mappers != null) {
|
||||
mappers = mappers.remove(mapper);
|
||||
if (mappers.isEmpty()) {
|
||||
tempIndexName.remove(mapper.names().indexName());
|
||||
mutatorIndexName.remove(mapper.names().indexName());
|
||||
} else {
|
||||
tempIndexName.put(mapper.names().indexName(), mappers);
|
||||
mutatorIndexName.put(mapper.names().indexName(), mappers);
|
||||
}
|
||||
}
|
||||
|
||||
mappers = tempFullName.get(mapper.names().fullName());
|
||||
mappers = mutatorFullName.get(mapper.names().fullName());
|
||||
if (mappers != null) {
|
||||
mappers = mappers.remove(mapper);
|
||||
if (mappers.isEmpty()) {
|
||||
tempFullName.remove(mapper.names().fullName());
|
||||
mutatorFullName.remove(mapper.names().fullName());
|
||||
} else {
|
||||
tempFullName.put(mapper.names().fullName(), mappers);
|
||||
mutatorFullName.put(mapper.names().fullName(), mappers);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -130,29 +141,30 @@ public class FieldMappersLookup implements Iterable<FieldMapper> {
|
|||
}
|
||||
|
||||
|
||||
this.mappers = ImmutableList.copyOf(tempMappers);
|
||||
this.name = tempName.build();
|
||||
this.indexName = tempIndexName.build();
|
||||
this.fullName = tempFullName.build();
|
||||
this.mappers = tempMappers.toArray(new FieldMapper[tempMappers.size()]);
|
||||
this.mappersAsList = Arrays.asList(this.mappers);
|
||||
mutatorName.close();
|
||||
mutatorIndexName.close();
|
||||
mutatorFullName.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public UnmodifiableIterator<FieldMapper> iterator() {
|
||||
return mappers.iterator();
|
||||
return Iterators.unmodifiableIterator(mappersAsList.iterator());
|
||||
}
|
||||
|
||||
/**
|
||||
* The list of all mappers.
|
||||
*/
|
||||
public ImmutableList<FieldMapper> mappers() {
|
||||
return this.mappers;
|
||||
public List<FieldMapper> mappers() {
|
||||
return this.mappersAsList;
|
||||
}
|
||||
|
||||
/**
|
||||
* Is there a mapper (based on unique {@link FieldMapper} identity)?
|
||||
*/
|
||||
public boolean hasMapper(FieldMapper fieldMapper) {
|
||||
return mappers.contains(fieldMapper);
|
||||
return mappersAsList.contains(fieldMapper);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -83,6 +83,16 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
|||
"_size", "_timestamp", "_ttl"
|
||||
);
|
||||
|
||||
public static final String FIELD_MAPPERS_COLLECTION_SWITCH = "index.mapper.field_mappers_collection_switch";
|
||||
public static final int DEFAULT_FIELD_MAPPERS_COLLECTION_SWITCH = 100;
|
||||
|
||||
public static int getFieldMappersCollectionSwitch(@Nullable Settings settings) {
|
||||
if (settings == null) {
|
||||
return DEFAULT_FIELD_MAPPERS_COLLECTION_SWITCH;
|
||||
}
|
||||
return settings.getAsInt(MapperService.FIELD_MAPPERS_COLLECTION_SWITCH, MapperService.DEFAULT_FIELD_MAPPERS_COLLECTION_SWITCH);
|
||||
}
|
||||
|
||||
private final AnalysisService analysisService;
|
||||
private final IndexFieldDataService fieldDataService;
|
||||
|
||||
|
@ -99,7 +109,7 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
|||
private final Object typeMutex = new Object();
|
||||
private final Object mappersMutex = new Object();
|
||||
|
||||
private final FieldMappersLookup fieldMappers = new FieldMappersLookup();
|
||||
private final FieldMappersLookup fieldMappers;
|
||||
private volatile ImmutableOpenMap<String, ObjectMappers> fullPathObjectMappers = ImmutableOpenMap.of();
|
||||
private boolean hasNested = false; // updated dynamically to true when a nested object is added
|
||||
|
||||
|
@ -119,6 +129,7 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
|||
super(index, indexSettings);
|
||||
this.analysisService = analysisService;
|
||||
this.fieldDataService = fieldDataService;
|
||||
this.fieldMappers = new FieldMappersLookup(indexSettings);
|
||||
this.documentParser = new DocumentMapperParser(index, indexSettings, analysisService, postingsFormatService, docValuesFormatService, similarityLookupService);
|
||||
this.searchAnalyzer = new SmartIndexNameSearchAnalyzer(analysisService.defaultSearchAnalyzer());
|
||||
this.searchQuoteAnalyzer = new SmartIndexNameSearchQuoteAnalyzer(analysisService.defaultSearchQuoteAnalyzer());
|
||||
|
@ -330,7 +341,7 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
|||
}
|
||||
}
|
||||
|
||||
private void addFieldMappers(Iterable<FieldMapper> fieldMappers) {
|
||||
private void addFieldMappers(List<FieldMapper> fieldMappers) {
|
||||
synchronized (mappersMutex) {
|
||||
this.fieldMappers.addNewMappers(fieldMappers);
|
||||
}
|
||||
|
@ -1046,7 +1057,7 @@ public class MapperService extends AbstractIndexComponent implements Iterable<Do
|
|||
}
|
||||
|
||||
@Override
|
||||
public void fieldMappers(Iterable<FieldMapper> fieldMappers) {
|
||||
public void fieldMappers(List<FieldMapper> fieldMappers) {
|
||||
addFieldMappers(fieldMappers);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -106,7 +106,7 @@ public class AnalyzerMapper implements Mapper, InternalMapper, RootMapper {
|
|||
List<IndexableField> fields = context.doc().getFields();
|
||||
for (int i = 0, fieldsSize = fields.size(); i < fieldsSize; i++) {
|
||||
IndexableField field = fields.get(i);
|
||||
if (field.name() == path) {
|
||||
if (field.name().equals(path)) {
|
||||
value = field.stringValue();
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.index.mapper.object;
|
||||
|
||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.IndexableField;
|
||||
import org.apache.lucene.index.Term;
|
||||
|
@ -28,9 +29,12 @@ import org.apache.lucene.search.Filter;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.elasticsearch.ElasticsearchIllegalStateException;
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.collect.UpdateInPlaceMap;
|
||||
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.CollectionUtils;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
@ -40,6 +44,7 @@ import org.elasticsearch.index.mapper.ParseContext.Document;
|
|||
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
@ -166,14 +171,14 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
context.path().pathType(origPathType);
|
||||
context.path().remove();
|
||||
|
||||
ObjectMapper objectMapper = createMapper(name, context.path().fullPathAsText(name), enabled, nested, dynamic, pathType, mappers);
|
||||
ObjectMapper objectMapper = createMapper(name, context.path().fullPathAsText(name), enabled, nested, dynamic, pathType, mappers, context.indexSettings());
|
||||
objectMapper.includeInAllIfNotSet(includeInAll);
|
||||
|
||||
return (Y) objectMapper;
|
||||
}
|
||||
|
||||
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers) {
|
||||
return new ObjectMapper(name, fullPath, enabled, nested, dynamic, pathType, mappers);
|
||||
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers, @Nullable @IndexSettings Settings settings) {
|
||||
return new ObjectMapper(name, fullPath, enabled, nested, dynamic, pathType, mappers, settings);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -309,19 +314,22 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
|
||||
private Boolean includeInAll;
|
||||
|
||||
private volatile ImmutableOpenMap<String, Mapper> mappers = ImmutableOpenMap.of();
|
||||
private final UpdateInPlaceMap<String, Mapper> mappers;
|
||||
|
||||
private final Object mutex = new Object();
|
||||
|
||||
ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers) {
|
||||
ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers, @Nullable @IndexSettings Settings settings) {
|
||||
this.name = name;
|
||||
this.fullPath = fullPath;
|
||||
this.enabled = enabled;
|
||||
this.nested = nested;
|
||||
this.dynamic = dynamic;
|
||||
this.pathType = pathType;
|
||||
this.mappers = UpdateInPlaceMap.of(MapperService.getFieldMappersCollectionSwitch(settings));
|
||||
if (mappers != null) {
|
||||
this.mappers = ImmutableOpenMap.builder(this.mappers).putAll(mappers).build();
|
||||
UpdateInPlaceMap<String, Mapper>.Mutator mappersMutator = this.mappers.mutator();
|
||||
mappersMutator.putAll(mappers);
|
||||
mappersMutator.close();
|
||||
}
|
||||
this.nestedTypePathAsString = "__" + fullPath;
|
||||
this.nestedTypePathAsBytes = new BytesRef(nestedTypePathAsString);
|
||||
|
@ -340,9 +348,9 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
}
|
||||
this.includeInAll = includeInAll;
|
||||
// when called from outside, apply this on all the inner mappers
|
||||
for (ObjectObjectCursor<String, Mapper> cursor : mappers) {
|
||||
if (cursor.value instanceof AllFieldMapper.IncludeInAll) {
|
||||
((AllFieldMapper.IncludeInAll) cursor.value).includeInAll(includeInAll);
|
||||
for (Mapper mapper : mappers.values()) {
|
||||
if (mapper instanceof AllFieldMapper.IncludeInAll) {
|
||||
((AllFieldMapper.IncludeInAll) mapper).includeInAll(includeInAll);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -353,9 +361,9 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
this.includeInAll = includeInAll;
|
||||
}
|
||||
// when called from outside, apply this on all the inner mappers
|
||||
for (ObjectObjectCursor<String, Mapper> cursor : mappers) {
|
||||
if (cursor.value instanceof AllFieldMapper.IncludeInAll) {
|
||||
((AllFieldMapper.IncludeInAll) cursor.value).includeInAllIfNotSet(includeInAll);
|
||||
for (Mapper mapper : mappers.values()) {
|
||||
if (mapper instanceof AllFieldMapper.IncludeInAll) {
|
||||
((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -364,9 +372,9 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
public void unsetIncludeInAll() {
|
||||
includeInAll = null;
|
||||
// when called from outside, apply this on all the inner mappers
|
||||
for (ObjectObjectCursor<String, Mapper> cursor : mappers) {
|
||||
if (cursor.value instanceof AllFieldMapper.IncludeInAll) {
|
||||
((AllFieldMapper.IncludeInAll) cursor.value).unsetIncludeInAll();
|
||||
for (Mapper mapper : mappers.values()) {
|
||||
if (mapper instanceof AllFieldMapper.IncludeInAll) {
|
||||
((AllFieldMapper.IncludeInAll) mapper).unsetIncludeInAll();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -384,23 +392,25 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll);
|
||||
}
|
||||
synchronized (mutex) {
|
||||
this.mappers = ImmutableOpenMap.builder(this.mappers).fPut(mapper.name(), mapper).build();
|
||||
UpdateInPlaceMap<String, Mapper>.Mutator mappingMutator = this.mappers.mutator();
|
||||
mappingMutator.put(mapper.name(), mapper);
|
||||
mappingMutator.close();
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void traverse(FieldMapperListener fieldMapperListener) {
|
||||
for (ObjectObjectCursor<String, Mapper> cursor : mappers) {
|
||||
cursor.value.traverse(fieldMapperListener);
|
||||
for (Mapper mapper : mappers.values()) {
|
||||
mapper.traverse(fieldMapperListener);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void traverse(ObjectMapperListener objectMapperListener) {
|
||||
objectMapperListener.objectMapper(this);
|
||||
for (ObjectObjectCursor<String, Mapper> cursor : mappers) {
|
||||
cursor.value.traverse(objectMapperListener);
|
||||
for (Mapper mapper : mappers.values()) {
|
||||
mapper.traverse(objectMapperListener);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -868,8 +878,8 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
FieldMapperListener.Aggregator newFieldMappers = new FieldMapperListener.Aggregator();
|
||||
ObjectMapperListener.Aggregator newObjectMappers = new ObjectMapperListener.Aggregator();
|
||||
synchronized (mutex) {
|
||||
for (ObjectObjectCursor<String, Mapper> cursor : mergeWithObject.mappers) {
|
||||
Mapper mergeWithMapper = cursor.value;
|
||||
for (Mapper mapper : mergeWithObject.mappers.values()) {
|
||||
Mapper mergeWithMapper = mapper;
|
||||
Mapper mergeIntoMapper = mappers.get(mergeWithMapper.name());
|
||||
if (mergeIntoMapper == null) {
|
||||
// no mapping, simply add it if not simulating
|
||||
|
@ -902,8 +912,8 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
|
||||
@Override
|
||||
public void close() {
|
||||
for (ObjectObjectCursor<String, Mapper> cursor : mappers) {
|
||||
cursor.value.close();
|
||||
for (Mapper mapper : mappers.values()) {
|
||||
mapper.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -946,7 +956,7 @@ public class ObjectMapper implements Mapper, AllFieldMapper.IncludeInAll {
|
|||
doXContent(builder, params);
|
||||
|
||||
// sort the mappers so we get consistent serialization format
|
||||
Mapper[] sortedMappers = mappers.values().toArray(Mapper.class);
|
||||
Mapper[] sortedMappers = Iterables.toArray(mappers.values(), Mapper.class);
|
||||
Arrays.sort(sortedMappers, new Comparator<Mapper>() {
|
||||
@Override
|
||||
public int compare(Mapper o1, Mapper o2) {
|
||||
|
|
|
@ -21,13 +21,16 @@ package org.elasticsearch.index.mapper.object;
|
|||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
|
||||
import org.elasticsearch.common.joda.Joda;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.mapper.*;
|
||||
import org.elasticsearch.index.mapper.core.DateFieldMapper;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
|
@ -96,7 +99,7 @@ public class RootObjectMapper extends ObjectMapper {
|
|||
|
||||
|
||||
@Override
|
||||
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers) {
|
||||
protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers, @Nullable @IndexSettings Settings settings) {
|
||||
assert !nested.isNested();
|
||||
FormatDateTimeFormatter[] dates = null;
|
||||
if (dynamicDateTimeFormatters == null) {
|
||||
|
@ -110,7 +113,7 @@ public class RootObjectMapper extends ObjectMapper {
|
|||
return new RootObjectMapper(name, enabled, dynamic, pathType, mappers,
|
||||
dates,
|
||||
dynamicTemplates.toArray(new DynamicTemplate[dynamicTemplates.size()]),
|
||||
dateDetection, numericDetection);
|
||||
dateDetection, numericDetection, settings);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -195,8 +198,9 @@ public class RootObjectMapper extends ObjectMapper {
|
|||
private volatile DynamicTemplate dynamicTemplates[];
|
||||
|
||||
RootObjectMapper(String name, boolean enabled, Dynamic dynamic, ContentPath.Type pathType, Map<String, Mapper> mappers,
|
||||
FormatDateTimeFormatter[] dynamicDateTimeFormatters, DynamicTemplate dynamicTemplates[], boolean dateDetection, boolean numericDetection) {
|
||||
super(name, name, enabled, Nested.NO, dynamic, pathType, mappers);
|
||||
FormatDateTimeFormatter[] dynamicDateTimeFormatters, DynamicTemplate dynamicTemplates[], boolean dateDetection, boolean numericDetection,
|
||||
@Nullable @IndexSettings Settings settings) {
|
||||
super(name, name, enabled, Nested.NO, dynamic, pathType, mappers, settings);
|
||||
this.dynamicTemplates = dynamicTemplates;
|
||||
this.dynamicDateTimeFormatters = dynamicDateTimeFormatters;
|
||||
this.dateDetection = dateDetection;
|
||||
|
|
|
@ -0,0 +1,132 @@
|
|||
package org.elasticsearch.benchmark.mapping;
|
||||
|
||||
import org.elasticsearch.action.bulk.BulkRequestBuilder;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.jna.Natives;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.node.Node;
|
||||
import org.elasticsearch.transport.TransportModule;
|
||||
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
|
||||
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ManyMappingsBenchmark {
|
||||
|
||||
private static final String MAPPING = "{\n" +
|
||||
" \"dynamic_templates\": [\n" +
|
||||
" {\n" +
|
||||
" \"t1\": {\n" +
|
||||
" \"mapping\": {\n" +
|
||||
" \"store\": false,\n" +
|
||||
" \"norms\": {\n" +
|
||||
" \"enabled\": false\n" +
|
||||
" },\n" +
|
||||
" \"type\": \"string\"\n" +
|
||||
" },\n" +
|
||||
" \"match\": \"*_ss\"\n" +
|
||||
" }\n" +
|
||||
" },\n" +
|
||||
" {\n" +
|
||||
" \"t2\": {\n" +
|
||||
" \"mapping\": {\n" +
|
||||
" \"store\": false,\n" +
|
||||
" \"type\": \"date\"\n" +
|
||||
" },\n" +
|
||||
" \"match\": \"*_dt\"\n" +
|
||||
" }\n" +
|
||||
" },\n" +
|
||||
" {\n" +
|
||||
" \"t3\": {\n" +
|
||||
" \"mapping\": {\n" +
|
||||
" \"store\": false,\n" +
|
||||
" \"type\": \"integer\"\n" +
|
||||
" },\n" +
|
||||
" \"match\": \"*_i\"\n" +
|
||||
" }\n" +
|
||||
" }\n" +
|
||||
" ],\n" +
|
||||
" \"_source\": {\n" +
|
||||
" \"enabled\": false\n" +
|
||||
" },\n" +
|
||||
" \"properties\": {}\n" +
|
||||
" }";
|
||||
|
||||
private static final String INDEX_NAME = "index";
|
||||
private static final String TYPE_NAME = "type";
|
||||
private static final int FIELD_COUNT = 100000;
|
||||
private static final int DOC_COUNT = 10000000;
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
System.setProperty("es.logger.prefix", "");
|
||||
Natives.tryMlockall();
|
||||
Settings settings = settingsBuilder()
|
||||
.put("index.refresh_interval", "-1")
|
||||
.put("gateway.type", "local")
|
||||
.put(SETTING_NUMBER_OF_SHARDS, 5)
|
||||
.put(SETTING_NUMBER_OF_REPLICAS, 0)
|
||||
.put(TransportModule.TRANSPORT_TYPE_KEY, "local")
|
||||
.build();
|
||||
|
||||
String clusterName = ManyMappingsBenchmark.class.getSimpleName();
|
||||
Node node = nodeBuilder().clusterName(clusterName)
|
||||
.settings(settingsBuilder().put(settings))
|
||||
.node();
|
||||
|
||||
Client client = node.client();
|
||||
|
||||
client.admin().indices().prepareDelete(INDEX_NAME)
|
||||
.setIndicesOptions(IndicesOptions.lenientExpandOpen())
|
||||
.get();
|
||||
client.admin().indices().prepareCreate(INDEX_NAME)
|
||||
.addMapping(TYPE_NAME, MAPPING)
|
||||
.get();
|
||||
|
||||
BulkRequestBuilder builder = client.prepareBulk();
|
||||
int fieldCount = 0;
|
||||
long time = System.currentTimeMillis();
|
||||
final int PRINT = 1000;
|
||||
for (int i = 0; i < DOC_COUNT; i++) {
|
||||
XContentBuilder sourceBuilder = jsonBuilder().startObject();
|
||||
sourceBuilder.field(++fieldCount + "_ss", "xyz");
|
||||
sourceBuilder.field(++fieldCount + "_dt", System.currentTimeMillis());
|
||||
sourceBuilder.field(++fieldCount + "_i", i % 100);
|
||||
sourceBuilder.endObject();
|
||||
|
||||
if (fieldCount >= FIELD_COUNT) {
|
||||
fieldCount = 0;
|
||||
System.out.println("dynamic fields rolled up");
|
||||
}
|
||||
|
||||
builder.add(
|
||||
client.prepareIndex(INDEX_NAME, TYPE_NAME, String.valueOf(i))
|
||||
.setSource(sourceBuilder)
|
||||
);
|
||||
|
||||
if (builder.numberOfActions() >= 1000) {
|
||||
builder.get();
|
||||
builder = client.prepareBulk();
|
||||
}
|
||||
|
||||
if (i % PRINT == 0) {
|
||||
long took = System.currentTimeMillis() - time;
|
||||
time = System.currentTimeMillis();
|
||||
System.out.println("Indexed " + i + " docs, in " + TimeValue.timeValueMillis(took));
|
||||
}
|
||||
}
|
||||
if (builder.numberOfActions() > 0) {
|
||||
builder.get();
|
||||
}
|
||||
|
||||
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,131 @@
|
|||
/*
|
||||
* 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.common.collect;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
import org.elasticsearch.ElasticsearchIllegalStateException;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.test.ElasticsearchTestCase;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import static org.hamcrest.Matchers.*;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class UpdateInPlaceMapTests extends ElasticsearchTestCase {
|
||||
|
||||
@Test
|
||||
public void testConcurrentMutator() {
|
||||
UpdateInPlaceMap<String, String> map = UpdateInPlaceMap.of(randomIntBetween(0, 500));
|
||||
UpdateInPlaceMap<String, String>.Mutator mutator = map.mutator();
|
||||
try {
|
||||
map.mutator();
|
||||
fail("should fail on concurrent mutator");
|
||||
} catch (ElasticsearchIllegalStateException e) {
|
||||
// all is well!
|
||||
}
|
||||
mutator.close();
|
||||
// now this should work well!
|
||||
map.mutator();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testImmutableMapSwitchToCHM() {
|
||||
int switchSize = randomIntBetween(1, 500);
|
||||
UpdateInPlaceMap<String, String> map = UpdateInPlaceMap.of(switchSize);
|
||||
int i;
|
||||
for (i = 0; i < switchSize; i++) {
|
||||
UpdateInPlaceMap<String, String>.Mutator mutator = map.mutator();
|
||||
String key = "key" + i;
|
||||
String value = "value" + i;
|
||||
mutator.put(key, value);
|
||||
assertThat(mutator.get(key), equalTo(value));
|
||||
assertThat(map.get(key), nullValue());
|
||||
mutator.close();
|
||||
assertThat(map.get(key), equalTo(value));
|
||||
}
|
||||
int countAfter = switchSize + randomIntBetween(0, 100);
|
||||
for (; i < countAfter; i++) {
|
||||
UpdateInPlaceMap<String, String>.Mutator mutator = map.mutator();
|
||||
String key = "key" + i;
|
||||
String value = "value" + i;
|
||||
mutator.put(key, value);
|
||||
assertThat(mutator.get(key), equalTo(value));
|
||||
assertThat(map.get(key), equalTo(value));
|
||||
mutator.close();
|
||||
assertThat(map.get(key), equalTo(value));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInitializeWithCHM() {
|
||||
UpdateInPlaceMap<String, String> map = UpdateInPlaceMap.of(0);
|
||||
UpdateInPlaceMap<String, String>.Mutator mutator = map.mutator();
|
||||
mutator.put("key1", "value1");
|
||||
assertThat(mutator.get("key1"), equalTo("value1"));
|
||||
mutator.put("key2", "value2");
|
||||
assertThat(mutator.get("key2"), equalTo("value2"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConcurrentAccess() throws Exception {
|
||||
final int numberOfThreads = scaledRandomIntBetween(1, 10);
|
||||
final int switchSize = randomIntBetween(1, 500);
|
||||
final CountDownLatch numberOfMutations = new CountDownLatch(scaledRandomIntBetween(300, 1000));
|
||||
|
||||
final UpdateInPlaceMap<String, String> map = UpdateInPlaceMap.of(switchSize);
|
||||
final ConcurrentMap<String, String> verifier = ConcurrentCollections.newConcurrentMap();
|
||||
|
||||
Thread[] threads = new Thread[numberOfThreads];
|
||||
for (int i = 0; i < numberOfThreads; i++) {
|
||||
threads[i] = new Thread(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
while (numberOfMutations.getCount() > 0) {
|
||||
try {
|
||||
UpdateInPlaceMap<String, String>.Mutator mutator = map.mutator();
|
||||
String str = Strings.randomBase64UUID();
|
||||
mutator.put(str, str);
|
||||
verifier.put(str, str);
|
||||
mutator.close();
|
||||
numberOfMutations.countDown();
|
||||
} catch (ElasticsearchIllegalStateException e) {
|
||||
// ok, double mutating, continue
|
||||
}
|
||||
}
|
||||
}
|
||||
}, getClass().getName() + "concurrent_access_i");
|
||||
threads[i].setDaemon(true);
|
||||
}
|
||||
|
||||
for (Thread thread : threads) {
|
||||
thread.start();
|
||||
}
|
||||
|
||||
numberOfMutations.await();
|
||||
|
||||
// verify the 2 maps are the same
|
||||
assertThat(Iterables.toArray(map.values(), String.class), arrayContainingInAnyOrder(Iterables.toArray(verifier.values(), String.class)));
|
||||
}
|
||||
}
|
|
@ -65,6 +65,7 @@ import org.elasticsearch.index.cache.filter.FilterCacheModule;
|
|||
import org.elasticsearch.index.cache.filter.none.NoneFilterCache;
|
||||
import org.elasticsearch.index.cache.filter.weighted.WeightedFilterCache;
|
||||
import org.elasticsearch.index.engine.IndexEngineModule;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.node.Node;
|
||||
import org.elasticsearch.node.internal.InternalNode;
|
||||
import org.elasticsearch.plugins.PluginsService;
|
||||
|
@ -374,6 +375,9 @@ public final class InternalTestCluster extends TestCluster {
|
|||
if (random.nextBoolean()) {
|
||||
builder.put(MappingUpdatedAction.INDICES_MAPPING_ADDITIONAL_MAPPING_CHANGE_TIME, RandomInts.randomIntBetween(random, 0, 500) /*milliseconds*/);
|
||||
}
|
||||
if (random.nextBoolean()) {
|
||||
builder.put(MapperService.DEFAULT_FIELD_MAPPERS_COLLECTION_SWITCH, RandomInts.randomIntBetween(random, 0, 5));
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue