Merge remote-tracking branch 'es/master' into ccr

* es/master:
  Take into account the return value of TcpTransport.readMessageLength(...) in Netty4SizeHeaderFrameDecoder
  Move caching of the size of a directory to `StoreDirectory`. (#30581)
  Clarify docs about boolean operator precedence. (#30808)
  Docs: remove notes on sparsity. (#30905)
  Fix MatchPhrasePrefixQueryBuilderTests#testPhraseOnFieldWithNoTerms
  run overflow forecast a 2nd time as regression test for elastic/ml-cpp#110 (#30969)
  Improve documentation of dynamic mappings. (#30952)
  Decouple MultiValueMode. (#31075)
  Docs: Clarify constraints on scripted similarities. (#31076)
  Update get.asciidoc (#31084)
This commit is contained in:
Martijn van Groningen 2018-06-05 10:37:38 +02:00
commit 530089f132
No known key found for this signature in database
GPG Key ID: AB236F4FCF2AF12A
29 changed files with 629 additions and 344 deletions

View File

@ -167,7 +167,7 @@ The result of the above get operation is:
// TESTRESPONSE
Field values fetched from the document it self are always returned as an array.
Field values fetched from the document itself are always returned as an array.
Since the `counter` field is not stored the get request simply ignores it when trying to get the `stored_fields.`
It is also possible to retrieve metadata fields like the `_routing` field:

View File

@ -40,94 +40,3 @@ better. For instance if a user searches for two words `foo` and `bar`, a match
across different chapters is probably very poor, while a match within the same
paragraph is likely good.
[float]
[[sparsity]]
=== Avoid sparsity
The data-structures behind Lucene, which Elasticsearch relies on in order to
index and store data, work best with dense data, ie. when all documents have the
same fields. This is especially true for fields that have norms enabled (which
is the case for `text` fields by default) or doc values enabled (which is the
case for numerics, `date`, `ip` and `keyword` by default).
The reason is that Lucene internally identifies documents with so-called doc
ids, which are integers between 0 and the total number of documents in the
index. These doc ids are used for communication between the internal APIs of
Lucene: for instance searching on a term with a `match` query produces an
iterator of doc ids, and these doc ids are then used to retrieve the value of
the `norm` in order to compute a score for these documents. The way this `norm`
lookup is implemented currently is by reserving one byte for each document.
The `norm` value for a given doc id can then be retrieved by reading the
byte at index `doc_id`. While this is very efficient and helps Lucene quickly
have access to the `norm` values of every document, this has the drawback that
documents that do not have a value will also require one byte of storage.
In practice, this means that if an index has `M` documents, norms will require
`M` bytes of storage *per field*, even for fields that only appear in a small
fraction of the documents of the index. Although slightly more complex with doc
values due to the fact that doc values have multiple ways that they can be
encoded depending on the type of field and on the actual data that the field
stores, the problem is very similar. In case you wonder: `fielddata`, which was
used in Elasticsearch pre-2.0 before being replaced with doc values, also
suffered from this issue, except that the impact was only on the memory
footprint since `fielddata` was not explicitly materialized on disk.
Note that even though the most notable impact of sparsity is on storage
requirements, it also has an impact on indexing speed and search speed since
these bytes for documents that do not have a field still need to be written
at index time and skipped over at search time.
It is totally fine to have a minority of sparse fields in an index. But beware
that if sparsity becomes the rule rather than the exception, then the index
will not be as efficient as it could be.
This section mostly focused on `norms` and `doc values` because those are the
two features that are most affected by sparsity. Sparsity also affect the
efficiency of the inverted index (used to index `text`/`keyword` fields) and
dimensional points (used to index `geo_point` and numerics) but to a lesser
extent.
Here are some recommendations that can help avoid sparsity:
[float]
==== Avoid putting unrelated data in the same index
You should avoid putting documents that have totally different structures into
the same index in order to avoid sparsity. It is often better to put these
documents into different indices, you could also consider giving fewer shards
to these smaller indices since they will contain fewer documents overall.
Note that this advice does not apply in the case that you need to use
parent/child relations between your documents since this feature is only
supported on documents that live in the same index.
[float]
==== Normalize document structures
Even if you really need to put different kinds of documents in the same index,
maybe there are opportunities to reduce sparsity. For instance if all documents
in the index have a timestamp field but some call it `timestamp` and others
call it `creation_date`, it would help to rename it so that all documents have
the same field name for the same data.
[float]
==== Avoid types
Types might sound like a good way to store multiple tenants in a single index.
They are not: given that types store everything in a single index, having
multiple types that have different fields in a single index will also cause
problems due to sparsity as described above. If your types do not have very
similar mappings, you might want to consider moving them to a dedicated index.
[float]
==== Disable `norms` and `doc_values` on sparse fields
If none of the above recommendations apply in your case, you might want to
check whether you actually need `norms` and `doc_values` on your sparse fields.
`norms` can be disabled if producing scores is not necessary on a field, this is
typically true for fields that are only used for filtering. `doc_values` can be
disabled on fields that are neither used for sorting nor for aggregations.
Beware that this decision should not be made lightly since these parameters
cannot be changed on a live index, so you would have to reindex if you realize
that you need `norms` or `doc_values`.

View File

@ -341,7 +341,18 @@ Which yields:
// TESTRESPONSE[s/"took": 12/"took" : $body.took/]
// TESTRESPONSE[s/OzrdjxNtQGaqs4DmioFw9A/$body.hits.hits.0._node/]
You might have noticed that a significant part of the script depends on
WARNING: While scripted similarities provide a lot of flexibility, there is
a set of rules that they need to satisfy. Failing to do so could make
Elasticsearch silently return wrong top hits or fail with internal errors at
search time:
- Returned scores must be positive.
- All other variables remaining equal, scores must not decrease when
`doc.freq` increases.
- All other variables remaining equal, scores must not increase when
`doc.length` increases.
You might have noticed that a significant part of the above script depends on
statistics that are the same for every document. It is possible to make the
above slightly more efficient by providing an `weight_script` which will
compute the document-independent part of the score and will be available
@ -506,7 +517,6 @@ GET /index/_search?explain=true
////////////////////
Type name: `scripted`
[float]

View File

@ -135,6 +135,6 @@ PUT my_index/_doc/1
}
--------------------------------------------------
// CONSOLE
<1> The `my_float` field is added as a <<number,`double`>> field.
<1> The `my_float` field is added as a <<number,`float`>> field.
<2> The `my_integer` field is added as a <<number,`long`>> field.

View File

@ -46,11 +46,22 @@ name as an existing template, it will replace the old version.
[[match-mapping-type]]
==== `match_mapping_type`
The `match_mapping_type` matches on the datatype detected by
<<dynamic-field-mapping,dynamic field mapping>>, in other words, the datatype
that Elasticsearch thinks the field should have. Only the following datatypes
can be automatically detected: `boolean`, `date`, `double`, `long`, `object`,
`string`. It also accepts `*` to match all datatypes.
The `match_mapping_type` is the datatype detected by the json parser. Since
JSON doesn't allow to distinguish a `long` from an `integer` or a `double` from
a `float`, it will always choose the wider datatype, ie. `long` for integers
and `double` for floating-point numbers.
The following datatypes may be automatically detected:
- `boolean` when `true` or `false` are encountered.
- `date` when <<date-detection,date detection>> is enabled and a string is
found that matches any of the configured date formats.
- `double` for numbers with a decimal part.
- `long` for numbers without a decimal part.
- `object` for objects, also called hashes.
- `string` for character strings.
`*` may also be used in order to match all datatypes.
For example, if we wanted to map all integer fields as `integer` instead of
`long`, and all `string` fields as both `text` and `keyword`, we

View File

@ -233,26 +233,10 @@ states that:
* `news` must not be present
* `quick` and `brown` are optional -- their presence increases the relevance
The familiar operators `AND`, `OR` and `NOT` (also written `&&`, `||` and `!`)
are also supported. However, the effects of these operators can be more
complicated than is obvious at first glance. `NOT` takes precedence over
`AND`, which takes precedence over `OR`. While the `+` and `-` only affect
the term to the right of the operator, `AND` and `OR` can affect the terms to
the left and right.
****
Rewriting the above query using `AND`, `OR` and `NOT` demonstrates the
complexity:
`quick OR brown AND fox AND NOT news`::
This is incorrect, because `brown` is now a required term.
`(quick OR brown) AND fox AND NOT news`::
This is incorrect because at least one of `quick` or `brown` is now required
and the search for those terms would be scored differently from the original
query.
The familiar boolean operators `AND`, `OR` and `NOT` (also written `&&`, `||`
and `!`) are also supported but beware that they do not honor the usual
precedence rules, so parentheses should be used whenever multiple operators are
used together. For instance the previous query could be rewritten as:
`((quick AND fox) OR (brown AND fox) OR fox) AND NOT news`::

View File

@ -47,7 +47,7 @@ public abstract class MultiValuesSource <VS extends ValuesSource> {
if (ordinal > names.length) {
throw new IndexOutOfBoundsException("ValuesSource array index " + ordinal + " out of bounds");
}
return multiValueMode.select(values[ordinal].doubleValues(ctx), Double.NEGATIVE_INFINITY);
return multiValueMode.select(values[ordinal].doubleValues(ctx));
}
}

View File

@ -54,7 +54,7 @@ class DateMethodValueSource extends FieldDataValueSource {
public FunctionValues getValues(Map context, LeafReaderContext leaf) throws IOException {
AtomicNumericFieldData leafData = (AtomicNumericFieldData) fieldData.load(leaf);
final Calendar calendar = Calendar.getInstance(TimeZone.getTimeZone("UTC"), Locale.ROOT);
NumericDoubleValues docValues = multiValueMode.select(leafData.getDoubleValues(), 0d);
NumericDoubleValues docValues = multiValueMode.select(leafData.getDoubleValues());
return new DoubleDocValues(this) {
@Override
public double doubleVal(int docId) throws IOException {

View File

@ -56,7 +56,7 @@ class DateObjectValueSource extends FieldDataValueSource {
public FunctionValues getValues(Map context, LeafReaderContext leaf) throws IOException {
AtomicNumericFieldData leafData = (AtomicNumericFieldData) fieldData.load(leaf);
MutableDateTime joda = new MutableDateTime(0, DateTimeZone.UTC);
NumericDoubleValues docValues = multiValueMode.select(leafData.getDoubleValues(), 0d);
NumericDoubleValues docValues = multiValueMode.select(leafData.getDoubleValues());
return new DoubleDocValues(this) {
@Override
public double doubleVal(int docId) throws IOException {

View File

@ -68,7 +68,7 @@ class FieldDataValueSource extends ValueSource {
@SuppressWarnings("rawtypes") // ValueSource uses a rawtype
public FunctionValues getValues(Map context, LeafReaderContext leaf) throws IOException {
AtomicNumericFieldData leafData = (AtomicNumericFieldData) fieldData.load(leaf);
NumericDoubleValues docValues = multiValueMode.select(leafData.getDoubleValues(), 0d);
NumericDoubleValues docValues = multiValueMode.select(leafData.getDoubleValues());
return new DoubleDocValues(this) {
@Override
public double doubleVal(int doc) throws IOException {

View File

@ -37,17 +37,19 @@ final class Netty4SizeHeaderFrameDecoder extends ByteToMessageDecoder {
protected void decode(ChannelHandlerContext ctx, ByteBuf in, List<Object> out) throws Exception {
try {
BytesReference networkBytes = Netty4Utils.toBytesReference(in);
int messageLength = TcpTransport.readMessageLength(networkBytes) + HEADER_SIZE;
// If the message length is -1, we have not read a complete header. If the message length is
// greater than the network bytes available, we have not read a complete frame.
if (messageLength != -1 && messageLength <= networkBytes.length()) {
final ByteBuf message = in.skipBytes(HEADER_SIZE);
// 6 bytes would mean it is a ping. And we should ignore.
if (messageLength != 6) {
out.add(message);
int messageLength = TcpTransport.readMessageLength(networkBytes);
// If the message length is -1, we have not read a complete header.
if (messageLength != -1) {
int messageLengthWithHeader = messageLength + HEADER_SIZE;
// If the message length is greater than the network bytes available, we have not read a complete frame.
if (messageLengthWithHeader <= networkBytes.length()) {
final ByteBuf message = in.skipBytes(HEADER_SIZE);
// 6 bytes would mean it is a ping. And we should ignore.
if (messageLengthWithHeader != 6) {
out.add(message);
}
}
}
} catch (IllegalArgumentException ex) {
throw new TooLongFrameException(ex);
}

View File

@ -64,6 +64,11 @@ public abstract class SingleObjectCache<T>{
return cached;
}
/** Return the potentially stale cached entry. */
protected final T getNoRefresh() {
return cached;
}
/**
* Returns a new instance to cache
*/

View File

@ -291,38 +291,6 @@ public enum FieldData {
return DocValues.unwrapSingleton(values) == null;
}
/**
* Returns whether the provided values *might* be multi-valued. There is no
* guarantee that this method will return {@code false} in the single-valued case.
*/
public static boolean isMultiValued(SortedNumericDocValues values) {
return DocValues.unwrapSingleton(values) == null;
}
/**
* Returns whether the provided values *might* be multi-valued. There is no
* guarantee that this method will return {@code false} in the single-valued case.
*/
public static boolean isMultiValued(SortedNumericDoubleValues values) {
return unwrapSingleton(values) == null;
}
/**
* Returns whether the provided values *might* be multi-valued. There is no
* guarantee that this method will return {@code false} in the single-valued case.
*/
public static boolean isMultiValued(SortedBinaryDocValues values) {
return unwrapSingleton(values) != null;
}
/**
* Returns whether the provided values *might* be multi-valued. There is no
* guarantee that this method will return {@code false} in the single-valued case.
*/
public static boolean isMultiValued(MultiGeoPointValues values) {
return unwrapSingleton(values) == null;
}
/**
* Return a {@link String} representation of the provided values. That is
* typically used for scripts or for the `map` execution mode of terms aggs.
@ -555,4 +523,63 @@ public enum FieldData {
}
}
/**
* Return a {@link NumericDocValues} instance that has a value for every
* document, returns the same value as {@code values} if there is a value
* for the current document and {@code missing} otherwise.
*/
public static NumericDocValues replaceMissing(NumericDocValues values, long missing) {
return new AbstractNumericDocValues() {
private long value;
@Override
public int docID() {
return values.docID();
}
@Override
public boolean advanceExact(int target) throws IOException {
if (values.advanceExact(target)) {
value = values.longValue();
} else {
value = missing;
}
return true;
}
@Override
public long longValue() throws IOException {
return value;
}
};
}
/**
* Return a {@link NumericDoubleValues} instance that has a value for every
* document, returns the same value as {@code values} if there is a value
* for the current document and {@code missing} otherwise.
*/
public static NumericDoubleValues replaceMissing(NumericDoubleValues values, double missing) {
return new NumericDoubleValues() {
private double value;
@Override
public boolean advanceExact(int target) throws IOException {
if (values.advanceExact(target)) {
value = values.doubleValue();
} else {
value = missing;
}
return true;
}
@Override
public double doubleValue() throws IOException {
return value;
}
};
}
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.search.Scorer;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BitSet;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
@ -71,7 +72,7 @@ public class DoubleValuesComparatorSource extends IndexFieldData.XFieldComparato
final SortedNumericDoubleValues values = getValues(context);
final NumericDoubleValues selectedValues;
if (nested == null) {
selectedValues = sortMode.select(values, dMissingValue);
selectedValues = FieldData.replaceMissing(sortMode.select(values), dMissingValue);
} else {
final BitSet rootDocs = nested.rootDocs(context);
final DocIdSetIterator innerDocs = nested.innerDocs(context);

View File

@ -25,6 +25,7 @@ import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BitSet;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.NumericDoubleValues;
@ -63,7 +64,7 @@ public class FloatValuesComparatorSource extends IndexFieldData.XFieldComparator
final SortedNumericDoubleValues values = indexFieldData.load(context).getDoubleValues();
final NumericDoubleValues selectedValues;
if (nested == null) {
selectedValues = sortMode.select(values, dMissingValue);
selectedValues = FieldData.replaceMissing(sortMode.select(values), dMissingValue);
} else {
final BitSet rootDocs = nested.rootDocs(context);
final DocIdSetIterator innerDocs = nested.innerDocs(context);

View File

@ -26,6 +26,7 @@ import org.apache.lucene.search.FieldComparator;
import org.apache.lucene.search.SortField;
import org.apache.lucene.util.BitSet;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.search.MultiValueMode;
@ -62,7 +63,7 @@ public class LongValuesComparatorSource extends IndexFieldData.XFieldComparatorS
final SortedNumericDocValues values = indexFieldData.load(context).getLongValues();
final NumericDocValues selectedValues;
if (nested == null) {
selectedValues = sortMode.select(values, dMissingValue);
selectedValues = FieldData.replaceMissing(sortMode.select(values), dMissingValue);
} else {
final BitSet rootDocs = nested.rootDocs(context);
final DocIdSetIterator innerDocs = nested.innerDocs(context);

View File

@ -40,6 +40,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
import org.elasticsearch.index.fielddata.MultiGeoPointValues;
@ -354,7 +355,7 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder<DFB>
@Override
protected NumericDoubleValues distance(LeafReaderContext context) {
final MultiGeoPointValues geoPointValues = fieldData.load(context).getGeoPointValues();
return mode.select(new SortingNumericDoubleValues() {
return FieldData.replaceMissing(mode.select(new SortingNumericDoubleValues() {
@Override
public boolean advanceExact(int docId) throws IOException {
if (geoPointValues.advanceExact(docId)) {
@ -372,7 +373,7 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder<DFB>
return false;
}
}
}, 0.0);
}), 0);
}
@Override
@ -436,7 +437,7 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder<DFB>
@Override
protected NumericDoubleValues distance(LeafReaderContext context) {
final SortedNumericDoubleValues doubleValues = fieldData.load(context).getDoubleValues();
return mode.select(new SortingNumericDoubleValues() {
return FieldData.replaceMissing(mode.select(new SortingNumericDoubleValues() {
@Override
public boolean advanceExact(int docId) throws IOException {
if (doubleValues.advanceExact(docId)) {
@ -451,7 +452,7 @@ public abstract class DecayFunctionBuilder<DFB extends DecayFunctionBuilder<DFB>
return false;
}
}
}, 0.0);
}), 0);
}
@Override

View File

@ -0,0 +1,183 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.store;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FilterDirectory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.elasticsearch.common.lucene.store.FilterIndexOutput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.SingleObjectCache;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.nio.file.AccessDeniedException;
import java.nio.file.NoSuchFileException;
final class ByteSizeCachingDirectory extends FilterDirectory {
private static class SizeAndModCount {
final long size;
final long modCount;
final boolean pendingWrite;
SizeAndModCount(long length, long modCount, boolean pendingWrite) {
this.size = length;
this.modCount = modCount;
this.pendingWrite = pendingWrite;
}
}
private static long estimateSizeInBytes(Directory directory) throws IOException {
long estimatedSize = 0;
String[] files = directory.listAll();
for (String file : files) {
try {
estimatedSize += directory.fileLength(file);
} catch (NoSuchFileException | FileNotFoundException | AccessDeniedException e) {
// ignore, the file is not there no more; on Windows, if one thread concurrently deletes a file while
// calling Files.size, you can also sometimes hit AccessDeniedException
}
}
return estimatedSize;
}
private final SingleObjectCache<SizeAndModCount> size;
// Both these variables need to be accessed under `this` lock.
private long modCount = 0;
private long numOpenOutputs = 0;
ByteSizeCachingDirectory(Directory in, TimeValue refreshInterval) {
super(in);
size = new SingleObjectCache<SizeAndModCount>(refreshInterval, new SizeAndModCount(0L, -1L, true)) {
@Override
protected SizeAndModCount refresh() {
// It is ok for the size of the directory to be more recent than
// the mod count, we would just recompute the size of the
// directory on the next call as well. However the opposite
// would be bad as we would potentially have a stale cache
// entry for a long time. So we fetch the values of modCount and
// numOpenOutputs BEFORE computing the size of the directory.
final long modCount;
final boolean pendingWrite;
synchronized(ByteSizeCachingDirectory.this) {
modCount = ByteSizeCachingDirectory.this.modCount;
pendingWrite = ByteSizeCachingDirectory.this.numOpenOutputs != 0;
}
final long size;
try {
// Compute this OUTSIDE of the lock
size = estimateSizeInBytes(getDelegate());
} catch (IOException e) {
throw new UncheckedIOException(e);
}
return new SizeAndModCount(size, modCount, pendingWrite);
}
@Override
protected boolean needsRefresh() {
if (super.needsRefresh() == false) {
// The size was computed recently, don't recompute
return false;
}
SizeAndModCount cached = getNoRefresh();
if (cached.pendingWrite) {
// The cached entry was generated while there were pending
// writes, so the size might be stale: recompute.
return true;
}
synchronized(ByteSizeCachingDirectory.this) {
// If there are pending writes or if new files have been
// written/deleted since last time: recompute
return numOpenOutputs != 0 || cached.modCount != modCount;
}
}
};
}
/** Return the cumulative size of all files in this directory. */
long estimateSizeInBytes() throws IOException {
try {
return size.getOrRefresh().size;
} catch (UncheckedIOException e) {
// we wrapped in the cache and unwrap here
throw e.getCause();
}
}
@Override
public IndexOutput createOutput(String name, IOContext context) throws IOException {
return wrapIndexOutput(super.createOutput(name, context));
}
@Override
public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException {
return wrapIndexOutput(super.createTempOutput(prefix, suffix, context));
}
private IndexOutput wrapIndexOutput(IndexOutput out) {
synchronized (this) {
numOpenOutputs++;
}
return new FilterIndexOutput(out.toString(), out) {
@Override
public void writeBytes(byte[] b, int length) throws IOException {
// Don't write to atomicXXX here since it might be called in
// tight loops and memory barriers are costly
super.writeBytes(b, length);
}
@Override
public void writeByte(byte b) throws IOException {
// Don't write to atomicXXX here since it might be called in
// tight loops and memory barriers are costly
super.writeByte(b);
}
@Override
public void close() throws IOException {
// Close might cause some data to be flushed from in-memory buffers, so
// increment the modification counter too.
try {
super.close();
} finally {
synchronized (this) {
numOpenOutputs--;
modCount++;
}
}
}
};
}
@Override
public void deleteFile(String name) throws IOException {
try {
super.deleteFile(name);
} finally {
synchronized (this) {
modCount++;
}
}
}
}

View File

@ -50,7 +50,6 @@ import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.Version;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.bytes.BytesReference;
@ -67,7 +66,6 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.SingleObjectCache;
import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
import org.elasticsearch.common.util.concurrent.RefCounted;
import org.elasticsearch.common.util.iterable.Iterables;
@ -91,7 +89,6 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.io.PrintStream;
import java.nio.file.AccessDeniedException;
import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
import java.util.ArrayList;
@ -146,7 +143,6 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
private final ReentrantReadWriteLock metadataLock = new ReentrantReadWriteLock();
private final ShardLock shardLock;
private final OnClose onClose;
private final SingleObjectCache<StoreStats> statsCache;
private final AbstractRefCounted refCounter = new AbstractRefCounted("store") {
@Override
@ -164,12 +160,13 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
OnClose onClose) throws IOException {
super(shardId, indexSettings);
final Settings settings = indexSettings.getSettings();
this.directory = new StoreDirectory(directoryService.newDirectory(), Loggers.getLogger("index.store.deletes", settings, shardId));
Directory dir = directoryService.newDirectory();
final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING);
logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);
ByteSizeCachingDirectory sizeCachingDir = new ByteSizeCachingDirectory(dir, refreshInterval);
this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", settings, shardId));
this.shardLock = shardLock;
this.onClose = onClose;
final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING);
this.statsCache = new StoreStatsCache(refreshInterval, directory);
logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);
assert onClose != null;
assert shardLock != null;
@ -377,7 +374,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
public StoreStats stats() throws IOException {
ensureOpen();
return statsCache.getOrRefresh();
return new StoreStats(directory.estimateSize());
}
/**
@ -731,11 +728,16 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
private final Logger deletesLogger;
StoreDirectory(Directory delegateDirectory, Logger deletesLogger) {
StoreDirectory(ByteSizeCachingDirectory delegateDirectory, Logger deletesLogger) {
super(delegateDirectory);
this.deletesLogger = deletesLogger;
}
/** Estimate the cumulative size of all files in this directory in bytes. */
long estimateSize() throws IOException {
return ((ByteSizeCachingDirectory) getDelegate()).estimateSizeInBytes();
}
@Override
public void close() {
assert false : "Nobody should close this directory except of the Store itself";
@ -1437,38 +1439,6 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
};
}
private static class StoreStatsCache extends SingleObjectCache<StoreStats> {
private final Directory directory;
StoreStatsCache(TimeValue refreshInterval, Directory directory) throws IOException {
super(refreshInterval, new StoreStats(estimateSize(directory)));
this.directory = directory;
}
@Override
protected StoreStats refresh() {
try {
return new StoreStats(estimateSize(directory));
} catch (IOException ex) {
throw new ElasticsearchException("failed to refresh store stats", ex);
}
}
private static long estimateSize(Directory directory) throws IOException {
long estimatedSize = 0;
String[] files = directory.listAll();
for (String file : files) {
try {
estimatedSize += directory.fileLength(file);
} catch (NoSuchFileException | FileNotFoundException | AccessDeniedException e) {
// ignore, the file is not there no more; on Windows, if one thread concurrently deletes a file while
// calling Files.size, you can also sometimes hit AccessDeniedException
}
}
return estimatedSize;
}
}
/**
* creates an empty lucene index and a corresponding empty translog. Any existing data will be deleted.
*/

View File

@ -411,29 +411,10 @@ public enum MultiValueMode implements Writeable {
*
* Allowed Modes: SUM, AVG, MEDIAN, MIN, MAX
*/
public NumericDocValues select(final SortedNumericDocValues values, final long missingValue) {
public NumericDocValues select(final SortedNumericDocValues values) {
final NumericDocValues singleton = DocValues.unwrapSingleton(values);
if (singleton != null) {
return new AbstractNumericDocValues() {
private long value;
@Override
public boolean advanceExact(int target) throws IOException {
this.value = singleton.advanceExact(target) ? singleton.longValue() : missingValue;
return true;
}
@Override
public int docID() {
return singleton.docID();
}
@Override
public long longValue() throws IOException {
return this.value;
}
};
return singleton;
} else {
return new AbstractNumericDocValues() {
@ -441,8 +422,11 @@ public enum MultiValueMode implements Writeable {
@Override
public boolean advanceExact(int target) throws IOException {
this.value = values.advanceExact(target) ? pick(values) : missingValue;
return true;
if (values.advanceExact(target)) {
value = pick(values);
return true;
}
return false;
}
@Override
@ -476,7 +460,7 @@ public enum MultiValueMode implements Writeable {
*/
public NumericDocValues select(final SortedNumericDocValues values, final long missingValue, final BitSet parentDocs, final DocIdSetIterator childDocs, int maxDoc) throws IOException {
if (parentDocs == null || childDocs == null) {
return select(DocValues.emptySortedNumeric(maxDoc), missingValue);
return FieldData.replaceMissing(DocValues.emptyNumeric(), missingValue);
}
return new AbstractNumericDocValues() {
@ -529,23 +513,10 @@ public enum MultiValueMode implements Writeable {
*
* Allowed Modes: SUM, AVG, MEDIAN, MIN, MAX
*/
public NumericDoubleValues select(final SortedNumericDoubleValues values, final double missingValue) {
public NumericDoubleValues select(final SortedNumericDoubleValues values) {
final NumericDoubleValues singleton = FieldData.unwrapSingleton(values);
if (singleton != null) {
return new NumericDoubleValues() {
private double value;
@Override
public boolean advanceExact(int doc) throws IOException {
this.value = singleton.advanceExact(doc) ? singleton.doubleValue() : missingValue;
return true;
}
@Override
public double doubleValue() throws IOException {
return this.value;
}
};
return singleton;
} else {
return new NumericDoubleValues() {
@ -553,8 +524,11 @@ public enum MultiValueMode implements Writeable {
@Override
public boolean advanceExact(int target) throws IOException {
value = values.advanceExact(target) ? pick(values) : missingValue;
return true;
if (values.advanceExact(target)) {
value = pick(values);
return true;
}
return false;
}
@Override
@ -583,7 +557,7 @@ public enum MultiValueMode implements Writeable {
*/
public NumericDoubleValues select(final SortedNumericDoubleValues values, final double missingValue, final BitSet parentDocs, final DocIdSetIterator childDocs, int maxDoc) throws IOException {
if (parentDocs == null || childDocs == null) {
return select(FieldData.emptySortedNumericDoubles(), missingValue);
return FieldData.replaceMissing(FieldData.emptyNumericDouble(), missingValue);
}
return new NumericDoubleValues() {

View File

@ -72,7 +72,7 @@ public class MaxAggregator extends NumericMetricsAggregator.SingleValue {
}
final BigArrays bigArrays = context.bigArrays();
final SortedNumericDoubleValues allValues = valuesSource.doubleValues(ctx);
final NumericDoubleValues values = MultiValueMode.MAX.select(allValues, Double.NEGATIVE_INFINITY);
final NumericDoubleValues values = MultiValueMode.MAX.select(allValues);
return new LeafBucketCollectorBase(sub, allValues) {
@Override

View File

@ -71,7 +71,7 @@ public class MinAggregator extends NumericMetricsAggregator.SingleValue {
}
final BigArrays bigArrays = context.bigArrays();
final SortedNumericDoubleValues allValues = valuesSource.doubleValues(ctx);
final NumericDoubleValues values = MultiValueMode.MIN.select(allValues, Double.POSITIVE_INFINITY);
final NumericDoubleValues values = MultiValueMode.MIN.select(allValues);
return new LeafBucketCollectorBase(sub, allValues) {
@Override

View File

@ -41,6 +41,7 @@ import org.elasticsearch.common.unit.DistanceUnit;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import org.elasticsearch.index.fielddata.FieldData;
import org.elasticsearch.index.fielddata.IndexFieldData;
import org.elasticsearch.index.fielddata.IndexFieldData.XFieldComparatorSource.Nested;
import org.elasticsearch.index.fielddata.IndexGeoPointFieldData;
@ -637,7 +638,7 @@ public class GeoDistanceSortBuilder extends SortBuilder<GeoDistanceSortBuilder>
localPoints);
final NumericDoubleValues selectedValues;
if (nested == null) {
selectedValues = finalSortMode.select(distanceValues, Double.POSITIVE_INFINITY);
selectedValues = FieldData.replaceMissing(finalSortMode.select(distanceValues), Double.POSITIVE_INFINITY);
} else {
final BitSet rootDocs = nested.rootDocs(context);
final DocIdSetIterator innerDocs = nested.innerDocs(context);

View File

@ -137,4 +137,91 @@ public class FieldDataTests extends ESTestCase {
assertEquals(valueBits, asMultiLongs.nextValue());
assertSame(multiValues, FieldData.sortableLongBitsToDoubles(asMultiLongs));
}
private static NumericDocValues asNumericDocValues(Long... values) {
return new AbstractNumericDocValues() {
int docID = -1;
@Override
public int docID() {
return docID;
}
@Override
public boolean advanceExact(int target) throws IOException {
docID = target;
return target < values.length && values[target] != null;
}
@Override
public long longValue() throws IOException {
return values[docID];
}
};
}
public void testReplaceMissingLongs() throws IOException {
final NumericDocValues values = asNumericDocValues(null, 3L, 2L, null, 5L, null);
final NumericDocValues replaced = FieldData.replaceMissing(values, 4);
assertTrue(replaced.advanceExact(0));
assertEquals(4L, replaced.longValue());
assertTrue(replaced.advanceExact(1));
assertEquals(3L, replaced.longValue());
assertTrue(replaced.advanceExact(2));
assertEquals(2L, replaced.longValue());
assertTrue(replaced.advanceExact(3));
assertEquals(4L, replaced.longValue());
assertTrue(replaced.advanceExact(4));
assertEquals(5L, replaced.longValue());
assertTrue(replaced.advanceExact(5));
assertEquals(4L, replaced.longValue());
}
private static NumericDoubleValues asNumericDoubleValues(Double... values) {
return new NumericDoubleValues() {
int docID = -1;
@Override
public boolean advanceExact(int target) throws IOException {
docID = target;
return target < values.length && values[target] != null;
}
@Override
public double doubleValue() throws IOException {
return values[docID];
}
};
}
public void testReplaceMissingDoubles() throws IOException {
final NumericDoubleValues values = asNumericDoubleValues(null, 1.3, 1.2, null, 1.5, null);
final NumericDoubleValues replaced = FieldData.replaceMissing(values, 1.4);
assertTrue(replaced.advanceExact(0));
assertEquals(1.4, replaced.doubleValue(), 0d);
assertTrue(replaced.advanceExact(1));
assertEquals(1.3, replaced.doubleValue(), 0d);
assertTrue(replaced.advanceExact(2));
assertEquals(1.2, replaced.doubleValue(), 0d);
assertTrue(replaced.advanceExact(3));
assertEquals(1.4, replaced.doubleValue(), 0d);
assertTrue(replaced.advanceExact(4));
assertEquals(1.5, replaced.doubleValue(), 0d);
assertTrue(replaced.advanceExact(5));
assertEquals(1.4, replaced.doubleValue(), 0d);
}
}

View File

@ -18,6 +18,7 @@
*/
package org.elasticsearch.index.fielddata.ordinals;
import org.apache.lucene.index.DocValues;
import org.apache.lucene.index.SortedDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.util.packed.PackedInts;
@ -261,7 +262,7 @@ public class MultiOrdinalsTests extends ESTestCase {
}
}
assertThat(docs.getValueCount(), equalTo(maxOrd));
assertThat(FieldData.isMultiValued(docs), equalTo(true));
assertNull(DocValues.unwrapSingleton(docs));
for (int doc = 0; doc < ordinalPlan.length; ++doc) {
long[] ords = ordinalPlan[doc];
assertEquals(ords.length > 0, docs.advanceExact(doc));

View File

@ -119,6 +119,7 @@ public class MatchPhrasePrefixQueryBuilderTests extends AbstractQueryTestCase<Ma
}
public void testPhraseOnFieldWithNoTerms() {
assumeTrue("test runs only when at least a type is registered", getCurrentTypes().length > 0);
MatchPhrasePrefixQueryBuilder matchQuery = new MatchPhrasePrefixQueryBuilder(DATE_FIELD_NAME, "three term phrase");
matchQuery.analyzer("whitespace");
expectThrows(IllegalArgumentException.class, () -> matchQuery.doToQuery(createShardContext()));

View File

@ -0,0 +1,102 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.store;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FilterDirectory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
import java.io.IOException;
public class ByteSizeCachingDirectoryTests extends ESTestCase {
private static class LengthCountingDirectory extends FilterDirectory {
int numFileLengthCalls;
LengthCountingDirectory(Directory in) {
super(in);
}
@Override
public long fileLength(String name) throws IOException {
numFileLengthCalls++;
return super.fileLength(name);
}
}
public void testBasics() throws IOException {
try (Directory dir = newDirectory()) {
try (IndexOutput out = dir.createOutput("quux", IOContext.DEFAULT)) {
out.writeBytes(new byte[11], 11);
}
LengthCountingDirectory countingDir = new LengthCountingDirectory(dir);
ByteSizeCachingDirectory cachingDir = new ByteSizeCachingDirectory(countingDir, new TimeValue(0));
assertEquals(11, cachingDir.estimateSizeInBytes());
assertEquals(11, cachingDir.estimateSizeInBytes());
assertEquals(1, countingDir.numFileLengthCalls);
try (IndexOutput out = cachingDir.createOutput("foo", IOContext.DEFAULT)) {
out.writeBytes(new byte[5], 5);
cachingDir.estimateSizeInBytes();
// +2 because there are 3 files
assertEquals(3, countingDir.numFileLengthCalls);
// An index output is open so no caching
cachingDir.estimateSizeInBytes();
assertEquals(5, countingDir.numFileLengthCalls);
}
assertEquals(16, cachingDir.estimateSizeInBytes());
assertEquals(7, countingDir.numFileLengthCalls);
assertEquals(16, cachingDir.estimateSizeInBytes());
assertEquals(7, countingDir.numFileLengthCalls);
try (IndexOutput out = cachingDir.createTempOutput("bar", "baz", IOContext.DEFAULT)) {
out.writeBytes(new byte[4], 4);
cachingDir.estimateSizeInBytes();
assertEquals(10, countingDir.numFileLengthCalls);
// An index output is open so no caching
cachingDir.estimateSizeInBytes();
assertEquals(13, countingDir.numFileLengthCalls);
}
assertEquals(20, cachingDir.estimateSizeInBytes());
// +3 because there are 3 files
assertEquals(16, countingDir.numFileLengthCalls);
assertEquals(20, cachingDir.estimateSizeInBytes());
assertEquals(16, countingDir.numFileLengthCalls);
cachingDir.deleteFile("foo");
assertEquals(15, cachingDir.estimateSizeInBytes());
// +2 because there are 2 files now
assertEquals(18, countingDir.numFileLengthCalls);
assertEquals(15, cachingDir.estimateSizeInBytes());
assertEquals(18, countingDir.numFileLengthCalls);
}
}
}

View File

@ -151,54 +151,55 @@ public class MultiValueModeTests extends ESTestCase {
}
private void verifySortedNumeric(Supplier<SortedNumericDocValues> supplier, int maxDoc) throws IOException {
for (long missingValue : new long[] { 0, randomLong() }) {
for (MultiValueMode mode : MultiValueMode.values()) {
SortedNumericDocValues values = supplier.get();
final NumericDocValues selected = mode.select(values, missingValue);
for (int i = 0; i < maxDoc; ++i) {
assertTrue(selected.advanceExact(i));
final long actual = selected.longValue();
for (MultiValueMode mode : MultiValueMode.values()) {
SortedNumericDocValues values = supplier.get();
final NumericDocValues selected = mode.select(values);
for (int i = 0; i < maxDoc; ++i) {
Long actual = null;
if (selected.advanceExact(i)) {
actual = selected.longValue();
verifyLongValueCanCalledMoreThanOnce(selected, actual);
}
long expected = 0;
if (values.advanceExact(i) == false) {
expected = missingValue;
Long expected = null;
if (values.advanceExact(i)) {
int numValues = values.docValueCount();
if (mode == MultiValueMode.MAX) {
expected = Long.MIN_VALUE;
} else if (mode == MultiValueMode.MIN) {
expected = Long.MAX_VALUE;
} else {
int numValues = values.docValueCount();
if (mode == MultiValueMode.MAX) {
expected = Long.MIN_VALUE;
expected = 0L;
}
for (int j = 0; j < numValues; ++j) {
if (mode == MultiValueMode.SUM || mode == MultiValueMode.AVG) {
expected += values.nextValue();
} else if (mode == MultiValueMode.MIN) {
expected = Long.MAX_VALUE;
}
for (int j = 0; j < numValues; ++j) {
if (mode == MultiValueMode.SUM || mode == MultiValueMode.AVG) {
expected += values.nextValue();
} else if (mode == MultiValueMode.MIN) {
expected = Math.min(expected, values.nextValue());
} else if (mode == MultiValueMode.MAX) {
expected = Math.max(expected, values.nextValue());
}
}
if (mode == MultiValueMode.AVG) {
expected = numValues > 1 ? Math.round((double)expected/(double)numValues) : expected;
} else if (mode == MultiValueMode.MEDIAN) {
int value = numValues/2;
if (numValues % 2 == 0) {
for (int j = 0; j < value - 1; ++j) {
values.nextValue();
}
expected = Math.round(((double) values.nextValue() + values.nextValue())/2.0);
} else {
for (int j = 0; j < value; ++j) {
values.nextValue();
}
expected = values.nextValue();
}
expected = Math.min(expected, values.nextValue());
} else if (mode == MultiValueMode.MAX) {
expected = Math.max(expected, values.nextValue());
}
}
if (mode == MultiValueMode.AVG) {
expected = numValues > 1 ? Math.round((double)expected/(double)numValues) : expected;
} else if (mode == MultiValueMode.MEDIAN) {
int value = numValues/2;
if (numValues % 2 == 0) {
for (int j = 0; j < value - 1; ++j) {
values.nextValue();
}
expected = Math.round(((double) values.nextValue() + values.nextValue())/2.0);
} else {
for (int j = 0; j < value; ++j) {
values.nextValue();
}
expected = values.nextValue();
}
}
assertEquals(mode.toString() + " docId=" + i, expected, actual);
}
assertEquals(mode.toString() + " docId=" + i, expected, actual);
}
}
}
@ -326,54 +327,54 @@ public class MultiValueModeTests extends ESTestCase {
}
private void verifySortedNumericDouble(Supplier<SortedNumericDoubleValues> supplier, int maxDoc) throws IOException {
for (long missingValue : new long[] { 0, randomLong() }) {
for (MultiValueMode mode : MultiValueMode.values()) {
SortedNumericDoubleValues values = supplier.get();
final NumericDoubleValues selected = mode.select(values, missingValue);
for (int i = 0; i < maxDoc; ++i) {
assertTrue(selected.advanceExact(i));
final double actual = selected.doubleValue();
for (MultiValueMode mode : MultiValueMode.values()) {
SortedNumericDoubleValues values = supplier.get();
final NumericDoubleValues selected = mode.select(values);
for (int i = 0; i < maxDoc; ++i) {
Double actual = null;
if (selected.advanceExact(i)) {
actual = selected.doubleValue();
verifyDoubleValueCanCalledMoreThanOnce(selected, actual);
}
double expected = 0.0;
if (values.advanceExact(i) == false) {
expected = missingValue;
Double expected = null;
if (values.advanceExact(i)) {
int numValues = values.docValueCount();
if (mode == MultiValueMode.MAX) {
expected = Double.NEGATIVE_INFINITY;
} else if (mode == MultiValueMode.MIN) {
expected = Double.POSITIVE_INFINITY;
} else {
int numValues = values.docValueCount();
if (mode == MultiValueMode.MAX) {
expected = Long.MIN_VALUE;
expected = 0d;
}
for (int j = 0; j < numValues; ++j) {
if (mode == MultiValueMode.SUM || mode == MultiValueMode.AVG) {
expected += values.nextValue();
} else if (mode == MultiValueMode.MIN) {
expected = Long.MAX_VALUE;
}
for (int j = 0; j < numValues; ++j) {
if (mode == MultiValueMode.SUM || mode == MultiValueMode.AVG) {
expected += values.nextValue();
} else if (mode == MultiValueMode.MIN) {
expected = Math.min(expected, values.nextValue());
} else if (mode == MultiValueMode.MAX) {
expected = Math.max(expected, values.nextValue());
}
}
if (mode == MultiValueMode.AVG) {
expected = expected/numValues;
} else if (mode == MultiValueMode.MEDIAN) {
int value = numValues/2;
if (numValues % 2 == 0) {
for (int j = 0; j < value - 1; ++j) {
values.nextValue();
}
expected = (values.nextValue() + values.nextValue())/2.0;
} else {
for (int j = 0; j < value; ++j) {
values.nextValue();
}
expected = values.nextValue();
}
expected = Math.min(expected, values.nextValue());
} else if (mode == MultiValueMode.MAX) {
expected = Math.max(expected, values.nextValue());
}
}
if (mode == MultiValueMode.AVG) {
expected = expected/numValues;
} else if (mode == MultiValueMode.MEDIAN) {
int value = numValues/2;
if (numValues % 2 == 0) {
for (int j = 0; j < value - 1; ++j) {
values.nextValue();
}
expected = (values.nextValue() + values.nextValue())/2.0;
} else {
for (int j = 0; j < value; ++j) {
values.nextValue();
}
expected = values.nextValue();
}
}
assertEquals(mode.toString() + " docId=" + i, expected, actual, 0.1);
}
assertEquals(mode.toString() + " docId=" + i, expected, actual);
}
}
}

View File

@ -239,8 +239,6 @@ public class ForecastIT extends MlNativeAutodetectIntegTestCase {
throw e;
}
closeJob(job.getId());
List<ForecastRequestStats> forecastStats = getForecastStats();
assertThat(forecastStats.size(), equalTo(1));
ForecastRequestStats forecastRequestStats = forecastStats.get(0);
@ -248,6 +246,21 @@ public class ForecastIT extends MlNativeAutodetectIntegTestCase {
assertThat(forecastRequestStats.getRecordCount(), equalTo(8000L));
assertThat(forecasts.size(), equalTo(8000));
// run forecast a 2nd time
try {
String forecastId = forecast(job.getId(), TimeValue.timeValueHours(1), null);
waitForecastToFinish(job.getId(), forecastId);
} catch (ElasticsearchStatusException e) {
if (e.getMessage().contains("disk space")) {
throw new ElasticsearchStatusException(
"Test likely fails due to insufficient disk space on test machine, please free up space.", e.status(), e);
}
throw e;
}
closeJob(job.getId());
}
private void createDataWithLotsOfClientIps(TimeValue bucketSpan, Job.Builder job) throws IOException {