diff --git a/pom.xml b/pom.xml index f5b711ce9f82a..839535430c9ae 100644 --- a/pom.xml +++ b/pom.xml @@ -260,6 +260,12 @@ compile + + org.apache.commons + commons-lang3 + 3.3.2 + + commons-cli commons-cli @@ -691,6 +697,7 @@ com.ning:compress-lzf com.github.spullara.mustache.java:compiler com.tdunning:t-digest + org.apache.commons:commons-lang3 commons-cli:commons-cli @@ -731,6 +738,10 @@ com.tdunning.math.stats org.elasticsearch.common.stats + + org.apache.commons.lang + org.elasticsearch.common.lang + org.apache.commons.cli org.elasticsearch.common.cli.commons diff --git a/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java b/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java index cc974535ad2fe..4771d4879ee98 100644 --- a/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java +++ b/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/TransportGetFieldMappingsIndexAction.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.mapper.DocumentFieldMappers; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.service.IndexService; @@ -48,7 +49,8 @@ import java.io.IOException; import java.util.Collection; -import java.util.List; +import java.util.Iterator; +import java.util.LinkedList; /** * Transport action used to retrieve the mappings related to fields that belong to a specific index @@ -175,47 +177,41 @@ public Boolean paramAsBooleanOptional(String key, Boolean defaultValue) { private ImmutableMap findFieldMappingsByType(DocumentMapper documentMapper, GetFieldMappingsIndexRequest request) throws ElasticsearchException { MapBuilder fieldMappings = new MapBuilder<>(); - final List allFieldMappers = documentMapper.mappers().mappers(); + final DocumentFieldMappers allFieldMappers = documentMapper.mappers(); for (String field : request.fields()) { if (Regex.isMatchAllPattern(field)) { - for (FieldMapper fieldMapper : allFieldMappers) { + for (FieldMapper fieldMapper : allFieldMappers) { addFieldMapper(fieldMapper.names().fullName(), fieldMapper, fieldMappings, request.includeDefaults()); } } else if (Regex.isSimpleMatchPattern(field)) { // go through the field mappers 3 times, to make sure we give preference to the resolve order: full name, index name, name. // also make sure we only store each mapper once. - boolean[] resolved = new boolean[allFieldMappers.size()]; - for (int i = 0; i < allFieldMappers.size(); i++) { - FieldMapper fieldMapper = allFieldMappers.get(i); + Collection> remainingFieldMappers = new LinkedList<>(allFieldMappers); + for (Iterator> it = remainingFieldMappers.iterator(); it.hasNext(); ) { + final FieldMapper fieldMapper = it.next(); if (Regex.simpleMatch(field, fieldMapper.names().fullName())) { addFieldMapper(fieldMapper.names().fullName(), fieldMapper, fieldMappings, request.includeDefaults()); - resolved[i] = true; + it.remove(); } } - for (int i = 0; i < allFieldMappers.size(); i++) { - if (resolved[i]) { - continue; - } - FieldMapper fieldMapper = allFieldMappers.get(i); + for (Iterator> it = remainingFieldMappers.iterator(); it.hasNext(); ) { + final FieldMapper fieldMapper = it.next(); if (Regex.simpleMatch(field, fieldMapper.names().indexName())) { addFieldMapper(fieldMapper.names().indexName(), fieldMapper, fieldMappings, request.includeDefaults()); - resolved[i] = true; + it.remove(); } } - for (int i = 0; i < allFieldMappers.size(); i++) { - if (resolved[i]) { - continue; - } - FieldMapper fieldMapper = allFieldMappers.get(i); + for (Iterator> it = remainingFieldMappers.iterator(); it.hasNext(); ) { + final FieldMapper fieldMapper = it.next(); if (Regex.simpleMatch(field, fieldMapper.names().name())) { addFieldMapper(fieldMapper.names().name(), fieldMapper, fieldMappings, request.includeDefaults()); - resolved[i] = true; + it.remove(); } } } else { // not a pattern - FieldMapper fieldMapper = documentMapper.mappers().smartNameFieldMapper(field); + FieldMapper fieldMapper = allFieldMappers.smartNameFieldMapper(field); if (fieldMapper != null) { addFieldMapper(field, fieldMapper, fieldMappings, request.includeDefaults()); } else if (request.probablySingleFieldRequest()) { @@ -226,7 +222,7 @@ private ImmutableMap findFieldMappingsByType(Docum return fieldMappings.immutableMap(); } - private void addFieldMapper(String field, FieldMapper fieldMapper, MapBuilder fieldMappings, boolean includeDefaults) { + private void addFieldMapper(String field, FieldMapper fieldMapper, MapBuilder fieldMappings, boolean includeDefaults) { if (fieldMappings.containsKey(field)) { return; } diff --git a/src/main/java/org/elasticsearch/common/collect/CopyOnWriteHashMap.java b/src/main/java/org/elasticsearch/common/collect/CopyOnWriteHashMap.java new file mode 100644 index 0000000000000..cf5f4ac4f0aa7 --- /dev/null +++ b/src/main/java/org/elasticsearch/common/collect/CopyOnWriteHashMap.java @@ -0,0 +1,527 @@ +/* + * 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.Maps; +import com.google.common.collect.UnmodifiableIterator; +import org.apache.commons.lang3.ArrayUtils; +import org.apache.lucene.util.mutable.MutableValueInt; +import org.elasticsearch.common.Preconditions; + +import java.util.*; + +/** + * An immutable map whose writes result in a new copy of the map to be created. + * + * This is essentially a hash array mapped trie: inner nodes use a bitmap in + * order to map hashes to slots by counting ones. In case of a collision (two + * values having the same 32-bits hash), a leaf node is created which stores + * and searches for values sequentially. + * + * Reads and writes both perform in logarithmic time. Null keys and values are + * not supported. + * + * This structure might need to perform several object creations per write so + * it is better suited for work-loads that are not too write-intensive. + * + * @see the wikipedia page + */ +public final class CopyOnWriteHashMap extends AbstractMap { + + private static final int TOTAL_HASH_BITS = 32; + private static final Object[] EMPTY_ARRAY = new Object[0]; + + private static final int HASH_BITS = 6; + private static final int HASH_MASK = 0x3F; + + /** + * Return a copy of the provided map. + */ + public static CopyOnWriteHashMap copyOf(Map map) { + if (map instanceof CopyOnWriteHashMap) { + // no need to copy in that case + @SuppressWarnings("unchecked") + final CopyOnWriteHashMap cowMap = (CopyOnWriteHashMap) map; + return cowMap; + } else { + return new CopyOnWriteHashMap().copyAndPutAll(map); + } + } + + /** + * Abstraction of a node, implemented by both inner and leaf nodes. + */ + private static abstract class Node { + + /** + * Recursively get the key with the given hash. + */ + abstract V get(Object key, int hash); + + /** + * Recursively add a new entry to this node. hashBits is + * the number of bits that are still set in the hash. When this value + * reaches a number that is less than or equal to 0, a leaf + * node needs to be created since it means that a collision occurred + * on the 32 bits of the hash. + */ + abstract Node put(K key, int hash, int hashBits, V value, MutableValueInt newValue); + + /** + * Recursively remove an entry from this node. + */ + abstract Node remove(Object key, int hash); + + /** + * For the current node only, append entries that are stored on this + * node to entries and sub nodes to nodes. + */ + abstract void visit(Deque> entries, Deque> nodes); + + /** + * Whether this node stores nothing under it. + */ + abstract boolean isEmpty(); + + } + + /** + * A leaf of the tree where all hashes are equal. Values are added and retrieved in linear time. + */ + private static class Leaf extends Node { + + private final K[] keys; + private final V[] values; + + Leaf(K[] keys, V[] values) { + this.keys = keys; + this.values = values; + } + + @SuppressWarnings("unchecked") + Leaf() { + this((K[]) EMPTY_ARRAY, (V[]) EMPTY_ARRAY); + } + + @Override + boolean isEmpty() { + return keys.length == 0; + } + + @Override + void visit(Deque> entries, Deque> nodes) { + for (int i = 0; i < keys.length; ++i) { + entries.add(Maps.immutableEntry(keys[i], values[i])); + } + } + + @Override + V get(Object key, int hash) { + final int slot = ArrayUtils.indexOf(keys, key); + if (slot < 0) { + return null; + } else { + return values[slot]; + } + } + + private static T[] replace(T[] array, int index, T value) { + final T[] copy = Arrays.copyOf(array, array.length); + copy[index] = value; + return copy; + } + + @Override + Leaf put(K key, int hash, int hashBits, V value, MutableValueInt newValue) { + assert hashBits <= 0 : hashBits; + final int slot = ArrayUtils.indexOf(keys, key); + + final K[] keys2; + final V[] values2; + + if (slot < 0) { + keys2 = ArrayUtils.add(keys, key); + values2 = ArrayUtils.add(values, value); + newValue.value = 1; + } else { + keys2 = replace(keys, slot, key); + values2 = replace(values, slot, value); + } + + return new Leaf<>(keys2, values2); + } + + @Override + Leaf remove(Object key, int hash) { + final int slot = ArrayUtils.indexOf(keys, key); + if (slot < 0) { + return this; + } + final K[] keys2 = ArrayUtils.remove(keys, slot); + final V[] values2 = ArrayUtils.remove(values, slot); + return new Leaf<>(keys2, values2); + } + } + + /** + * An inner node in this trie. Inner nodes store up to 64 key-value pairs + * and use a bitmap in order to associate hashes to them. For example, if + * an inner node contains 5 values, then 5 bits will be set in the bitmap + * and the ordinal of the bit set in this bit map will be the slot number. + * + * As a consequence, the number of slots in an inner node is equal to the + * number of one bits in the bitmap. + */ + private static class InnerNode extends Node { + + private final long mask; // the bitmap + private final K[] keys; + final Object[] subNodes; // subNodes[slot] is either a value or a sub node in case of a hash collision + + InnerNode(long mask, K[] keys, Object[] subNodes) { + this.mask = mask; + this.keys = keys; + this.subNodes = subNodes; + assert consistent(); + } + + // only used in assert + private boolean consistent() { + assert Long.bitCount(mask) == keys.length; + assert Long.bitCount(mask) == subNodes.length; + for (int i = 0; i < keys.length; ++i) { + if (subNodes[i] instanceof Node) { + assert keys[i] == null; + } else { + assert keys[i] != null; + } + } + return true; + } + + @Override + boolean isEmpty() { + return mask == 0; + } + + @SuppressWarnings("unchecked") + InnerNode() { + this(0, (K[]) EMPTY_ARRAY, EMPTY_ARRAY); + } + + @Override + void visit(Deque> entries, Deque> nodes) { + for (int i = 0; i < keys.length; ++i) { + final Object sub = subNodes[i]; + if (sub instanceof Node) { + @SuppressWarnings("unchecked") + final Node subNode = (Node) sub; + assert keys[i] == null; + nodes.add(subNode); + } else { + @SuppressWarnings("unchecked") + final V value = (V) sub; + entries.add(Maps.immutableEntry(keys[i], value)); + } + } + } + + /** + * For a given hash on 6 bits, its value is set if the bitmap has a one + * at the corresponding index. + */ + private boolean exists(int hash6) { + return (mask & (1L << hash6)) != 0; + } + + /** + * For a given hash on 6 bits, the slot number is the number of one + * bits on the right of the hash6-th bit. + */ + private int slot(int hash6) { + return Long.bitCount(mask & ((1L << hash6) - 1)); + } + + @Override + V get(Object key, int hash) { + final int hash6 = hash & HASH_MASK; + if (!exists(hash6)) { + return null; + } + final int slot = slot(hash6); + final Object sub = subNodes[slot]; + assert sub != null; + if (sub instanceof Node) { + assert keys[slot] == null; // keys don't make sense on inner nodes + @SuppressWarnings("unchecked") + final Node subNode = (Node) sub; + return subNode.get(key, hash >>> HASH_BITS); + } else { + if (keys[slot].equals(key)) { + @SuppressWarnings("unchecked") + final V v = (V) sub; + return v; + } else { + // we have an entry for this hash, but the value is different + return null; + } + } + } + + private Node newSubNode(int hashBits) { + if (hashBits <= 0) { + return new Leaf(); + } else { + return new InnerNode(); + } + } + + private InnerNode putExisting(K key, int hash, int hashBits, int slot, V value, MutableValueInt newValue) { + final K[] keys2 = Arrays.copyOf(keys, keys.length); + final Object[] subNodes2 = Arrays.copyOf(subNodes, subNodes.length); + + final Object previousValue = subNodes2[slot]; + if (previousValue instanceof Node) { + // insert recursively + assert keys[slot] == null; + subNodes2[slot] = ((Node) previousValue).put(key, hash, hashBits, value, newValue); + } else if (keys[slot].equals(key)) { + // replace the existing entry + subNodes2[slot] = value; + } else { + // hash collision + final K previousKey = keys[slot]; + final int previousHash = previousKey.hashCode() >>> (TOTAL_HASH_BITS - hashBits); + Node subNode = newSubNode(hashBits); + subNode = subNode.put(previousKey, previousHash, hashBits, (V) previousValue, newValue); + subNode = subNode.put(key, hash, hashBits, value, newValue); + keys2[slot] = null; + subNodes2[slot] = subNode; + } + return new InnerNode<>(mask, keys2, subNodes2); + } + + private InnerNode putNew(K key, int hash6, int slot, V value) { + final long mask2 = mask | (1L << hash6); + final K[] keys2 = ArrayUtils.add(keys, slot, key); + final Object[] subNodes2 = ArrayUtils.add(subNodes, slot, value); + return new InnerNode<>(mask2, keys2, subNodes2); + } + + @Override + InnerNode put(K key, int hash, int hashBits, V value, MutableValueInt newValue) { + final int hash6 = hash & HASH_MASK; + final int slot = slot(hash6); + + if (exists(hash6)) { + hash >>>= HASH_BITS; + hashBits -= HASH_BITS; + return putExisting(key, hash, hashBits, slot, value, newValue); + } else { + newValue.value = 1; + return putNew(key, hash6, slot, value); + } + } + + private InnerNode removeSlot(int hash6, int slot) { + final long mask2 = mask & ~(1L << hash6); + final K[] keys2 = ArrayUtils.remove(keys, slot); + final Object[] subNodes2 = ArrayUtils.remove(subNodes, slot); + return new InnerNode<>(mask2, keys2, subNodes2); + } + + @Override + InnerNode remove(Object key, int hash) { + final int hash6 = hash & HASH_MASK; + if (!exists(hash6)) { + return this; + } + final int slot = slot(hash6); + final Object previousValue = subNodes[slot]; + if (previousValue instanceof Node) { + @SuppressWarnings("unchecked") + final Node subNode = (Node) previousValue; + final Node removed = subNode.remove(key, hash >>> HASH_BITS); + if (removed == subNode) { + // not in sub-nodes + return this; + } + if (removed.isEmpty()) { + return removeSlot(hash6, slot); + } + final K[] keys2 = Arrays.copyOf(keys, keys.length); + final Object[] subNodes2 = Arrays.copyOf(subNodes, subNodes.length); + subNodes2[slot] = removed; + return new InnerNode<>(mask, keys2, subNodes2); + } else if (keys[slot].equals(key)) { + // remove entry + return removeSlot(hash6, slot); + } else { + // hash collision, nothing to remove + return this; + } + } + + } + + private static class EntryIterator extends UnmodifiableIterator> { + + private final Deque> entries; + private final Deque> nodes; + + public EntryIterator(Node node) { + entries = new ArrayDeque<>(); + nodes = new ArrayDeque<>(); + node.visit(entries, nodes); + } + + @Override + public boolean hasNext() { + return !entries.isEmpty() || !nodes.isEmpty(); + } + + @Override + public Map.Entry next() { + while (entries.isEmpty()) { + if (nodes.isEmpty()) { + throw new NoSuchElementException(); + } + final Node nextNode = nodes.pop(); + nextNode.visit(entries, nodes); + } + return entries.pop(); + } + + } + + private final InnerNode root; + private final int size; + + /** + * Create a new empty map. + */ + public CopyOnWriteHashMap() { + this(new InnerNode(), 0); + } + + private CopyOnWriteHashMap(InnerNode root, int size) { + this.root = root; + this.size = size; + } + + @Override + public boolean containsKey(Object key) { + // works fine since null values are not supported + return get(key) != null; + } + + @Override + public V get(Object key) { + Preconditions.checkArgument(key != null, "Null keys are not supported"); + final int hash = key.hashCode(); + return root.get(key, hash); + } + + @Override + public int size() { + assert size != 0 || root.isEmpty(); + return size; + } + + /** + * Associate key with value and return a new copy + * of the hash table. The current hash table is not modified. + */ + public CopyOnWriteHashMap copyAndPut(K key, V value) { + Preconditions.checkArgument(key != null, "null keys are not supported"); + Preconditions.checkArgument(value != null, "null values are not supported"); + final int hash = key.hashCode(); + final MutableValueInt newValue = new MutableValueInt(); + final InnerNode newRoot = root.put(key, hash, TOTAL_HASH_BITS, value, newValue); + final int newSize = size + newValue.value; + return new CopyOnWriteHashMap<>(newRoot, newSize); + } + + /** + * Same as {@link #copyAndPut(Object, Object)} but for an arbitrary number of entries. + */ + public CopyOnWriteHashMap copyAndPutAll(Map other) { + return copyAndPutAll(other.entrySet()); + } + + CopyOnWriteHashMap copyAndPutAll(Collection> entries) { + CopyOnWriteHashMap result = this; + for (Map.Entry entry : entries) { + result = result.copyAndPut(entry.getKey(), entry.getValue()); + } + return result; + } + + /** + * Remove the given key from this map. The current hash table is not modified. + */ + public CopyOnWriteHashMap copyAndRemove(Object key) { + Preconditions.checkArgument(key != null, "Null keys are not supported"); + final int hash = key.hashCode(); + final InnerNode newRoot = root.remove(key, hash); + if (root == newRoot) { + return this; + } else { + return new CopyOnWriteHashMap<>(newRoot, size - 1); + } + } + + /** + * Same as {@link #copyAndRemove(Object)} but for an arbitrary number of entries. + */ + public CopyOnWriteHashMap copyAndRemoveAll(Collection keys) { + CopyOnWriteHashMap result = this; + for (Object key : keys) { + result = result.copyAndRemove(key); + } + return result; + } + + @Override + public Set> entrySet() { + return new AbstractSet>() { + + @Override + public Iterator> iterator() { + return new EntryIterator<>(root); + } + + @Override + public boolean contains(Object o) { + if (o == null || !(o instanceof Map.Entry)) { + return false; + } + Map.Entry entry = (java.util.Map.Entry) o; + return entry.getValue().equals(CopyOnWriteHashMap.this.get(entry.getKey())); + } + + @Override + public int size() { + return CopyOnWriteHashMap.this.size(); + } + }; + } + +} diff --git a/src/main/java/org/elasticsearch/common/collect/CopyOnWriteHashSet.java b/src/main/java/org/elasticsearch/common/collect/CopyOnWriteHashSet.java new file mode 100644 index 0000000000000..737d34fae7ffe --- /dev/null +++ b/src/main/java/org/elasticsearch/common/collect/CopyOnWriteHashSet.java @@ -0,0 +1,117 @@ +/* + * 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.base.Function; +import com.google.common.collect.Collections2; +import com.google.common.collect.ForwardingSet; +import com.google.common.collect.Maps; + +import java.util.Collection; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +/** + * {@link Set} implementation based on {@link CopyOnWriteHashMap}. + * Null values are not supported. + */ +public class CopyOnWriteHashSet extends ForwardingSet { + + /** + * Return a copy of the provided set. + */ + public static CopyOnWriteHashSet copyOf(Collection set) { + if (set instanceof CopyOnWriteHashSet) { + // no need to copy in that case + @SuppressWarnings("unchecked") + final CopyOnWriteHashSet cowSet = (CopyOnWriteHashSet) set; + return cowSet; + } else { + return new CopyOnWriteHashSet().copyAndAddAll(set); + } + } + + private final CopyOnWriteHashMap map; + + /** Create a new empty set. */ + public CopyOnWriteHashSet() { + this(new CopyOnWriteHashMap()); + } + + private CopyOnWriteHashSet(CopyOnWriteHashMap map) { + this.map = map; + } + + @Override + protected Set delegate() { + return map.keySet(); + } + + /** + * Copy the current set and return a copy that contains or replaces entry. + */ + public CopyOnWriteHashSet copyAndAdd(T entry) { + return new CopyOnWriteHashSet<>(map.copyAndPut(entry, true)); + } + + /** + * Copy the current set and return a copy that is the union of the current + * set and entries, potentially replacing existing entries in + * case of equality. + */ + public CopyOnWriteHashSet copyAndAddAll(Collection entries) { + final Collection> asMapEntries = Collections2.transform(entries,new Function>() { + @Override + public Entry apply(T input) { + return Maps.immutableEntry(input, true); + } + }); + CopyOnWriteHashMap updated = this.map.copyAndPutAll(asMapEntries); + return new CopyOnWriteHashSet<>(updated); + } + + /** + * Copy the current set and return a copy that removes entry + * if it exists. + */ + public CopyOnWriteHashSet copyAndRemove(Object entry) { + final CopyOnWriteHashMap updated = map.copyAndRemove(entry); + if (updated == map) { + return this; + } else { + return new CopyOnWriteHashSet<>(updated); + } + } + + /** + * Copy the current set and return a copy that is the difference of the current + * set and entries. + */ + public CopyOnWriteHashSet copyAndRemoveAll(Collection entries) { + CopyOnWriteHashMap updated = this.map.copyAndRemoveAll(entries); + if (updated == map) { + return this; + } else { + return new CopyOnWriteHashSet<>(updated); + } + } + +} diff --git a/src/main/java/org/elasticsearch/common/collect/UpdateInPlaceMap.java b/src/main/java/org/elasticsearch/common/collect/UpdateInPlaceMap.java deleted file mode 100644 index f2681148b2fdf..0000000000000 --- a/src/main/java/org/elasticsearch/common/collect/UpdateInPlaceMap.java +++ /dev/null @@ -1,177 +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.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. - *

- * 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. - *

- * This class only allows for a single concurrent mutator to execute at the same time. - */ -public final class UpdateInPlaceMap { - - final int switchSize; - final AtomicBoolean mutating = new AtomicBoolean(); - volatile ImmutableOpenMap immutableMap; - volatile ConcurrentMap 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 immutableMap = this.immutableMap; - final ConcurrentMap 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 immutableMap = this.immutableMap; - final ConcurrentMap 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 values() { - return new Iterable() { - @Override - public Iterator iterator() { - final ImmutableOpenMap immutableMap = UpdateInPlaceMap.this.immutableMap; - final ConcurrentMap 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 UpdateInPlaceMap of(int switchSize) { - return new UpdateInPlaceMap<>(switchSize); - } - - public final class Mutator implements Releasable { - - private ImmutableOpenMap.Builder 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 map) { - for (Map.Entry 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 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); - } - } -} diff --git a/src/main/java/org/elasticsearch/index/analysis/FieldNameAnalyzer.java b/src/main/java/org/elasticsearch/index/analysis/FieldNameAnalyzer.java index 525f72ca06f5e..a4149d7e196ce 100644 --- a/src/main/java/org/elasticsearch/index/analysis/FieldNameAnalyzer.java +++ b/src/main/java/org/elasticsearch/index/analysis/FieldNameAnalyzer.java @@ -21,23 +21,30 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.DelegatingAnalyzerWrapper; -import org.elasticsearch.common.collect.UpdateInPlaceMap; +import org.elasticsearch.common.collect.CopyOnWriteHashMap; + +import java.util.Collection; +import java.util.Map; /** * */ public final class FieldNameAnalyzer extends DelegatingAnalyzerWrapper { - private final UpdateInPlaceMap analyzers; + private final CopyOnWriteHashMap analyzers; private final Analyzer defaultAnalyzer; - public FieldNameAnalyzer(UpdateInPlaceMap analyzers, Analyzer defaultAnalyzer) { + public FieldNameAnalyzer(Analyzer defaultAnalyzer) { + this(new CopyOnWriteHashMap(), defaultAnalyzer); + } + + public FieldNameAnalyzer(Map analyzers, Analyzer defaultAnalyzer) { super(Analyzer.PER_FIELD_REUSE_STRATEGY); - this.analyzers = analyzers; + this.analyzers = CopyOnWriteHashMap.copyOf(analyzers); this.defaultAnalyzer = defaultAnalyzer; } - public UpdateInPlaceMap analyzers() { + public Map analyzers() { return analyzers; } @@ -57,4 +64,18 @@ private Analyzer getAnalyzer(String name) { } return defaultAnalyzer; } + + /** + * Return a new instance that contains the union of this and of the provided analyzers. + */ + public FieldNameAnalyzer copyAndAddAll(Collection> mappers) { + CopyOnWriteHashMap analyzers = this.analyzers; + for (Map.Entry entry : mappers) { + if (entry.getValue() != null) { + analyzers = analyzers.copyAndPut(entry.getKey(), entry.getValue()); + } + } + return new FieldNameAnalyzer(analyzers, defaultAnalyzer); + } + } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/index/mapper/DocumentFieldMappers.java b/src/main/java/org/elasticsearch/index/mapper/DocumentFieldMappers.java index 6e5f10854425c..a65485f54600f 100644 --- a/src/main/java/org/elasticsearch/index/mapper/DocumentFieldMappers.java +++ b/src/main/java/org/elasticsearch/index/mapper/DocumentFieldMappers.java @@ -19,74 +19,61 @@ package org.elasticsearch.index.mapper; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.UnmodifiableIterator; +import com.google.common.base.Function; +import com.google.common.collect.Collections2; +import com.google.common.collect.ForwardingSet; +import com.google.common.collect.Maps; 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.Collection; import java.util.List; +import java.util.Map; import java.util.Set; /** * */ -public class DocumentFieldMappers implements Iterable { +public final class DocumentFieldMappers extends ForwardingSet> { - private final DocumentMapper docMapper; private final FieldMappersLookup fieldMappers; private final FieldNameAnalyzer indexAnalyzer; private final FieldNameAnalyzer searchAnalyzer; private final FieldNameAnalyzer searchQuoteAnalyzer; - public DocumentFieldMappers(@Nullable @IndexSettings Settings settings, DocumentMapper docMapper) { - this.docMapper = docMapper; - this.fieldMappers = new FieldMappersLookup(settings); - this.indexAnalyzer = new FieldNameAnalyzer(UpdateInPlaceMap.of(MapperService.getFieldMappersCollectionSwitch(settings)), docMapper.indexAnalyzer()); - this.searchAnalyzer = new FieldNameAnalyzer(UpdateInPlaceMap.of(MapperService.getFieldMappersCollectionSwitch(settings)), docMapper.searchAnalyzer()); - this.searchQuoteAnalyzer = new FieldNameAnalyzer(UpdateInPlaceMap.of(MapperService.getFieldMappersCollectionSwitch(settings)), docMapper.searchQuotedAnalyzer()); + public DocumentFieldMappers(DocumentMapper docMapper) { + this(new FieldMappersLookup(), new FieldNameAnalyzer(docMapper.indexAnalyzer()), new FieldNameAnalyzer(docMapper.searchAnalyzer()), new FieldNameAnalyzer(docMapper.searchQuotedAnalyzer())); } - public void addNewMappers(List newMappers) { - fieldMappers.addNewMappers(newMappers); - - final UpdateInPlaceMap.Mutator indexAnalyzersMutator = this.indexAnalyzer.analyzers().mutator(); - final UpdateInPlaceMap.Mutator searchAnalyzersMutator = this.searchAnalyzer.analyzers().mutator(); - final UpdateInPlaceMap.Mutator searchQuoteAnalyzersMutator = this.searchQuoteAnalyzer.analyzers().mutator(); + private DocumentFieldMappers(FieldMappersLookup fieldMappers, FieldNameAnalyzer indexAnalyzer, FieldNameAnalyzer searchAnalyzer, FieldNameAnalyzer searchQuoteAnalyzer) { + this.fieldMappers = fieldMappers; + this.indexAnalyzer = indexAnalyzer; + this.searchAnalyzer = searchAnalyzer; + this.searchQuoteAnalyzer = searchQuoteAnalyzer; + } - for (FieldMapper fieldMapper : newMappers) { - if (fieldMapper.indexAnalyzer() != null) { - indexAnalyzersMutator.put(fieldMapper.names().indexName(), fieldMapper.indexAnalyzer()); + public DocumentFieldMappers copyAndAllAll(Collection> newMappers) { + FieldMappersLookup fieldMappers = this.fieldMappers.copyAndAddAll(newMappers); + FieldNameAnalyzer indexAnalyzer = this.indexAnalyzer.copyAndAddAll(Collections2.transform(newMappers, new Function, Map.Entry>() { + @Override + public Map.Entry apply(FieldMapper input) { + return Maps.immutableEntry(input.names().indexName(), input.indexAnalyzer()); } - if (fieldMapper.searchAnalyzer() != null) { - searchAnalyzersMutator.put(fieldMapper.names().indexName(), fieldMapper.searchAnalyzer()); + })); + FieldNameAnalyzer searchAnalyzer = this.searchAnalyzer.copyAndAddAll(Collections2.transform(newMappers, new Function, Map.Entry>() { + @Override + public Map.Entry apply(FieldMapper input) { + return Maps.immutableEntry(input.names().indexName(), input.searchAnalyzer()); } - if (fieldMapper.searchQuoteAnalyzer() != null) { - searchQuoteAnalyzersMutator.put(fieldMapper.names().indexName(), fieldMapper.searchQuoteAnalyzer()); + })); + FieldNameAnalyzer searchQuoteAnalyzer = this.searchQuoteAnalyzer.copyAndAddAll(Collections2.transform(newMappers, new Function, Map.Entry>() { + @Override + public Map.Entry apply(FieldMapper input) { + return Maps.immutableEntry(input.names().indexName(), input.searchQuoteAnalyzer()); } - } - - indexAnalyzersMutator.close(); - searchAnalyzersMutator.close(); - searchQuoteAnalyzersMutator.close(); - } - - @Override - public UnmodifiableIterator iterator() { - return fieldMappers.iterator(); - } - - public List mappers() { - return this.fieldMappers.mappers(); - } - - public boolean hasMapper(FieldMapper fieldMapper) { - return fieldMappers.mappers().contains(fieldMapper); + })); + return new DocumentFieldMappers(fieldMappers, indexAnalyzer, searchAnalyzer, searchQuoteAnalyzer); } public FieldMappers name(String name) { @@ -117,7 +104,7 @@ public FieldMappers smartName(String name) { return fieldMappers.smartName(name); } - public FieldMapper smartNameFieldMapper(String name) { + public FieldMapper smartNameFieldMapper(String name) { return fieldMappers.smartNameFieldMapper(name); } @@ -148,4 +135,9 @@ public Analyzer searchAnalyzer() { public Analyzer searchQuoteAnalyzer() { return this.searchQuoteAnalyzer; } + + @Override + protected Set> delegate() { + return fieldMappers; + } } diff --git a/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java b/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java index d2ddcfd32a38c..28744863c8604 100644 --- a/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java +++ b/src/main/java/org/elasticsearch/index/mapper/DocumentMapper.java @@ -280,7 +280,7 @@ protected ParseContext.InternalParseContext initialValue() { private final NamedAnalyzer searchAnalyzer; private final NamedAnalyzer searchQuoteAnalyzer; - private final DocumentFieldMappers fieldMappers; + private volatile DocumentFieldMappers fieldMappers; private volatile ImmutableMap objectMappers = ImmutableMap.of(); @@ -345,8 +345,7 @@ public DocumentMapper(String index, @Nullable Settings indexSettings, DocumentMa // now traverse and get all the statically defined ones rootObjectMapper.traverse(fieldMappersAgg); - this.fieldMappers = new DocumentFieldMappers(indexSettings, this); - this.fieldMappers.addNewMappers(fieldMappersAgg.mappers); + this.fieldMappers = new DocumentFieldMappers(this).copyAndAllAll(fieldMappersAgg.mappers); final Map objectMappers = Maps.newHashMap(); rootObjectMapper.traverse(new ObjectMapperListener() { @@ -615,9 +614,9 @@ private XContentParser transform(XContentParser parser) throws IOException { return SmileXContent.smileXContent.createParser(builder.bytes()); } - public void addFieldMappers(List fieldMappers) { + public void addFieldMappers(List> fieldMappers) { synchronized (mappersMutex) { - this.fieldMappers.addNewMappers(fieldMappers); + this.fieldMappers = this.fieldMappers.copyAndAllAll(fieldMappers); } for (FieldMapperListener listener : fieldMapperListeners) { listener.fieldMappers(fieldMappers); diff --git a/src/main/java/org/elasticsearch/index/mapper/FieldMapperListener.java b/src/main/java/org/elasticsearch/index/mapper/FieldMapperListener.java index d69d6c1c00e2e..3b7da0eb8c6f8 100644 --- a/src/main/java/org/elasticsearch/index/mapper/FieldMapperListener.java +++ b/src/main/java/org/elasticsearch/index/mapper/FieldMapperListener.java @@ -28,18 +28,18 @@ public abstract class FieldMapperListener { public static class Aggregator extends FieldMapperListener { - public final List mappers = new ArrayList<>(); + public final List> mappers = new ArrayList<>(); @Override - public void fieldMapper(FieldMapper fieldMapper) { + public void fieldMapper(FieldMapper fieldMapper) { mappers.add(fieldMapper); } } - public abstract void fieldMapper(FieldMapper fieldMapper); + public abstract void fieldMapper(FieldMapper fieldMapper); - public void fieldMappers(List fieldMappers) { - for (FieldMapper mapper : fieldMappers) { + public void fieldMappers(List> fieldMappers) { + for (FieldMapper mapper : fieldMappers) { fieldMapper(mapper); } } diff --git a/src/main/java/org/elasticsearch/index/mapper/FieldMappersLookup.java b/src/main/java/org/elasticsearch/index/mapper/FieldMappersLookup.java index 61adc4f67acf6..76aab32635dfc 100644 --- a/src/main/java/org/elasticsearch/index/mapper/FieldMappersLookup.java +++ b/src/main/java/org/elasticsearch/index/mapper/FieldMappersLookup.java @@ -19,173 +19,136 @@ package org.elasticsearch.index.mapper; -import com.google.common.collect.*; +import com.google.common.collect.ForwardingSet; +import com.google.common.collect.Lists; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.collect.UpdateInPlaceMap; +import org.elasticsearch.common.collect.CopyOnWriteHashMap; +import org.elasticsearch.common.collect.CopyOnWriteHashSet; import org.elasticsearch.common.regex.Regex; -import org.elasticsearch.common.settings.Settings; -import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Set; /** * A class that holds a map of field mappers from name, index name, and full name. */ -public class FieldMappersLookup implements Iterable { - - private volatile FieldMapper[] mappers; - private volatile List mappersAsList; - private final UpdateInPlaceMap name; - private final UpdateInPlaceMap indexName; - private final UpdateInPlaceMap fullName; - - 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)); +public class FieldMappersLookup extends ForwardingSet> { + + private static CopyOnWriteHashMap add(CopyOnWriteHashMap map, String key, FieldMapper mapper) { + FieldMappers mappers = map.get(key); + if (mappers == null) { + mappers = new FieldMappers(mapper); + } else { + mappers = mappers.concat(mapper); + } + return map.copyAndPut(key, mappers); } - /** - * Adds a new set of mappers. - */ - public void addNewMappers(List newMappers) { - final UpdateInPlaceMap.Mutator mutatorName = name.mutator(); - final UpdateInPlaceMap.Mutator mutatorIndexName = indexName.mutator(); - final UpdateInPlaceMap.Mutator mutatorFullName = fullName.mutator(); - - for (FieldMapper fieldMapper : newMappers) { - FieldMappers mappers = mutatorName.get(fieldMapper.names().name()); - if (mappers == null) { - mappers = new FieldMappers(fieldMapper); - } else { - mappers = mappers.concat(fieldMapper); - } - mutatorName.put(fieldMapper.names().name(), mappers); - - mappers = mutatorIndexName.get(fieldMapper.names().indexName()); - if (mappers == null) { - mappers = new FieldMappers(fieldMapper); - } else { - mappers = mappers.concat(fieldMapper); - } - mutatorIndexName.put(fieldMapper.names().indexName(), mappers); - - mappers = mutatorFullName.get(fieldMapper.names().fullName()); - if (mappers == null) { - mappers = new FieldMappers(fieldMapper); - } else { - mappers = mappers.concat(fieldMapper); - } - mutatorFullName.put(fieldMapper.names().fullName(), mappers); + private static CopyOnWriteHashMap remove(CopyOnWriteHashMap map, String key, FieldMapper mapper) { + FieldMappers mappers = map.get(key); + if (mappers == null) { + return map; } - 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++); + mappers = mappers.remove(mapper); + if (mappers.isEmpty()) { + return map.copyAndRemove(key); + } else { + return map.copyAndPut(key, mappers); } - this.mappers = tempMappers; - this.mappersAsList = Arrays.asList(this.mappers); - - mutatorName.close(); - mutatorIndexName.close(); - mutatorFullName.close(); } - /** - * Removes the set of mappers. - */ - public void removeMappers(Iterable mappersToRemove) { - List tempMappers = Lists.newArrayList(this.mappers); - final UpdateInPlaceMap.Mutator mutatorName = name.mutator(); - final UpdateInPlaceMap.Mutator mutatorIndexName = indexName.mutator(); - final UpdateInPlaceMap.Mutator mutatorFullName = fullName.mutator(); - - for (FieldMapper mapper : mappersToRemove) { - FieldMappers mappers = mutatorName.get(mapper.names().name()); - if (mappers != null) { - mappers = mappers.remove(mapper); - if (mappers.isEmpty()) { - mutatorName.remove(mapper.names().name()); - } else { - mutatorName.put(mapper.names().name(), mappers); - } - } + private static class MappersLookup { - mappers = mutatorIndexName.get(mapper.names().indexName()); - if (mappers != null) { - mappers = mappers.remove(mapper); - if (mappers.isEmpty()) { - mutatorIndexName.remove(mapper.names().indexName()); - } else { - mutatorIndexName.put(mapper.names().indexName(), mappers); - } + final CopyOnWriteHashMap name, indexName, fullName; + + MappersLookup(CopyOnWriteHashMap name, CopyOnWriteHashMap indexName, CopyOnWriteHashMap fullName) { + this.name = name; + this.indexName = indexName; + this.fullName = fullName; + } + + MappersLookup addNewMappers(Iterable> mappers) { + CopyOnWriteHashMap name = this.name; + CopyOnWriteHashMap indexName = this.indexName; + CopyOnWriteHashMap fullName = this.fullName; + for (FieldMapper mapper : mappers) { + name = add(name, mapper.names().name(), mapper); + indexName = add(indexName, mapper.names().indexName(), mapper); + fullName = add(fullName, mapper.names().fullName(), mapper); } + return new MappersLookup(name, indexName, fullName); + } - mappers = mutatorFullName.get(mapper.names().fullName()); - if (mappers != null) { - mappers = mappers.remove(mapper); - if (mappers.isEmpty()) { - mutatorFullName.remove(mapper.names().fullName()); - } else { - mutatorFullName.put(mapper.names().fullName(), mappers); + MappersLookup removeMappers(Iterable mappers) { + CopyOnWriteHashMap name = this.name; + CopyOnWriteHashMap indexName = this.indexName; + CopyOnWriteHashMap fullName = this.fullName; + for (Object o : mappers) { + if (!(o instanceof FieldMapper)) { + continue; } + FieldMapper mapper = (FieldMapper) o; + name = remove(name, mapper.names().name(), mapper); + indexName = remove(indexName, mapper.names().indexName(), mapper); + fullName = remove(fullName, mapper.names().fullName(), mapper); } - - tempMappers.remove(mapper); + return new MappersLookup(name, indexName, fullName); } + } + private final CopyOnWriteHashSet> mappers; + private final MappersLookup lookup; - this.mappers = tempMappers.toArray(new FieldMapper[tempMappers.size()]); - this.mappersAsList = Arrays.asList(this.mappers); - mutatorName.close(); - mutatorIndexName.close(); - mutatorFullName.close(); + /** Create a new empty instance. */ + public FieldMappersLookup() { + this(new CopyOnWriteHashSet>(), new MappersLookup(new CopyOnWriteHashMap(), new CopyOnWriteHashMap(), new CopyOnWriteHashMap())); } - @Override - public UnmodifiableIterator iterator() { - return Iterators.unmodifiableIterator(mappersAsList.iterator()); + private FieldMappersLookup(CopyOnWriteHashSet> mappers, MappersLookup lookup) { + this.mappers = mappers; + this.lookup = lookup; } /** - * The list of all mappers. + * Return a new instance that contains the union of this instance and the provided mappers. */ - public List mappers() { - return this.mappersAsList; + public FieldMappersLookup copyAndAddAll(Collection> newMappers) { + return new FieldMappersLookup(mappers.copyAndAddAll(newMappers), lookup.addNewMappers(newMappers)); } /** - * Is there a mapper (based on unique {@link FieldMapper} identity)? + * Return a new instance that contains this instance minus the provided mappers. */ - public boolean hasMapper(FieldMapper fieldMapper) { - return mappersAsList.contains(fieldMapper); + public FieldMappersLookup copyAndRemoveAll(Collection mappersToRemove) { + final CopyOnWriteHashSet> newMappers = mappers.copyAndRemoveAll(mappersToRemove); + if (newMappers != mappers) { + return new FieldMappersLookup(newMappers, lookup.removeMappers(mappersToRemove)); + } else { + return this; + } } /** * Returns the field mappers based on the mapper name. */ public FieldMappers name(String name) { - return this.name.get(name); + return lookup.name.get(name); } /** * Returns the field mappers based on the mapper index name. */ public FieldMappers indexName(String indexName) { - return this.indexName.get(indexName); + return lookup.indexName.get(indexName); } /** * Returns the field mappers based on the mapper full name. */ public FieldMappers fullName(String fullName) { - return this.fullName.get(fullName); + return lookup.fullName.get(fullName); } /** @@ -193,7 +156,7 @@ public FieldMappers fullName(String fullName) { */ public List simpleMatchToIndexNames(String pattern) { List fields = Lists.newArrayList(); - for (FieldMapper fieldMapper : mappers) { + for (FieldMapper fieldMapper : mappers) { if (Regex.simpleMatch(pattern, fieldMapper.names().fullName())) { fields.add(fieldMapper.names().indexName()); } else if (Regex.simpleMatch(pattern, fieldMapper.names().indexName())) { @@ -210,7 +173,7 @@ public List simpleMatchToIndexNames(String pattern) { */ public List simpleMatchToFullName(String pattern) { List fields = Lists.newArrayList(); - for (FieldMapper fieldMapper : mappers) { + for (FieldMapper fieldMapper : mappers) { if (Regex.simpleMatch(pattern, fieldMapper.names().fullName())) { fields.add(fieldMapper.names().fullName()); } else if (Regex.simpleMatch(pattern, fieldMapper.names().indexName())) { @@ -244,11 +207,16 @@ public FieldMappers smartName(String name) { * by {@link #name(String)} and return the first mapper for it (see {@link org.elasticsearch.index.mapper.FieldMappers#mapper()}). */ @Nullable - public FieldMapper smartNameFieldMapper(String name) { + public FieldMapper smartNameFieldMapper(String name) { FieldMappers fieldMappers = smartName(name); if (fieldMappers == null) { return null; } return fieldMappers.mapper(); } + + @Override + protected Set> delegate() { + return mappers; + } } diff --git a/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 2869a4ffabd60..dd7a083da0e2c 100755 --- a/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -89,13 +89,6 @@ public class MapperService extends AbstractIndexComponent { 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; @@ -113,7 +106,7 @@ public static int getFieldMappersCollectionSwitch(@Nullable Settings settings) { private final Object typeMutex = new Object(); private final Object mappersMutex = new Object(); - private final FieldMappersLookup fieldMappers; + private volatile FieldMappersLookup fieldMappers; private volatile ImmutableOpenMap fullPathObjectMappers = ImmutableOpenMap.of(); private boolean hasNested = false; // updated dynamically to true when a nested object is added @@ -136,7 +129,7 @@ public MapperService(Index index, @IndexSettings Settings indexSettings, Environ super(index, indexSettings); this.analysisService = analysisService; this.fieldDataService = fieldDataService; - this.fieldMappers = new FieldMappersLookup(indexSettings); + this.fieldMappers = new FieldMappersLookup(); this.documentParser = new DocumentMapperParser(index, indexSettings, analysisService, postingsFormatService, docValuesFormatService, similarityLookupService, scriptService); this.searchAnalyzer = new SmartIndexNameSearchAnalyzer(analysisService.defaultSearchAnalyzer()); this.searchQuoteAnalyzer = new SmartIndexNameSearchQuoteAnalyzer(analysisService.defaultSearchQuoteAnalyzer()); @@ -392,9 +385,9 @@ private void addObjectMappers(ObjectMapper[] objectMappers) { } } - private void addFieldMappers(List fieldMappers) { + private void addFieldMappers(List> fieldMappers) { synchronized (mappersMutex) { - this.fieldMappers.addNewMappers(fieldMappers); + this.fieldMappers = this.fieldMappers.copyAndAddAll(fieldMappers); } } @@ -415,7 +408,7 @@ public void remove(String type) { private void removeObjectAndFieldMappers(DocumentMapper docMapper) { synchronized (mappersMutex) { - fieldMappers.removeMappers(docMapper.mappers()); + fieldMappers = fieldMappers.copyAndRemoveAll(docMapper.mappers()); ImmutableOpenMap.Builder fullPathObjectMappers = ImmutableOpenMap.builder(this.fullPathObjectMappers); for (ObjectMapper mapper : docMapper.objectMappers().values()) { @@ -1130,12 +1123,12 @@ protected Analyzer getWrappedAnalyzer(String fieldName) { class InternalFieldMapperListener extends FieldMapperListener { @Override - public void fieldMapper(FieldMapper fieldMapper) { - addFieldMappers(Arrays.asList(fieldMapper)); + public void fieldMapper(FieldMapper fieldMapper) { + addFieldMappers(Collections.>singletonList(fieldMapper)); } @Override - public void fieldMappers(List fieldMappers) { + public void fieldMappers(List> fieldMappers) { addFieldMappers(fieldMappers); } } diff --git a/src/main/java/org/elasticsearch/index/mapper/core/AbstractFieldMapper.java b/src/main/java/org/elasticsearch/index/mapper/core/AbstractFieldMapper.java index 823d759cb7c37..e7ba70545826a 100644 --- a/src/main/java/org/elasticsearch/index/mapper/core/AbstractFieldMapper.java +++ b/src/main/java/org/elasticsearch/index/mapper/core/AbstractFieldMapper.java @@ -936,7 +936,7 @@ public void parse(AbstractFieldMapper mainField, ParseContext context) throws IO public void merge(Mapper mergeWith, MergeContext mergeContext) throws MergeMappingException { AbstractFieldMapper mergeWithMultiField = (AbstractFieldMapper) mergeWith; - List newFieldMappers = null; + List> newFieldMappers = null; ImmutableOpenMap.Builder newMappersBuilder = null; for (ObjectCursor cursor : mergeWithMultiField.multiFields.mappers.values()) { diff --git a/src/main/java/org/elasticsearch/index/mapper/object/ObjectMapper.java b/src/main/java/org/elasticsearch/index/mapper/object/ObjectMapper.java index 19da8f1122d93..9a631a05c4367 100644 --- a/src/main/java/org/elasticsearch/index/mapper/object/ObjectMapper.java +++ b/src/main/java/org/elasticsearch/index/mapper/object/ObjectMapper.java @@ -30,7 +30,7 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.collect.UpdateInPlaceMap; +import org.elasticsearch.common.collect.CopyOnWriteHashMap; import org.elasticsearch.common.joda.FormatDateTimeFormatter; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContent; @@ -175,7 +175,7 @@ public Y build(BuilderContext context) { } protected ObjectMapper createMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map mappers, @Nullable @IndexSettings Settings settings) { - return new ObjectMapper(name, fullPath, enabled, nested, dynamic, pathType, mappers, settings); + return new ObjectMapper(name, fullPath, enabled, nested, dynamic, pathType, mappers); } } @@ -318,22 +318,21 @@ protected Builder createBuilder(String name) { private Boolean includeInAll; - private final UpdateInPlaceMap mappers; + private volatile CopyOnWriteHashMap mappers; private final Object mutex = new Object(); - ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map mappers, @Nullable @IndexSettings Settings settings) { + ObjectMapper(String name, String fullPath, boolean enabled, Nested nested, Dynamic dynamic, ContentPath.Type pathType, Map mappers) { 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) { - UpdateInPlaceMap.Mutator mappersMutator = this.mappers.mutator(); - mappersMutator.putAll(mappers); - mappersMutator.close(); + if (mappers == null) { + this.mappers = new CopyOnWriteHashMap<>(); + } else { + this.mappers = CopyOnWriteHashMap.copyOf(mappers); } this.nestedTypePathAsString = "__" + fullPath; this.nestedTypePathAsBytes = new BytesRef(nestedTypePathAsString); @@ -396,9 +395,7 @@ public ObjectMapper putMapper(Mapper mapper) { ((AllFieldMapper.IncludeInAll) mapper).includeInAllIfNotSet(includeInAll); } synchronized (mutex) { - UpdateInPlaceMap.Mutator mappingMutator = this.mappers.mutator(); - mappingMutator.put(mapper.name(), mapper); - mappingMutator.close(); + mappers = mappers.copyAndPut(mapper.name(), mapper); } return this; } diff --git a/src/main/java/org/elasticsearch/index/mapper/object/RootObjectMapper.java b/src/main/java/org/elasticsearch/index/mapper/object/RootObjectMapper.java index d895c628aeeb7..08938c72c3196 100644 --- a/src/main/java/org/elasticsearch/index/mapper/object/RootObjectMapper.java +++ b/src/main/java/org/elasticsearch/index/mapper/object/RootObjectMapper.java @@ -113,7 +113,7 @@ protected ObjectMapper createMapper(String name, String fullPath, boolean enable return new RootObjectMapper(name, enabled, dynamic, pathType, mappers, dates, dynamicTemplates.toArray(new DynamicTemplate[dynamicTemplates.size()]), - dateDetection, numericDetection, settings); + dateDetection, numericDetection); } } @@ -198,9 +198,8 @@ protected boolean processField(ObjectMapper.Builder builder, String fieldName, O private volatile DynamicTemplate dynamicTemplates[]; RootObjectMapper(String name, boolean enabled, Dynamic dynamic, ContentPath.Type pathType, Map mappers, - FormatDateTimeFormatter[] dynamicDateTimeFormatters, DynamicTemplate dynamicTemplates[], boolean dateDetection, boolean numericDetection, - @Nullable @IndexSettings Settings settings) { - super(name, name, enabled, Nested.NO, dynamic, pathType, mappers, settings); + FormatDateTimeFormatter[] dynamicDateTimeFormatters, DynamicTemplate dynamicTemplates[], boolean dateDetection, boolean numericDetection) { + super(name, name, enabled, Nested.NO, dynamic, pathType, mappers); this.dynamicTemplates = dynamicTemplates; this.dynamicDateTimeFormatters = dynamicDateTimeFormatters; this.dateDetection = dateDetection; diff --git a/src/main/java/org/elasticsearch/search/SearchService.java b/src/main/java/org/elasticsearch/search/SearchService.java index e72bb0e7a9e88..a6e229ac175d8 100644 --- a/src/main/java/org/elasticsearch/search/SearchService.java +++ b/src/main/java/org/elasticsearch/search/SearchService.java @@ -764,7 +764,7 @@ public TerminationHandle warmNewReaders(final IndexShard indexShard, IndexMetaDa final MapperService mapperService = indexShard.mapperService(); final ObjectSet warmUp = new ObjectOpenHashSet<>(); for (DocumentMapper docMapper : mapperService.docMappers(false)) { - for (FieldMapper fieldMapper : docMapper.mappers().mappers()) { + for (FieldMapper fieldMapper : docMapper.mappers()) { final String indexName = fieldMapper.names().indexName(); if (fieldMapper.fieldType().indexed() && !fieldMapper.fieldType().omitNorms() && fieldMapper.normsLoading(defaultLoading) == Loading.EAGER) { warmUp.add(indexName); @@ -820,7 +820,7 @@ public TerminationHandle warmNewReaders(final IndexShard indexShard, IndexMetaDa final MapperService mapperService = indexShard.mapperService(); final Map> warmUp = new HashMap<>(); for (DocumentMapper docMapper : mapperService.docMappers(false)) { - for (FieldMapper fieldMapper : docMapper.mappers().mappers()) { + for (FieldMapper fieldMapper : docMapper.mappers()) { final FieldDataType fieldDataType = fieldMapper.fieldDataType(); if (fieldDataType == null) { continue; @@ -874,7 +874,7 @@ public TerminationHandle warmTopReader(final IndexShard indexShard, IndexMetaDat final MapperService mapperService = indexShard.mapperService(); final Map> warmUpGlobalOrdinals = new HashMap<>(); for (DocumentMapper docMapper : mapperService.docMappers(false)) { - for (FieldMapper fieldMapper : docMapper.mappers().mappers()) { + for (FieldMapper fieldMapper : docMapper.mappers()) { final FieldDataType fieldDataType = fieldMapper.fieldDataType(); if (fieldDataType == null) { continue; diff --git a/src/test/java/org/elasticsearch/common/collect/CopyOnWriteHashMapTests.java b/src/test/java/org/elasticsearch/common/collect/CopyOnWriteHashMapTests.java new file mode 100644 index 0000000000000..c0e900b69d4f8 --- /dev/null +++ b/src/test/java/org/elasticsearch/common/collect/CopyOnWriteHashMapTests.java @@ -0,0 +1,151 @@ +/* + * 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.ant.tasks.junit4.dependencies.com.google.common.collect.ImmutableMap; +import org.elasticsearch.ElasticsearchIllegalArgumentException; +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.util.HashMap; +import java.util.Map; + +public class CopyOnWriteHashMapTests extends ElasticsearchTestCase { + + private static class O { + + private final int value, hashCode; + + O(int value, int hashCode) { + super(); + this.value = value; + this.hashCode = hashCode; + } + + @Override + public int hashCode() { + return hashCode; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || !(obj instanceof O)) { + return false; + } + return value == ((O) obj).value; + } + } + + public void testDuel() { + final int iters = scaledRandomIntBetween(2, 5); + for (int iter = 0; iter < iters; ++iter) { + final int numOps = randomInt(5000); + final int valueBits = randomIntBetween(1, 30); + final int hashBits = randomInt(valueBits); + + Map ref = new HashMap<>(); + CopyOnWriteHashMap map = new CopyOnWriteHashMap<>(); + assertEquals(ref, map); + final int hashBase = randomInt(); + for (int i = 0; i < numOps; ++i) { + final int v = randomInt(1 << valueBits); + final int h = (v & ((1 << hashBits) - 1)) ^ hashBase; + O key = new O(v, h); + + Map newRef = new HashMap<>(ref); + final CopyOnWriteHashMap newMap; + + if (randomBoolean()) { + // ADD + Integer value = v; + newRef.put(key, value); + newMap = map.copyAndPut(key, value); + } else { + // REMOVE + final Integer removed = newRef.remove(key); + newMap = map.copyAndRemove(key); + if (removed == null) { + assertSame(map, newMap); + } + } + + assertEquals(ref, map); // make sure that the old copy has not been modified + assertEquals(newRef, newMap); + assertEquals(newMap, newRef); + + ref = newRef; + map = newMap; + } + assertEquals(ref, CopyOnWriteHashMap.copyOf(ref)); + assertEquals(ImmutableMap.of(), CopyOnWriteHashMap.copyOf(ref).copyAndRemoveAll(ref.keySet())); + } + } + + public void testCollision() { + CopyOnWriteHashMap map = new CopyOnWriteHashMap<>(); + map = map.copyAndPut(new O(3, 0), 2); + assertEquals((Integer) 2, map.get(new O(3, 0))); + assertNull(map.get(new O(5, 0))); + + map = map.copyAndPut(new O(5, 0), 5); + assertEquals((Integer) 2, map.get(new O(3, 0))); + assertEquals((Integer) 5, map.get(new O(5, 0))); + + map = map.copyAndRemove(new O(3, 0)); + assertNull(map.get(new O(3, 0))); + assertEquals((Integer) 5, map.get(new O(5, 0))); + + map = map.copyAndRemove(new O(5, 0)); + assertNull(map.get(new O(3, 0))); + assertNull(map.get(new O(5, 0))); + } + + public void testUnsupportedAPIs() { + try { + new CopyOnWriteHashMap<>().put("a", "b"); + fail(); + } catch (UnsupportedOperationException e) { + // expected + } + + try { + new CopyOnWriteHashMap<>().copyAndPut("a", "b").remove("a"); + fail(); + } catch (UnsupportedOperationException e) { + // expected + } + } + + public void testUnsupportedValues() { + try { + new CopyOnWriteHashMap<>().copyAndPut("a", null); + fail(); + } catch (ElasticsearchIllegalArgumentException e) { + // expected + } + + try { + new CopyOnWriteHashMap<>().copyAndPut(null, "b"); + fail(); + } catch (ElasticsearchIllegalArgumentException e) { + // expected + } + } + +} diff --git a/src/test/java/org/elasticsearch/common/collect/CopyOnWriteHashSetTests.java b/src/test/java/org/elasticsearch/common/collect/CopyOnWriteHashSetTests.java new file mode 100644 index 0000000000000..49bca02f6284a --- /dev/null +++ b/src/test/java/org/elasticsearch/common/collect/CopyOnWriteHashSetTests.java @@ -0,0 +1,126 @@ +/* + * 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.ImmutableSet; +import org.elasticsearch.ElasticsearchIllegalArgumentException; +import org.elasticsearch.test.ElasticsearchTestCase; + +import java.util.HashSet; +import java.util.Set; + +public class CopyOnWriteHashSetTests extends ElasticsearchTestCase { + + private static class O { + + private final int value, hashCode; + + O(int value, int hashCode) { + super(); + this.value = value; + this.hashCode = hashCode; + } + + @Override + public int hashCode() { + return hashCode; + } + + @Override + public boolean equals(Object obj) { + if (obj == null || !(obj instanceof O)) { + return false; + } + return value == ((O) obj).value; + } + } + + public void testDuel() { + final int iters = scaledRandomIntBetween(2, 5); + for (int iter = 0; iter < iters; ++iter) { + final int numOps = randomInt(5000); + final int valueBits = randomIntBetween(1, 30); + final int hashBits = randomInt(valueBits); + + Set ref = new HashSet<>(); + CopyOnWriteHashSet set = new CopyOnWriteHashSet<>(); + assertEquals(ref, set); + final int hashBase = randomInt(); + for (int i = 0; i < numOps; ++i) { + final int v = randomInt(1 << valueBits); + final int h = (v & ((1 << hashBits) - 1)) ^ hashBase; + O key = new O(v, h); + + Set newRef = new HashSet<>(ref); + final CopyOnWriteHashSet newSet; + + if (randomBoolean()) { + // ADD + newRef.add(key); + newSet = set.copyAndAdd(key); + } else { + // REMOVE + final boolean modified = newRef.remove(key); + newSet = set.copyAndRemove(key); + if (!modified) { + assertSame(set, newSet); + } + } + + assertEquals(ref, set); // make sure that the old copy has not been modified + assertEquals(newRef, newSet); + assertEquals(newSet, newRef); + assertEquals(ref.isEmpty(), set.isEmpty()); + assertEquals(newRef.isEmpty(), newSet.isEmpty()); + + ref = newRef; + set = newSet; + } + assertEquals(ref, CopyOnWriteHashSet.copyOf(ref)); + assertEquals(ImmutableSet.of(), CopyOnWriteHashSet.copyOf(ref).copyAndRemoveAll(ref)); + } + } + + public void testUnsupportedAPIs() { + try { + new CopyOnWriteHashSet<>().add("a"); + fail(); + } catch (UnsupportedOperationException e) { + // expected + } + + try { + new CopyOnWriteHashSet<>().copyAndAdd("a").remove("a"); + fail(); + } catch (UnsupportedOperationException e) { + // expected + } + } + + public void testUnsupportedValues() { + try { + new CopyOnWriteHashSet<>().copyAndAdd(null); + fail(); + } catch (ElasticsearchIllegalArgumentException e) { + // expected + } + } + +} diff --git a/src/test/java/org/elasticsearch/common/collect/UpdateInPlaceMapTests.java b/src/test/java/org/elasticsearch/common/collect/UpdateInPlaceMapTests.java deleted file mode 100644 index c3724f5d45304..0000000000000 --- a/src/test/java/org/elasticsearch/common/collect/UpdateInPlaceMapTests.java +++ /dev/null @@ -1,135 +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.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 map = UpdateInPlaceMap.of(randomIntBetween(0, 500)); - UpdateInPlaceMap.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 map = UpdateInPlaceMap.of(switchSize); - int i; - for (i = 0; i < switchSize; i++) { - UpdateInPlaceMap.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.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 map = UpdateInPlaceMap.of(0); - UpdateInPlaceMap.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 map = UpdateInPlaceMap.of(switchSize); - final ConcurrentMap 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.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(); - - for (Thread thread : threads) { - thread.join(); - } - - // verify the 2 maps are the same - assertThat(Iterables.toArray(map.values(), String.class), arrayContainingInAnyOrder(Iterables.toArray(verifier.values(), String.class))); - } -} diff --git a/src/test/java/org/elasticsearch/index/query/SimpleIndexQueryParserTests.java b/src/test/java/org/elasticsearch/index/query/SimpleIndexQueryParserTests.java index d829be259f9b8..42b421c9441a8 100644 --- a/src/test/java/org/elasticsearch/index/query/SimpleIndexQueryParserTests.java +++ b/src/test/java/org/elasticsearch/index/query/SimpleIndexQueryParserTests.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.apache.lucene.analysis.core.WhitespaceAnalyzer; import org.apache.lucene.index.*; import org.apache.lucene.index.memory.MemoryIndex; @@ -193,8 +194,9 @@ public void testQueryStringFieldsMatch() throws Exception { assertThat(parsedQuery, instanceOf(BooleanQuery.class)); BooleanQuery bQuery = (BooleanQuery) parsedQuery; assertThat(bQuery.clauses().size(), equalTo(2)); - assertThat(assertBooleanSubQuery(parsedQuery, TermQuery.class, 0).getTerm(), equalTo(new Term("name.first", "test"))); - assertThat(assertBooleanSubQuery(parsedQuery, TermQuery.class, 1).getTerm(), equalTo(new Term("name.last", "test"))); + assertEquals(Sets.newHashSet(new Term("name.first", "test"), new Term("name.last", "test")), + Sets.newHashSet(assertBooleanSubQuery(parsedQuery, TermQuery.class, 0).getTerm(), + assertBooleanSubQuery(parsedQuery, TermQuery.class, 1).getTerm())); } @Test