lucene 4: remove bloom cache
we can add bloom cache, if we need it, as a codec on the uid field we still need to rewrite the UidFilter to not use bloom, but that will be the regular one
This commit is contained in:
parent
a4d0e3a0e8
commit
386c2ebdb9
|
@ -34,7 +34,6 @@ public class ClearIndicesCacheRequest extends BroadcastOperationRequest<ClearInd
|
|||
private boolean filterCache = false;
|
||||
private boolean fieldDataCache = false;
|
||||
private boolean idCache = false;
|
||||
private boolean bloomCache = false;
|
||||
private String[] fields = null;
|
||||
|
||||
ClearIndicesCacheRequest() {
|
||||
|
@ -82,26 +81,16 @@ public class ClearIndicesCacheRequest extends BroadcastOperationRequest<ClearInd
|
|||
return this;
|
||||
}
|
||||
|
||||
public boolean bloomCache() {
|
||||
return this.bloomCache;
|
||||
}
|
||||
|
||||
public ClearIndicesCacheRequest bloomCache(boolean bloomCache) {
|
||||
this.bloomCache = bloomCache;
|
||||
return this;
|
||||
}
|
||||
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
filterCache = in.readBoolean();
|
||||
fieldDataCache = in.readBoolean();
|
||||
idCache = in.readBoolean();
|
||||
bloomCache = in.readBoolean();
|
||||
int size = in.readVInt();
|
||||
if (size > 0) {
|
||||
fields = new String[size];
|
||||
for (int i = 0; i < size; i++) {
|
||||
fields[i] = in.readUTF();
|
||||
fields[i] = in.readString();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -111,13 +100,12 @@ public class ClearIndicesCacheRequest extends BroadcastOperationRequest<ClearInd
|
|||
out.writeBoolean(filterCache);
|
||||
out.writeBoolean(fieldDataCache);
|
||||
out.writeBoolean(idCache);
|
||||
out.writeBoolean(bloomCache);
|
||||
if (fields == null) {
|
||||
out.writeVInt(0);
|
||||
} else {
|
||||
out.writeVInt(fields.length);
|
||||
for (String field : fields) {
|
||||
out.writeUTF(field);
|
||||
out.writeString(field);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -53,11 +53,6 @@ public class ClearIndicesCacheRequestBuilder extends BroadcastOperationRequestBu
|
|||
return this;
|
||||
}
|
||||
|
||||
public ClearIndicesCacheRequestBuilder setBloomCache(boolean bloomCache) {
|
||||
request.bloomCache(bloomCache);
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doExecute(ActionListener<ClearIndicesCacheResponse> listener) {
|
||||
((IndicesAdminClient) client).clearCache(request, listener);
|
||||
|
|
|
@ -33,7 +33,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
|
|||
private boolean filterCache = false;
|
||||
private boolean fieldDataCache = false;
|
||||
private boolean idCache = false;
|
||||
private boolean bloomCache = false;
|
||||
private String[] fields = null;
|
||||
|
||||
ShardClearIndicesCacheRequest() {
|
||||
|
@ -44,7 +43,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
|
|||
filterCache = request.filterCache();
|
||||
fieldDataCache = request.fieldDataCache();
|
||||
idCache = request.idCache();
|
||||
bloomCache = request.bloomCache();
|
||||
fields = request.fields();
|
||||
}
|
||||
|
||||
|
@ -60,10 +58,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
|
|||
return this.idCache;
|
||||
}
|
||||
|
||||
public boolean bloomCache() {
|
||||
return this.bloomCache;
|
||||
}
|
||||
|
||||
public String[] fields() {
|
||||
return this.fields;
|
||||
}
|
||||
|
@ -79,7 +73,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
|
|||
filterCache = in.readBoolean();
|
||||
fieldDataCache = in.readBoolean();
|
||||
idCache = in.readBoolean();
|
||||
bloomCache = in.readBoolean();
|
||||
int size = in.readVInt();
|
||||
if (size > 0) {
|
||||
fields = new String[size];
|
||||
|
@ -95,7 +88,6 @@ class ShardClearIndicesCacheRequest extends BroadcastShardOperationRequest {
|
|||
out.writeBoolean(filterCache);
|
||||
out.writeBoolean(fieldDataCache);
|
||||
out.writeBoolean(idCache);
|
||||
out.writeBoolean(bloomCache);
|
||||
if (fields == null) {
|
||||
out.writeVInt(0);
|
||||
} else {
|
||||
|
|
|
@ -138,10 +138,6 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastOperatio
|
|||
clearedAtLeastOne = true;
|
||||
service.cache().idCache().clear();
|
||||
}
|
||||
if (request.bloomCache()) {
|
||||
clearedAtLeastOne = true;
|
||||
service.cache().bloomCache().clear();
|
||||
}
|
||||
if (!clearedAtLeastOne) {
|
||||
if (request.fields() != null && request.fields().length > 0) {
|
||||
// only clear caches relating to the specified fields
|
||||
|
|
|
@ -1,172 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search 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.bloom;
|
||||
|
||||
/**
|
||||
* The following calculations are taken from:
|
||||
* http://www.cs.wisc.edu/~cao/papers/summary-cache/node8.html
|
||||
* "Bloom Filters - the math"
|
||||
* <p/>
|
||||
* This class's static methods are meant to facilitate the use of the Bloom
|
||||
* Filter class by helping to choose correct values of 'bits per element' and
|
||||
* 'number of hash functions, k'.
|
||||
*/
|
||||
class BloomCalculations {
|
||||
|
||||
private static final int minBuckets = 2;
|
||||
private static final int minK = 1;
|
||||
|
||||
private static final int EXCESS = 20;
|
||||
|
||||
/**
|
||||
* In the following table, the row 'i' shows false positive rates if i buckets
|
||||
* per element are used. Column 'j' shows false positive rates if j hash
|
||||
* functions are used. The first row is 'i=0', the first column is 'j=0'.
|
||||
* Each cell (i,j) the false positive rate determined by using i buckets per
|
||||
* element and j hash functions.
|
||||
*/
|
||||
static final double[][] probs = new double[][]{
|
||||
{1.0}, // dummy row representing 0 buckets per element
|
||||
{1.0, 1.0}, // dummy row representing 1 buckets per element
|
||||
{1.0, 0.393, 0.400},
|
||||
{1.0, 0.283, 0.237, 0.253},
|
||||
{1.0, 0.221, 0.155, 0.147, 0.160},
|
||||
{1.0, 0.181, 0.109, 0.092, 0.092, 0.101}, // 5
|
||||
{1.0, 0.154, 0.0804, 0.0609, 0.0561, 0.0578, 0.0638},
|
||||
{1.0, 0.133, 0.0618, 0.0423, 0.0359, 0.0347, 0.0364},
|
||||
{1.0, 0.118, 0.0489, 0.0306, 0.024, 0.0217, 0.0216, 0.0229},
|
||||
{1.0, 0.105, 0.0397, 0.0228, 0.0166, 0.0141, 0.0133, 0.0135, 0.0145},
|
||||
{1.0, 0.0952, 0.0329, 0.0174, 0.0118, 0.00943, 0.00844, 0.00819, 0.00846}, // 10
|
||||
{1.0, 0.0869, 0.0276, 0.0136, 0.00864, 0.0065, 0.00552, 0.00513, 0.00509},
|
||||
{1.0, 0.08, 0.0236, 0.0108, 0.00646, 0.00459, 0.00371, 0.00329, 0.00314},
|
||||
{1.0, 0.074, 0.0203, 0.00875, 0.00492, 0.00332, 0.00255, 0.00217, 0.00199, 0.00194},
|
||||
{1.0, 0.0689, 0.0177, 0.00718, 0.00381, 0.00244, 0.00179, 0.00146, 0.00129, 0.00121, 0.0012},
|
||||
{1.0, 0.0645, 0.0156, 0.00596, 0.003, 0.00183, 0.00128, 0.001, 0.000852, 0.000775, 0.000744}, // 15
|
||||
{1.0, 0.0606, 0.0138, 0.005, 0.00239, 0.00139, 0.000935, 0.000702, 0.000574, 0.000505, 0.00047, 0.000459},
|
||||
{1.0, 0.0571, 0.0123, 0.00423, 0.00193, 0.00107, 0.000692, 0.000499, 0.000394, 0.000335, 0.000302, 0.000287, 0.000284},
|
||||
{1.0, 0.054, 0.0111, 0.00362, 0.00158, 0.000839, 0.000519, 0.00036, 0.000275, 0.000226, 0.000198, 0.000183, 0.000176},
|
||||
{1.0, 0.0513, 0.00998, 0.00312, 0.0013, 0.000663, 0.000394, 0.000264, 0.000194, 0.000155, 0.000132, 0.000118, 0.000111, 0.000109},
|
||||
{1.0, 0.0488, 0.00906, 0.0027, 0.00108, 0.00053, 0.000303, 0.000196, 0.00014, 0.000108, 8.89e-05, 7.77e-05, 7.12e-05, 6.79e-05, 6.71e-05} // 20
|
||||
}; // the first column is a dummy column representing K=0.
|
||||
|
||||
/**
|
||||
* The optimal number of hashes for a given number of bits per element.
|
||||
* These values are automatically calculated from the data above.
|
||||
*/
|
||||
private static final int[] optKPerBuckets = new int[probs.length];
|
||||
|
||||
static {
|
||||
for (int i = 0; i < probs.length; i++) {
|
||||
double min = Double.MAX_VALUE;
|
||||
double[] prob = probs[i];
|
||||
for (int j = 0; j < prob.length; j++) {
|
||||
if (prob[j] < min) {
|
||||
min = prob[j];
|
||||
optKPerBuckets[i] = Math.max(minK, j);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Given the number of buckets that can be used per element, return a
|
||||
* specification that minimizes the false positive rate.
|
||||
*
|
||||
* @param bucketsPerElement The number of buckets per element for the filter.
|
||||
* @return A spec that minimizes the false positive rate.
|
||||
*/
|
||||
public static BloomSpecification computeBloomSpec(int bucketsPerElement) {
|
||||
assert bucketsPerElement >= 1;
|
||||
assert bucketsPerElement <= probs.length - 1;
|
||||
return new BloomSpecification(optKPerBuckets[bucketsPerElement], bucketsPerElement);
|
||||
}
|
||||
|
||||
/**
|
||||
* A wrapper class that holds two key parameters for a Bloom Filter: the
|
||||
* number of hash functions used, and the number of buckets per element used.
|
||||
*/
|
||||
public static class BloomSpecification {
|
||||
final int K; // number of hash functions.
|
||||
final int bucketsPerElement;
|
||||
|
||||
public BloomSpecification(int k, int bucketsPerElement) {
|
||||
K = k;
|
||||
this.bucketsPerElement = bucketsPerElement;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a maximum tolerable false positive probability, compute a Bloom
|
||||
* specification which will give less than the specified false positive rate,
|
||||
* but minimize the number of buckets per element and the number of hash
|
||||
* functions used. Because bandwidth (and therefore total bitvector size)
|
||||
* is considered more expensive than computing power, preference is given
|
||||
* to minimizing buckets per element rather than number of hash functions.
|
||||
*
|
||||
* @param maxBucketsPerElement The maximum number of buckets available for the filter.
|
||||
* @param maxFalsePosProb The maximum tolerable false positive rate.
|
||||
* @return A Bloom Specification which would result in a false positive rate
|
||||
* less than specified by the function call
|
||||
* @throws UnsupportedOperationException if a filter satisfying the parameters cannot be met
|
||||
*/
|
||||
public static BloomSpecification computeBloomSpec(int maxBucketsPerElement, double maxFalsePosProb) {
|
||||
assert maxBucketsPerElement >= 1;
|
||||
assert maxBucketsPerElement <= probs.length - 1;
|
||||
int maxK = probs[maxBucketsPerElement].length - 1;
|
||||
|
||||
// Handle the trivial cases
|
||||
if (maxFalsePosProb >= probs[minBuckets][minK]) {
|
||||
return new BloomSpecification(2, optKPerBuckets[2]);
|
||||
}
|
||||
if (maxFalsePosProb < probs[maxBucketsPerElement][maxK]) {
|
||||
throw new UnsupportedOperationException(String.format("Unable to satisfy %s with %s buckets per element",
|
||||
maxFalsePosProb, maxBucketsPerElement));
|
||||
}
|
||||
|
||||
// First find the minimal required number of buckets:
|
||||
int bucketsPerElement = 2;
|
||||
int K = optKPerBuckets[2];
|
||||
while (probs[bucketsPerElement][K] > maxFalsePosProb) {
|
||||
bucketsPerElement++;
|
||||
K = optKPerBuckets[bucketsPerElement];
|
||||
}
|
||||
// Now that the number of buckets is sufficient, see if we can relax K
|
||||
// without losing too much precision.
|
||||
while (probs[bucketsPerElement][K - 1] <= maxFalsePosProb) {
|
||||
K--;
|
||||
}
|
||||
|
||||
return new BloomSpecification(K, bucketsPerElement);
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculates the maximum number of buckets per element that this implementation
|
||||
* can support. Crucially, it will lower the bucket count if necessary to meet
|
||||
* BitSet's size restrictions.
|
||||
*/
|
||||
public static int maxBucketsPerElement(long numElements) {
|
||||
numElements = Math.max(1, numElements);
|
||||
double v = (Long.MAX_VALUE - EXCESS) / (double) numElements;
|
||||
if (v < 1.0) {
|
||||
throw new UnsupportedOperationException("Cannot compute probabilities for " + numElements + " elements.");
|
||||
}
|
||||
return Math.min(BloomCalculations.probs.length - 1, (int) v);
|
||||
}
|
||||
}
|
|
@ -1,64 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search 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.bloom;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public interface BloomFilter {
|
||||
|
||||
public static final BloomFilter NONE = new BloomFilter() {
|
||||
@Override
|
||||
public void add(byte[] key, int offset, int length) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPresent(byte[] key, int offset, int length) {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeInBytes() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
public static final BloomFilter EMPTY = new BloomFilter() {
|
||||
@Override
|
||||
public void add(byte[] key, int offset, int length) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPresent(byte[] key, int offset, int length) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeInBytes() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
|
||||
void add(byte[] key, int offset, int length);
|
||||
|
||||
boolean isPresent(byte[] key, int offset, int length);
|
||||
|
||||
long sizeInBytes();
|
||||
}
|
|
@ -1,98 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search 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.bloom;
|
||||
|
||||
import org.elasticsearch.common.UUID;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.ESLoggerFactory;
|
||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||
import org.elasticsearch.common.unit.SizeValue;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
||||
import java.io.UnsupportedEncodingException;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class BloomFilterFactory {
|
||||
|
||||
private static ESLogger logger = ESLoggerFactory.getLogger(BloomFilterFactory.class.getName());
|
||||
|
||||
private static final int EXCESS = 20;
|
||||
|
||||
/**
|
||||
* @return A BloomFilter with the lowest practical false positive probability
|
||||
* for the given number of elements.
|
||||
*/
|
||||
public static BloomFilter getFilter(long numElements, int targetBucketsPerElem) {
|
||||
int maxBucketsPerElement = Math.max(1, BloomCalculations.maxBucketsPerElement(numElements));
|
||||
int bucketsPerElement = Math.min(targetBucketsPerElem, maxBucketsPerElement);
|
||||
if (bucketsPerElement < targetBucketsPerElem) {
|
||||
logger.warn(String.format("Cannot provide an optimal BloomFilter for %d elements (%d/%d buckets per element).",
|
||||
numElements, bucketsPerElement, targetBucketsPerElem));
|
||||
}
|
||||
BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(bucketsPerElement);
|
||||
return new ObsBloomFilter(spec.K, bucketsFor(numElements, spec.bucketsPerElement));
|
||||
}
|
||||
|
||||
/**
|
||||
* @return The smallest BloomFilter that can provide the given false positive
|
||||
* probability rate for the given number of elements.
|
||||
* <p/>
|
||||
* Asserts that the given probability can be satisfied using this filter.
|
||||
*/
|
||||
public static BloomFilter getFilter(long numElements, double maxFalsePosProbability) {
|
||||
assert maxFalsePosProbability <= 1.0 : "Invalid probability";
|
||||
int bucketsPerElement = BloomCalculations.maxBucketsPerElement(numElements);
|
||||
BloomCalculations.BloomSpecification spec = BloomCalculations.computeBloomSpec(bucketsPerElement, maxFalsePosProbability);
|
||||
return new ObsBloomFilter(spec.K, bucketsFor(numElements, spec.bucketsPerElement));
|
||||
}
|
||||
|
||||
private static long bucketsFor(long numElements, int bucketsPer) {
|
||||
return numElements * bucketsPer + EXCESS;
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws UnsupportedEncodingException {
|
||||
long elements = SizeValue.parseSizeValue("100m").singles();
|
||||
BloomFilter filter = BloomFilterFactory.getFilter(elements, 15);
|
||||
System.out.println("Filter size: " + new ByteSizeValue(filter.sizeInBytes()));
|
||||
for (long i = 0; i < elements; i++) {
|
||||
byte[] utf8s = UUID.randomBase64UUID().getBytes("UTF8");
|
||||
filter.add(utf8s, 0, utf8s.length);
|
||||
}
|
||||
long falsePositives = 0;
|
||||
for (long i = 0; i < elements; i++) {
|
||||
byte[] utf8s = UUID.randomBase64UUID().getBytes("UTF8");
|
||||
if (filter.isPresent(utf8s, 0, utf8s.length)) {
|
||||
falsePositives++;
|
||||
}
|
||||
}
|
||||
System.out.println("false positives: " + falsePositives);
|
||||
|
||||
byte[] utf8s = UUID.randomBase64UUID().getBytes("UTF8");
|
||||
long time = System.currentTimeMillis();
|
||||
for (long i = 0; i < elements; i++) {
|
||||
if (filter.isPresent(utf8s, 0, utf8s.length)) {
|
||||
}
|
||||
}
|
||||
long timeSize = System.currentTimeMillis() - time;
|
||||
System.out.println("Indexed in " + new TimeValue(timeSize) + ", TPS: " + (elements / timeSize) + " per millisecond");
|
||||
}
|
||||
}
|
|
@ -1,97 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search 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.bloom;
|
||||
|
||||
import org.apache.lucene.util.OpenBitSet;
|
||||
import org.elasticsearch.common.MurmurHash;
|
||||
import org.elasticsearch.common.RamUsage;
|
||||
|
||||
public class ObsBloomFilter implements BloomFilter {
|
||||
|
||||
private final int hashCount;
|
||||
|
||||
private final OpenBitSet bitset;
|
||||
private final long size;
|
||||
|
||||
ObsBloomFilter(int hashCount, long size) {
|
||||
this.hashCount = hashCount;
|
||||
this.bitset = new OpenBitSet(size);
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
long emptyBuckets() {
|
||||
long n = 0;
|
||||
for (long i = 0; i < buckets(); i++) {
|
||||
if (!bitset.get(i)) {
|
||||
n++;
|
||||
}
|
||||
}
|
||||
return n;
|
||||
}
|
||||
|
||||
private long buckets() {
|
||||
return size;
|
||||
}
|
||||
|
||||
private long[] getHashBuckets(byte[] key, int offset, int length) {
|
||||
return getHashBuckets(key, offset, length, hashCount, buckets());
|
||||
}
|
||||
|
||||
static long[] getHashBuckets(byte[] b, int offset, int length, int hashCount, long max) {
|
||||
long[] result = new long[hashCount];
|
||||
long[] hash = MurmurHash.hash3_x64_128(b, offset, length, 0L);
|
||||
for (int i = 0; i < hashCount; ++i) {
|
||||
result[i] = Math.abs((hash[0] + (long) i * hash[1]) % max);
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void add(byte[] key, int offset, int length) {
|
||||
// inline the hash buckets so we don't have to create the int[] each time...
|
||||
long[] hash = MurmurHash.hash3_x64_128(key, offset, length, 0L);
|
||||
for (int i = 0; i < hashCount; ++i) {
|
||||
long bucketIndex = Math.abs((hash[0] + (long) i * hash[1]) % size);
|
||||
bitset.fastSet(bucketIndex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isPresent(byte[] key, int offset, int length) {
|
||||
// inline the hash buckets so we don't have to create the int[] each time...
|
||||
long[] hash = MurmurHash.hash3_x64_128(key, offset, length, 0L);
|
||||
for (int i = 0; i < hashCount; ++i) {
|
||||
long bucketIndex = Math.abs((hash[0] + (long) i * hash[1]) % size);
|
||||
if (!bitset.fastGet(bucketIndex)) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
public void clear() {
|
||||
bitset.clear(0, bitset.size());
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeInBytes() {
|
||||
return bitset.getBits().length * RamUsage.NUM_BYTES_LONG + RamUsage.NUM_BYTES_ARRAY_HEADER + RamUsage.NUM_BYTES_INT /* wlen */;
|
||||
}
|
||||
}
|
|
@ -24,7 +24,9 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
|
|||
import org.apache.lucene.analysis.tokenattributes.PayloadAttribute;
|
||||
import org.apache.lucene.document.AbstractField;
|
||||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.AtomicReaderContext;
|
||||
import org.apache.lucene.index.FieldInfo;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.elasticsearch.common.Numbers;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
|
||||
|
@ -38,25 +40,23 @@ public class UidField extends AbstractField {
|
|||
|
||||
public static class DocIdAndVersion {
|
||||
public final int docId;
|
||||
public final int docStart;
|
||||
public final long version;
|
||||
public final IndexReader reader;
|
||||
public final AtomicReaderContext reader;
|
||||
|
||||
public DocIdAndVersion(int docId, long version, IndexReader reader, int docStart) {
|
||||
public DocIdAndVersion(int docId, long version, AtomicReaderContext reader) {
|
||||
this.docId = docId;
|
||||
this.version = version;
|
||||
this.reader = reader;
|
||||
this.docStart = docStart;
|
||||
}
|
||||
}
|
||||
|
||||
// this works fine for nested docs since they don't have the payload which has the version
|
||||
// so we iterate till we find the one with the payload
|
||||
public static DocIdAndVersion loadDocIdAndVersion(IndexReader subReader, int docStart, Term term) {
|
||||
public static DocIdAndVersion loadDocIdAndVersion(AtomicReaderContext reader, Term term) {
|
||||
int docId = Lucene.NO_DOC;
|
||||
TermPositions uid = null;
|
||||
try {
|
||||
uid = subReader.termPositions(term);
|
||||
uid = reader.termPositions(term);
|
||||
if (!uid.next()) {
|
||||
return null; // no doc
|
||||
}
|
||||
|
@ -72,11 +72,11 @@ public class UidField extends AbstractField {
|
|||
continue;
|
||||
}
|
||||
byte[] payload = uid.getPayload(new byte[8], 0);
|
||||
return new DocIdAndVersion(docId, Numbers.bytesToLong(payload), subReader, docStart);
|
||||
return new DocIdAndVersion(docId, Numbers.bytesToLong(payload), reader);
|
||||
} while (uid.next());
|
||||
return new DocIdAndVersion(docId, -2, subReader, docStart);
|
||||
return new DocIdAndVersion(docId, -2, reader);
|
||||
} catch (Exception e) {
|
||||
return new DocIdAndVersion(docId, -2, subReader, docStart);
|
||||
return new DocIdAndVersion(docId, -2, reader);
|
||||
} finally {
|
||||
if (uid != null) {
|
||||
try {
|
||||
|
@ -92,7 +92,7 @@ public class UidField extends AbstractField {
|
|||
* Load the version for the uid from the reader, returning -1 if no doc exists, or -2 if
|
||||
* no version is available (for backward comp.)
|
||||
*/
|
||||
public static long loadVersion(IndexReader reader, Term term) {
|
||||
public static long loadVersion(AtomicReaderContext reader, Term term) {
|
||||
TermPositions uid = null;
|
||||
try {
|
||||
uid = reader.termPositions(term);
|
||||
|
|
|
@ -39,19 +39,17 @@ public class CacheStats implements Streamable, ToXContent {
|
|||
long filterCount;
|
||||
long fieldSize;
|
||||
long filterSize;
|
||||
long bloomSize;
|
||||
long idCacheSize;
|
||||
|
||||
public CacheStats() {
|
||||
}
|
||||
|
||||
public CacheStats(long fieldEvictions, long filterEvictions, long fieldSize, long filterSize, long filterCount, long bloomSize, long idCacheSize) {
|
||||
public CacheStats(long fieldEvictions, long filterEvictions, long fieldSize, long filterSize, long filterCount, long idCacheSize) {
|
||||
this.fieldEvictions = fieldEvictions;
|
||||
this.filterEvictions = filterEvictions;
|
||||
this.fieldSize = fieldSize;
|
||||
this.filterSize = filterSize;
|
||||
this.filterCount = filterCount;
|
||||
this.bloomSize = bloomSize;
|
||||
this.idCacheSize = idCacheSize;
|
||||
}
|
||||
|
||||
|
@ -61,7 +59,6 @@ public class CacheStats implements Streamable, ToXContent {
|
|||
this.fieldSize += stats.fieldSize;
|
||||
this.filterSize += stats.filterSize;
|
||||
this.filterCount += stats.filterCount;
|
||||
this.bloomSize += stats.bloomSize;
|
||||
this.idCacheSize += stats.idCacheSize;
|
||||
}
|
||||
|
||||
|
@ -129,22 +126,6 @@ public class CacheStats implements Streamable, ToXContent {
|
|||
return filterSize();
|
||||
}
|
||||
|
||||
public long bloomSizeInBytes() {
|
||||
return this.bloomSize;
|
||||
}
|
||||
|
||||
public long getBloomSizeInBytes() {
|
||||
return this.bloomSize;
|
||||
}
|
||||
|
||||
public ByteSizeValue bloomSize() {
|
||||
return new ByteSizeValue(bloomSize);
|
||||
}
|
||||
|
||||
public ByteSizeValue getBloomSize() {
|
||||
return bloomSize();
|
||||
}
|
||||
|
||||
public long idCacheSizeInBytes() {
|
||||
return idCacheSize;
|
||||
}
|
||||
|
@ -158,7 +139,7 @@ public class CacheStats implements Streamable, ToXContent {
|
|||
}
|
||||
|
||||
public ByteSizeValue getIdCacheSize() {
|
||||
return bloomSize();
|
||||
return idCacheSize();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -171,8 +152,6 @@ public class CacheStats implements Streamable, ToXContent {
|
|||
builder.field(Fields.FILTER_EVICTIONS, filterEvictions);
|
||||
builder.field(Fields.FILTER_SIZE, filterSize().toString());
|
||||
builder.field(Fields.FILTER_SIZE_IN_BYTES, filterSize);
|
||||
builder.field(Fields.BLOOM_SIZE, bloomSize().toString());
|
||||
builder.field(Fields.BLOOM_SIZE_IN_BYTES, bloomSize);
|
||||
builder.field(Fields.ID_CACHE_SIZE, idCacheSize().toString());
|
||||
builder.field(Fields.ID_CACHE_SIZE_IN_BYTES, idCacheSize);
|
||||
builder.endObject();
|
||||
|
@ -188,8 +167,6 @@ public class CacheStats implements Streamable, ToXContent {
|
|||
static final XContentBuilderString FILTER_COUNT = new XContentBuilderString("filter_count");
|
||||
static final XContentBuilderString FILTER_SIZE = new XContentBuilderString("filter_size");
|
||||
static final XContentBuilderString FILTER_SIZE_IN_BYTES = new XContentBuilderString("filter_size_in_bytes");
|
||||
static final XContentBuilderString BLOOM_SIZE = new XContentBuilderString("bloom_size");
|
||||
static final XContentBuilderString BLOOM_SIZE_IN_BYTES = new XContentBuilderString("bloom_size_in_bytes");
|
||||
static final XContentBuilderString ID_CACHE_SIZE = new XContentBuilderString("id_cache_size");
|
||||
static final XContentBuilderString ID_CACHE_SIZE_IN_BYTES = new XContentBuilderString("id_cache_size_in_bytes");
|
||||
}
|
||||
|
@ -207,7 +184,6 @@ public class CacheStats implements Streamable, ToXContent {
|
|||
fieldSize = in.readVLong();
|
||||
filterSize = in.readVLong();
|
||||
filterCount = in.readVLong();
|
||||
bloomSize = in.readVLong();
|
||||
idCacheSize = in.readVLong();
|
||||
}
|
||||
|
||||
|
@ -218,7 +194,6 @@ public class CacheStats implements Streamable, ToXContent {
|
|||
out.writeVLong(fieldSize);
|
||||
out.writeVLong(filterSize);
|
||||
out.writeVLong(filterCount);
|
||||
out.writeVLong(bloomSize);
|
||||
out.writeVLong(idCacheSize);
|
||||
}
|
||||
}
|
|
@ -31,7 +31,6 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.cache.bloom.BloomCache;
|
||||
import org.elasticsearch.index.cache.field.data.FieldDataCache;
|
||||
import org.elasticsearch.index.cache.filter.FilterCache;
|
||||
import org.elasticsearch.index.cache.id.IdCache;
|
||||
|
@ -51,8 +50,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
|
|||
|
||||
private final IdCache idCache;
|
||||
|
||||
private final BloomCache bloomCache;
|
||||
|
||||
private final TimeValue refreshInterval;
|
||||
|
||||
private ClusterService clusterService;
|
||||
|
@ -62,13 +59,12 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
|
|||
|
||||
@Inject
|
||||
public IndexCache(Index index, @IndexSettings Settings indexSettings, FilterCache filterCache, FieldDataCache fieldDataCache,
|
||||
QueryParserCache queryParserCache, IdCache idCache, BloomCache bloomCache) {
|
||||
QueryParserCache queryParserCache, IdCache idCache) {
|
||||
super(index, indexSettings);
|
||||
this.filterCache = filterCache;
|
||||
this.fieldDataCache = fieldDataCache;
|
||||
this.queryParserCache = queryParserCache;
|
||||
this.idCache = idCache;
|
||||
this.bloomCache = bloomCache;
|
||||
|
||||
this.refreshInterval = componentSettings.getAsTime("stats.refresh_interval", TimeValue.timeValueSeconds(1));
|
||||
|
||||
|
@ -85,7 +81,7 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
|
|||
|
||||
public synchronized void invalidateCache() {
|
||||
FilterCache.EntriesStats filterEntriesStats = filterCache.entriesStats();
|
||||
latestCacheStats = new CacheStats(fieldDataCache.evictions(), filterCache.evictions(), fieldDataCache.sizeInBytes(), filterEntriesStats.sizeInBytes, filterEntriesStats.count, bloomCache.sizeInBytes(), idCache.sizeInBytes());
|
||||
latestCacheStats = new CacheStats(fieldDataCache.evictions(), filterCache.evictions(), fieldDataCache.sizeInBytes(), filterEntriesStats.sizeInBytes, filterEntriesStats.count, idCache.sizeInBytes());
|
||||
latestCacheStatsTimestamp = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
|
@ -93,7 +89,7 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
|
|||
long timestamp = System.currentTimeMillis();
|
||||
if ((timestamp - latestCacheStatsTimestamp) > refreshInterval.millis()) {
|
||||
FilterCache.EntriesStats filterEntriesStats = filterCache.entriesStats();
|
||||
latestCacheStats = new CacheStats(fieldDataCache.evictions(), filterCache.evictions(), fieldDataCache.sizeInBytes(), filterEntriesStats.sizeInBytes, filterEntriesStats.count, bloomCache.sizeInBytes(), idCache.sizeInBytes());
|
||||
latestCacheStats = new CacheStats(fieldDataCache.evictions(), filterCache.evictions(), fieldDataCache.sizeInBytes(), filterEntriesStats.sizeInBytes, filterEntriesStats.count, idCache.sizeInBytes());
|
||||
latestCacheStatsTimestamp = timestamp;
|
||||
}
|
||||
return latestCacheStats;
|
||||
|
@ -111,10 +107,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
|
|||
return this.idCache;
|
||||
}
|
||||
|
||||
public BloomCache bloomCache() {
|
||||
return this.bloomCache;
|
||||
}
|
||||
|
||||
public QueryParserCache queryParserCache() {
|
||||
return this.queryParserCache;
|
||||
}
|
||||
|
@ -125,7 +117,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
|
|||
fieldDataCache.close();
|
||||
idCache.close();
|
||||
queryParserCache.close();
|
||||
bloomCache.close();
|
||||
if (clusterService != null) {
|
||||
clusterService.remove(this);
|
||||
}
|
||||
|
@ -135,7 +126,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
|
|||
filterCache.clear(reader);
|
||||
fieldDataCache.clear(reader);
|
||||
idCache.clear(reader);
|
||||
bloomCache.clear(reader);
|
||||
}
|
||||
|
||||
public void clear(String reason) {
|
||||
|
@ -143,7 +133,6 @@ public class IndexCache extends AbstractIndexComponent implements CloseableCompo
|
|||
fieldDataCache.clear(reason);
|
||||
idCache.clear();
|
||||
queryParserCache.clear();
|
||||
bloomCache.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.index.cache;
|
|||
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.cache.bloom.BloomCacheModule;
|
||||
import org.elasticsearch.index.cache.field.data.FieldDataCacheModule;
|
||||
import org.elasticsearch.index.cache.filter.FilterCacheModule;
|
||||
import org.elasticsearch.index.cache.id.IdCacheModule;
|
||||
|
@ -44,7 +43,6 @@ public class IndexCacheModule extends AbstractModule {
|
|||
new FieldDataCacheModule(settings).configure(binder());
|
||||
new IdCacheModule(settings).configure(binder());
|
||||
new QueryParserCacheModule(settings).configure(binder());
|
||||
new BloomCacheModule(settings).configure(binder());
|
||||
|
||||
bind(IndexCache.class).asEagerSingleton();
|
||||
}
|
||||
|
|
|
@ -1,45 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search licenses this
|
||||
* file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.cache.bloom;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.elasticsearch.common.bloom.BloomFilter;
|
||||
import org.elasticsearch.common.component.CloseableComponent;
|
||||
import org.elasticsearch.index.IndexComponent;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public interface BloomCache extends IndexComponent, CloseableComponent {
|
||||
|
||||
/**
|
||||
* *Async* loads a bloom filter for the field name. Note, this one only supports
|
||||
* for fields that have a single term per doc.
|
||||
*/
|
||||
BloomFilter filter(IndexReader reader, String fieldName, boolean asyncLoad);
|
||||
|
||||
void clear();
|
||||
|
||||
void clear(IndexReader reader);
|
||||
|
||||
long sizeInBytes();
|
||||
|
||||
long sizeInBytes(String fieldName);
|
||||
}
|
|
@ -1,48 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search licenses this
|
||||
* file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.cache.bloom;
|
||||
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.inject.Scopes;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.cache.bloom.simple.SimpleBloomCache;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class BloomCacheModule extends AbstractModule {
|
||||
|
||||
public static final class BloomCacheSettings {
|
||||
public static final String TYPE = "index.cache.bloom.type";
|
||||
}
|
||||
|
||||
private final Settings settings;
|
||||
|
||||
public BloomCacheModule(Settings settings) {
|
||||
this.settings = settings;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void configure() {
|
||||
bind(BloomCache.class)
|
||||
.to(settings.getAsClass(BloomCacheSettings.TYPE, SimpleBloomCache.class, "org.elasticsearch.index.cache.bloom.", "BloomCache"))
|
||||
.in(Scopes.SINGLETON);
|
||||
}
|
||||
}
|
|
@ -1,73 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search licenses this
|
||||
* file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.cache.bloom.none;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.common.bloom.BloomFilter;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.ImmutableSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.cache.bloom.BloomCache;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class NoneBloomCache extends AbstractIndexComponent implements BloomCache {
|
||||
|
||||
public NoneBloomCache(Index index) {
|
||||
super(index, ImmutableSettings.Builder.EMPTY_SETTINGS);
|
||||
}
|
||||
|
||||
@Inject
|
||||
public NoneBloomCache(Index index, @IndexSettings Settings indexSettings) {
|
||||
super(index, indexSettings);
|
||||
}
|
||||
|
||||
@Override
|
||||
public BloomFilter filter(IndexReader reader, String fieldName, boolean asyncLoad) {
|
||||
return BloomFilter.NONE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear(IndexReader reader) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeInBytes() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeInBytes(String fieldName) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws ElasticSearchException {
|
||||
}
|
||||
}
|
|
@ -1,250 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elastic Search and Shay Banon under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Elastic Search licenses this
|
||||
* file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.cache.bloom.simple;
|
||||
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.util.StringHelper;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.common.Unicode;
|
||||
import org.elasticsearch.common.bloom.BloomFilter;
|
||||
import org.elasticsearch.common.bloom.BloomFilterFactory;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.SizeUnit;
|
||||
import org.elasticsearch.common.unit.SizeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.cache.bloom.BloomCache;
|
||||
import org.elasticsearch.index.settings.IndexSettings;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public class SimpleBloomCache extends AbstractIndexComponent implements BloomCache, SegmentReader.CoreClosedListener {
|
||||
|
||||
private final ThreadPool threadPool;
|
||||
|
||||
private final long maxSize;
|
||||
|
||||
private final ConcurrentMap<Object, ConcurrentMap<String, BloomFilterEntry>> cache;
|
||||
|
||||
private final Object creationMutex = new Object();
|
||||
|
||||
@Inject
|
||||
public SimpleBloomCache(Index index, @IndexSettings Settings indexSettings, ThreadPool threadPool) {
|
||||
super(index, indexSettings);
|
||||
this.threadPool = threadPool;
|
||||
|
||||
this.maxSize = indexSettings.getAsSize("index.cache.bloom.max_size", new SizeValue(500, SizeUnit.MEGA)).singles();
|
||||
this.cache = ConcurrentCollections.newConcurrentMap();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws ElasticSearchException {
|
||||
clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear() {
|
||||
cache.clear();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onClose(SegmentReader owner) {
|
||||
clear(owner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clear(IndexReader reader) {
|
||||
ConcurrentMap<String, BloomFilterEntry> map = cache.remove(reader.getCoreCacheKey());
|
||||
// help soft/weak handling GC
|
||||
if (map != null) {
|
||||
map.clear();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeInBytes() {
|
||||
// the overhead of the map is not really relevant...
|
||||
long sizeInBytes = 0;
|
||||
for (ConcurrentMap<String, BloomFilterEntry> map : cache.values()) {
|
||||
for (BloomFilterEntry filter : map.values()) {
|
||||
sizeInBytes += filter.filter.sizeInBytes();
|
||||
}
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeInBytes(String fieldName) {
|
||||
long sizeInBytes = 0;
|
||||
for (ConcurrentMap<String, BloomFilterEntry> map : cache.values()) {
|
||||
BloomFilterEntry filter = map.get(fieldName);
|
||||
if (filter != null) {
|
||||
sizeInBytes += filter.filter.sizeInBytes();
|
||||
}
|
||||
}
|
||||
return sizeInBytes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BloomFilter filter(IndexReader reader, String fieldName, boolean asyncLoad) {
|
||||
int currentNumDocs = reader.numDocs();
|
||||
if (currentNumDocs == 0) {
|
||||
return BloomFilter.EMPTY;
|
||||
}
|
||||
ConcurrentMap<String, BloomFilterEntry> fieldCache = cache.get(reader.getCoreCacheKey());
|
||||
if (fieldCache == null) {
|
||||
synchronized (creationMutex) {
|
||||
fieldCache = cache.get(reader.getCoreCacheKey());
|
||||
if (fieldCache == null) {
|
||||
if (reader instanceof SegmentReader) {
|
||||
((SegmentReader) reader).addCoreClosedListener(this);
|
||||
}
|
||||
fieldCache = ConcurrentCollections.newConcurrentMap();
|
||||
cache.put(reader.getCoreCacheKey(), fieldCache);
|
||||
}
|
||||
}
|
||||
}
|
||||
BloomFilterEntry filter = fieldCache.get(fieldName);
|
||||
if (filter == null) {
|
||||
synchronized (fieldCache) {
|
||||
filter = fieldCache.get(fieldName);
|
||||
if (filter == null) {
|
||||
filter = new BloomFilterEntry(currentNumDocs, BloomFilter.NONE);
|
||||
fieldCache.put(fieldName, filter);
|
||||
// now, do the async load of it...
|
||||
if (currentNumDocs < maxSize) {
|
||||
filter.loading.set(true);
|
||||
BloomFilterLoader loader = new BloomFilterLoader(reader, fieldName);
|
||||
if (asyncLoad) {
|
||||
threadPool.executor(ThreadPool.Names.CACHE).execute(loader);
|
||||
} else {
|
||||
loader.run();
|
||||
filter = fieldCache.get(fieldName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
// if we too many deletes, we need to reload the bloom filter so it will be more effective
|
||||
if (filter.numDocs > 1000 && filter.numDocs < maxSize && (currentNumDocs / filter.numDocs) < 0.6) {
|
||||
if (filter.loading.compareAndSet(false, true)) {
|
||||
// do the async loading
|
||||
BloomFilterLoader loader = new BloomFilterLoader(reader, fieldName);
|
||||
if (asyncLoad) {
|
||||
threadPool.executor(ThreadPool.Names.CACHE).execute(loader);
|
||||
} else {
|
||||
loader.run();
|
||||
filter = fieldCache.get(fieldName);
|
||||
}
|
||||
}
|
||||
}
|
||||
return filter.filter;
|
||||
}
|
||||
|
||||
class BloomFilterLoader implements Runnable {
|
||||
private final IndexReader reader;
|
||||
private final String field;
|
||||
|
||||
BloomFilterLoader(IndexReader reader, String field) {
|
||||
this.reader = reader;
|
||||
this.field = StringHelper.intern(field);
|
||||
}
|
||||
|
||||
@SuppressWarnings({"StringEquality"})
|
||||
@Override
|
||||
public void run() {
|
||||
TermDocs termDocs = null;
|
||||
TermEnum termEnum = null;
|
||||
try {
|
||||
UnicodeUtil.UTF8Result utf8Result = new UnicodeUtil.UTF8Result();
|
||||
BloomFilter filter = BloomFilterFactory.getFilter(reader.numDocs(), 15);
|
||||
termDocs = reader.termDocs();
|
||||
termEnum = reader.terms(new Term(field));
|
||||
do {
|
||||
Term term = termEnum.term();
|
||||
if (term == null || term.field() != field) break;
|
||||
|
||||
// LUCENE MONITOR: 4.0, move to use bytes!
|
||||
Unicode.fromStringAsUtf8(term.text(), utf8Result);
|
||||
termDocs.seek(termEnum);
|
||||
while (termDocs.next()) {
|
||||
// when traversing, make sure to ignore deleted docs, so the key->docId will be correct
|
||||
if (!reader.isDeleted(termDocs.doc())) {
|
||||
filter.add(utf8Result.result, 0, utf8Result.length);
|
||||
}
|
||||
}
|
||||
} while (termEnum.next());
|
||||
ConcurrentMap<String, BloomFilterEntry> fieldCache = cache.get(reader.getCoreCacheKey());
|
||||
if (fieldCache != null) {
|
||||
if (fieldCache.containsKey(field)) {
|
||||
BloomFilterEntry filterEntry = new BloomFilterEntry(reader.numDocs(), filter);
|
||||
filterEntry.loading.set(false);
|
||||
fieldCache.put(field, filterEntry);
|
||||
}
|
||||
}
|
||||
} catch (AlreadyClosedException e) {
|
||||
// ignore, we are getting closed
|
||||
} catch (ClosedChannelException e) {
|
||||
// ignore, we are getting closed
|
||||
} catch (Exception e) {
|
||||
// ignore failures that result from a closed reader...
|
||||
if (reader.getRefCount() > 0) {
|
||||
logger.warn("failed to load bloom filter for [{}]", e, field);
|
||||
}
|
||||
} finally {
|
||||
try {
|
||||
if (termDocs != null) {
|
||||
termDocs.close();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
try {
|
||||
if (termEnum != null) {
|
||||
termEnum.close();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
static class BloomFilterEntry {
|
||||
final int numDocs;
|
||||
final BloomFilter filter;
|
||||
final AtomicBoolean loading = new AtomicBoolean();
|
||||
|
||||
public BloomFilterEntry(int numDocs, BloomFilter filter) {
|
||||
this.numDocs = numDocs;
|
||||
this.filter = filter;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -23,15 +23,12 @@ import com.google.common.collect.Lists;
|
|||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.util.UnicodeUtil;
|
||||
import org.elasticsearch.ElasticSearchException;
|
||||
import org.elasticsearch.ElasticSearchIllegalStateException;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.operation.hash.djb.DjbHashFunction;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Preconditions;
|
||||
import org.elasticsearch.common.Unicode;
|
||||
import org.elasticsearch.common.bloom.BloomFilter;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.Lucene;
|
||||
|
@ -43,12 +40,10 @@ import org.elasticsearch.common.unit.TimeValue;
|
|||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.index.VersionType;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
import org.elasticsearch.index.cache.bloom.BloomCache;
|
||||
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
|
||||
import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit;
|
||||
import org.elasticsearch.index.engine.*;
|
||||
import org.elasticsearch.index.indexing.ShardIndexingService;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.merge.policy.EnableMergePolicy;
|
||||
import org.elasticsearch.index.merge.policy.MergePolicyProvider;
|
||||
import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider;
|
||||
|
@ -91,7 +86,6 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
|
|||
private volatile int indexConcurrency;
|
||||
private long gcDeletesInMillis;
|
||||
private volatile boolean enableGcDeletes = true;
|
||||
private final boolean asyncLoadBloomFilter;
|
||||
|
||||
private final ThreadPool threadPool;
|
||||
|
||||
|
@ -106,7 +100,6 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
|
|||
private final MergeSchedulerProvider mergeScheduler;
|
||||
private final AnalysisService analysisService;
|
||||
private final SimilarityService similarityService;
|
||||
private final BloomCache bloomCache;
|
||||
|
||||
|
||||
private final ReadWriteLock rwl = new ReentrantReadWriteLock();
|
||||
|
@ -154,8 +147,7 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
|
|||
IndexSettingsService indexSettingsService, ShardIndexingService indexingService, @Nullable IndicesWarmer warmer,
|
||||
Store store, SnapshotDeletionPolicy deletionPolicy, Translog translog,
|
||||
MergePolicyProvider mergePolicyProvider, MergeSchedulerProvider mergeScheduler,
|
||||
AnalysisService analysisService, SimilarityService similarityService,
|
||||
BloomCache bloomCache) throws EngineException {
|
||||
AnalysisService analysisService, SimilarityService similarityService) throws EngineException {
|
||||
super(shardId, indexSettings);
|
||||
Preconditions.checkNotNull(store, "Store must be provided to the engine");
|
||||
Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the engine");
|
||||
|
@ -165,7 +157,6 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
|
|||
this.indexingBufferSize = componentSettings.getAsBytesSize("index_buffer_size", new ByteSizeValue(64, ByteSizeUnit.MB)); // not really important, as it is set by the IndexingMemory manager
|
||||
this.termIndexInterval = indexSettings.getAsInt("index.term_index_interval", IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL);
|
||||
this.termIndexDivisor = indexSettings.getAsInt("index.term_index_divisor", 1); // IndexReader#DEFAULT_TERMS_INDEX_DIVISOR
|
||||
this.asyncLoadBloomFilter = componentSettings.getAsBoolean("async_load_bloom", true); // Here for testing, should always be true
|
||||
|
||||
this.threadPool = threadPool;
|
||||
this.indexSettingsService = indexSettingsService;
|
||||
|
@ -178,7 +169,6 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
|
|||
this.mergeScheduler = mergeScheduler;
|
||||
this.analysisService = analysisService;
|
||||
this.similarityService = similarityService;
|
||||
this.bloomCache = bloomCache;
|
||||
|
||||
this.indexConcurrency = indexSettings.getAsInt("index.index_concurrency", IndexWriterConfig.DEFAULT_MAX_THREAD_STATES);
|
||||
this.versionMap = ConcurrentCollections.newConcurrentMap();
|
||||
|
@ -328,16 +318,10 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
|
|||
// no version, get the version from the index, we know that we refresh on flush
|
||||
Searcher searcher = searcher();
|
||||
try {
|
||||
UnicodeUtil.UTF8Result utf8 = Unicode.fromStringAsUtf8(get.uid().text());
|
||||
for (int i = 0; i < searcher.searcher().subReaders().length; i++) {
|
||||
IndexReader subReader = searcher.searcher().subReaders()[i];
|
||||
BloomFilter filter = bloomCache.filter(subReader, UidFieldMapper.NAME, asyncLoadBloomFilter);
|
||||
// we know that its not there...
|
||||
if (!filter.isPresent(utf8.result, 0, utf8.length)) {
|
||||
continue;
|
||||
}
|
||||
int docStart = searcher.searcher().docStarts()[i];
|
||||
UidField.DocIdAndVersion docIdAndVersion = UidField.loadDocIdAndVersion(subReader, docStart, get.uid());
|
||||
List<AtomicReaderContext> readers = searcher.reader().leaves();
|
||||
for (int i = 0; i < readers.size(); i++) {
|
||||
AtomicReaderContext readerContext = readers.get(i);
|
||||
UidField.DocIdAndVersion docIdAndVersion = UidField.loadDocIdAndVersion(readerContext, get.uid());
|
||||
if (docIdAndVersion != null && docIdAndVersion.docId != Lucene.NO_DOC) {
|
||||
return new GetResult(searcher, docIdAndVersion);
|
||||
}
|
||||
|
@ -1321,16 +1305,12 @@ public class RobinEngine extends AbstractIndexShardComponent implements Engine {
|
|||
}
|
||||
|
||||
private long loadCurrentVersionFromIndex(Term uid) {
|
||||
UnicodeUtil.UTF8Result utf8 = Unicode.fromStringAsUtf8(uid.text());
|
||||
Searcher searcher = searcher();
|
||||
try {
|
||||
for (IndexReader reader : searcher.searcher().subReaders()) {
|
||||
BloomFilter filter = bloomCache.filter(reader, UidFieldMapper.NAME, asyncLoadBloomFilter);
|
||||
// we know that its not there...
|
||||
if (!filter.isPresent(utf8.result, 0, utf8.length)) {
|
||||
continue;
|
||||
}
|
||||
long version = UidField.loadVersion(reader, uid);
|
||||
List<AtomicReaderContext> readers = searcher.reader().leaves();
|
||||
for (int i = 0; i < readers.size(); i++) {
|
||||
AtomicReaderContext readerContext = readers.get(i);
|
||||
long version = UidField.loadVersion(readerContext, uid);
|
||||
// either -2 (its there, but no version associated), or an actual version
|
||||
if (version != -1) {
|
||||
return version;
|
||||
|
|
|
@ -43,6 +43,7 @@ public class UidFilter extends Filter {
|
|||
|
||||
private final BloomCache bloomCache;
|
||||
|
||||
// LUCENE 4 UPGRADE: We removed the bloom cache, so once we rewrite this filter, do it without
|
||||
public UidFilter(Collection<String> types, List<String> ids, BloomCache bloomCache) {
|
||||
this.bloomCache = bloomCache;
|
||||
this.uids = new Term[types.size() * ids.size()];
|
||||
|
|
|
@ -66,7 +66,6 @@ public class RestClearIndicesCacheAction extends BaseRestHandler {
|
|||
clearIndicesCacheRequest.filterCache(request.paramAsBoolean("filter", clearIndicesCacheRequest.filterCache()));
|
||||
clearIndicesCacheRequest.fieldDataCache(request.paramAsBoolean("field_data", clearIndicesCacheRequest.fieldDataCache()));
|
||||
clearIndicesCacheRequest.idCache(request.paramAsBoolean("id", clearIndicesCacheRequest.idCache()));
|
||||
clearIndicesCacheRequest.bloomCache(request.paramAsBoolean("bloom", clearIndicesCacheRequest.bloomCache()));
|
||||
clearIndicesCacheRequest.fields(request.paramAsStringArray("fields", clearIndicesCacheRequest.fields()));
|
||||
|
||||
BroadcastOperationThreading operationThreading = BroadcastOperationThreading.fromString(request.param("operationThreading"), BroadcastOperationThreading.SINGLE_THREAD);
|
||||
|
|
Loading…
Reference in New Issue