From 5288235ca3d8ad8437952aa77fd5b89075f21018 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Mon, 26 Mar 2018 09:51:37 +0200 Subject: [PATCH 01/22] Optimize the composite aggregation for match_all and range queries (#28745) This change refactors the composite aggregation to add an execution mode that visits documents in the order of the values present in the leading source of the composite definition. This mode does not need to visit all documents since it can early terminate the collection when the leading source value is greater than the lowest value in the queue. Instead of collecting the documents in the order of their doc_id, this mode uses the inverted lists (or the bkd tree for numerics) to collect documents in the order of the values present in the leading source. For instance the following aggregation: ``` "composite" : { "sources" : [ { "value1": { "terms" : { "field": "timestamp", "order": "asc" } } } ], "size": 10 } ``` ... can use the field `timestamp` to collect the documents with the 10 lowest values for the field instead of visiting all documents. For composite aggregation with more than one source the execution can early terminate as soon as one of the 10 lowest values produces enough composite buckets. For instance if visiting the first two lowest timestamp created 10 composite buckets we can early terminate the collection since it is guaranteed that the third lowest timestamp cannot create a composite key that compares lower than the one already visited. This mode can execute iff: * The leading source in the composite definition uses an indexed field of type `date` (works also with `date_histogram` source), `integer`, `long` or `keyword`. * The query is a match_all query or a range query over the field that is used as the leading source in the composite definition. * The sort order of the leading source is the natural order (ascending since postings and numerics are sorted in ascending order only). If these conditions are not met this aggregation visits each document like any other agg. --- .../bucket/composite-aggregation.asciidoc | 85 ---- .../test/search.aggregation/230_composite.yml | 1 + .../bucket/composite/BinaryValuesSource.java | 131 ++++++ .../CompositeAggregationBuilder.java | 13 +- .../composite/CompositeAggregationPlugin.java | 36 -- .../bucket/composite/CompositeAggregator.java | 334 ++++++++------- .../CompositeValuesCollectorQueue.java | 247 +++++++++++ .../composite/CompositeValuesComparator.java | 144 ------- .../composite/CompositeValuesSource.java | 400 ------------------ .../CompositeValuesSourceBuilder.java | 52 +-- .../CompositeValuesSourceConfig.java | 23 +- .../DateHistogramValuesSourceBuilder.java | 23 +- .../bucket/composite/DoubleValuesSource.java | 129 ++++++ .../composite/GlobalOrdinalValuesSource.java | 189 +++++++++ .../HistogramValuesSourceBuilder.java | 23 +- .../bucket/composite/LongValuesSource.java | 190 +++++++++ .../composite/PointsSortedDocsProducer.java | 181 ++++++++ .../composite/RoundingValuesSource.java | 6 +- .../SingleDimensionValuesSource.java | 143 +++++++ .../bucket/composite/SortedDocsProducer.java | 108 +++++ .../composite/TermsSortedDocsProducer.java | 79 ++++ .../composite/TermsValuesSourceBuilder.java | 19 +- .../composite/CompositeAggregatorTests.java | 370 +++++++++++----- .../CompositeValuesCollectorQueueTests.java | 330 +++++++++++++++ .../SingleDimensionValuesSourceTests.java | 106 +++++ .../aggregations/AggregatorTestCase.java | 1 - 26 files changed, 2323 insertions(+), 1040 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/BinaryValuesSource.java delete mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationPlugin.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueue.java delete mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesComparator.java delete mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSource.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DoubleValuesSource.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GlobalOrdinalValuesSource.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/LongValuesSource.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/PointsSortedDocsProducer.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/SingleDimensionValuesSource.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/SortedDocsProducer.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsSortedDocsProducer.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueueTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/SingleDimensionValuesSourceTests.java diff --git a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc index 5d46a7f4c4a99..688cf20c5320f 100644 --- a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc @@ -545,88 +545,3 @@ GET /_search } -------------------------------------------------- // TESTRESPONSE[s/\.\.\.//] - -==== Index sorting - -By default this aggregation runs on every document that match the query. -Though if the index sort matches the composite sort this aggregation can optimize -the execution and can skip documents that contain composite buckets that would not -be part of the response. - -For instance the following aggregations: - -[source,js] --------------------------------------------------- -GET /_search -{ - "aggs" : { - "my_buckets": { - "composite" : { - "size": 2, - "sources" : [ - { "date": { "date_histogram": { "field": "timestamp", "interval": "1d", "order": "asc" } } }, - { "product": { "terms": { "field": "product", "order": "asc" } } } - ] - } - } - } -} --------------------------------------------------- -// CONSOLE - -\... is much faster on an index that uses the following sort: - -[source,js] --------------------------------------------------- -PUT twitter -{ - "settings" : { - "index" : { - "sort.field" : ["timestamp", "product"], - "sort.order" : ["asc", "asc"] - } - }, - "mappings": { - "sales": { - "properties": { - "timestamp": { - "type": "date" - }, - "product": { - "type": "keyword" - } - } - } - } -} --------------------------------------------------- -// CONSOLE - -WARNING: The optimization takes effect only if the fields used for sorting are single-valued and follow -the same order as the aggregation (`desc` or `asc`). - -If only the aggregation results are needed it is also better to set the size of the query to 0 -and `track_total_hits` to false in order to remove other slowing factors: - -[source,js] --------------------------------------------------- -GET /_search -{ - "size": 0, - "track_total_hits": false, - "aggs" : { - "my_buckets": { - "composite" : { - "size": 2, - "sources" : [ - { "date": { "date_histogram": { "field": "timestamp", "interval": "1d" } } }, - { "product": { "terms": { "field": "product" } } } - ] - } - } - } -} --------------------------------------------------- -// CONSOLE - -See <> for more details. diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml index b8c89517ec119..10fc9f33beb8b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml @@ -99,6 +99,7 @@ setup: - do: search: index: test + allow_partial_search_results: false body: aggregations: test: diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/BinaryValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/BinaryValuesSource.java new file mode 100644 index 0000000000000..cd46b90889d49 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/BinaryValuesSource.java @@ -0,0 +1,131 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.search.aggregations.bucket.composite; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.CheckedFunction; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.aggregations.LeafBucketCollector; + +import java.io.IOException; + +/** + * A {@link SingleDimensionValuesSource} for binary source ({@link BytesRef}). + */ +class BinaryValuesSource extends SingleDimensionValuesSource { + private final CheckedFunction docValuesFunc; + private final BytesRef[] values; + private BytesRef currentValue; + + BinaryValuesSource(MappedFieldType fieldType, CheckedFunction docValuesFunc, + int size, int reverseMul) { + super(fieldType, size, reverseMul); + this.docValuesFunc = docValuesFunc; + this.values = new BytesRef[size]; + } + + @Override + public void copyCurrent(int slot) { + values[slot] = BytesRef.deepCopyOf(currentValue); + } + + @Override + public int compare(int from, int to) { + return compareValues(values[from], values[to]); + } + + @Override + int compareCurrent(int slot) { + return compareValues(currentValue, values[slot]); + } + + @Override + int compareCurrentWithAfter() { + return compareValues(currentValue, afterValue); + } + + int compareValues(BytesRef v1, BytesRef v2) { + return v1.compareTo(v2) * reverseMul; + } + + @Override + void setAfter(Comparable value) { + if (value.getClass() == BytesRef.class) { + afterValue = (BytesRef) value; + } else if (value.getClass() == String.class) { + afterValue = new BytesRef((String) value); + } else { + throw new IllegalArgumentException("invalid value, expected string, got " + value.getClass().getSimpleName()); + } + } + + @Override + BytesRef toComparable(int slot) { + return values[slot]; + } + + @Override + LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector next) throws IOException { + final SortedBinaryDocValues dvs = docValuesFunc.apply(context); + return new LeafBucketCollector() { + @Override + public void collect(int doc, long bucket) throws IOException { + if (dvs.advanceExact(doc)) { + int num = dvs.docValueCount(); + for (int i = 0; i < num; i++) { + currentValue = dvs.nextValue(); + next.collect(doc, bucket); + } + } + } + }; + } + + @Override + LeafBucketCollector getLeafCollector(Comparable value, LeafReaderContext context, LeafBucketCollector next) { + if (value.getClass() != BytesRef.class) { + throw new IllegalArgumentException("Expected BytesRef, got " + value.getClass()); + } + currentValue = (BytesRef) value; + return new LeafBucketCollector() { + @Override + public void collect(int doc, long bucket) throws IOException { + next.collect(doc, bucket); + } + }; + } + + @Override + SortedDocsProducer createSortedDocsProducerOrNull(IndexReader reader, Query query) { + if (checkIfSortedDocsIsApplicable(reader, fieldType) == false || + (query != null && query.getClass() != MatchAllDocsQuery.class)) { + return null; + } + return new TermsSortedDocsProducer(fieldType.name()); + } + + @Override + public void close() {} +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java index b54371dce62ed..0912555ea711b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationBuilder.java @@ -19,16 +19,12 @@ package org.elasticsearch.search.aggregations.bucket.composite; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.index.IndexSortConfig; -import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; import org.elasticsearch.search.aggregations.AggregationBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories; @@ -154,16 +150,9 @@ protected AggregatorFactory doBuild(SearchContext context, AggregatorFactory< if (parent != null) { throw new IllegalArgumentException("[composite] aggregation cannot be used with a parent aggregation"); } - final QueryShardContext shardContext = context.getQueryShardContext(); CompositeValuesSourceConfig[] configs = new CompositeValuesSourceConfig[sources.size()]; - SortField[] sortFields = new SortField[configs.length]; - IndexSortConfig indexSortConfig = shardContext.getIndexSettings().getIndexSortConfig(); - if (indexSortConfig.hasIndexSort()) { - Sort sort = indexSortConfig.buildIndexSort(shardContext::fieldMapper, shardContext::getForField); - System.arraycopy(sort.getSort(), 0, sortFields, 0, sortFields.length); - } for (int i = 0; i < configs.length; i++) { - configs[i] = sources.get(i).build(context, i, configs.length, sortFields[i]); + configs[i] = sources.get(i).build(context); if (configs[i].valuesSource().needsScores()) { throw new IllegalArgumentException("[sources] cannot access _score"); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationPlugin.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationPlugin.java deleted file mode 100644 index bd49acbe4da0b..0000000000000 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregationPlugin.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket.composite; - -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.plugins.SearchPlugin; - -import java.util.Arrays; -import java.util.List; - -public class CompositeAggregationPlugin extends Plugin implements SearchPlugin { - @Override - public List getAggregations() { - return Arrays.asList( - new AggregationSpec(CompositeAggregationBuilder.NAME, CompositeAggregationBuilder::new, CompositeAggregationBuilder::parse) - .addResultReader(InternalComposite::new) - ); - } -} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java index 830aba3bcf1e1..04864e7419def 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregator.java @@ -19,22 +19,29 @@ package org.elasticsearch.search.aggregations.bucket.composite; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.CollectionTerminatedException; import org.apache.lucene.search.DocIdSet; import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.MultiCollector; import org.apache.lucene.search.Query; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; import org.apache.lucene.util.RoaringDocIdSet; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.BucketCollector; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.LeafBucketCollector; import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; @@ -43,97 +50,74 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.TreeMap; import java.util.stream.Collectors; final class CompositeAggregator extends BucketsAggregator { private final int size; - private final CompositeValuesSourceConfig[] sources; + private final SortedDocsProducer sortedDocsProducer; private final List sourceNames; + private final int[] reverseMuls; private final List formats; - private final boolean canEarlyTerminate; - private final TreeMap keys; - private final CompositeValuesComparator array; + private final CompositeValuesCollectorQueue queue; - private final List contexts = new ArrayList<>(); - private LeafContext leaf; - private RoaringDocIdSet.Builder builder; + private final List entries; + private LeafReaderContext currentLeaf; + private RoaringDocIdSet.Builder docIdSetBuilder; + private BucketCollector deferredCollectors; CompositeAggregator(String name, AggregatorFactories factories, SearchContext context, Aggregator parent, - List pipelineAggregators, Map metaData, - int size, CompositeValuesSourceConfig[] sources, CompositeKey rawAfterKey) throws IOException { + List pipelineAggregators, Map metaData, + int size, CompositeValuesSourceConfig[] sourceConfigs, CompositeKey rawAfterKey) throws IOException { super(name, factories, context, parent, pipelineAggregators, metaData); this.size = size; - this.sources = sources; - this.sourceNames = Arrays.stream(sources).map(CompositeValuesSourceConfig::name).collect(Collectors.toList()); - this.formats = Arrays.stream(sources).map(CompositeValuesSourceConfig::format).collect(Collectors.toList()); - // we use slot 0 to fill the current document (size+1). - this.array = new CompositeValuesComparator(context.searcher().getIndexReader(), sources, size+1); + this.sourceNames = Arrays.stream(sourceConfigs).map(CompositeValuesSourceConfig::name).collect(Collectors.toList()); + this.reverseMuls = Arrays.stream(sourceConfigs).mapToInt(CompositeValuesSourceConfig::reverseMul).toArray(); + this.formats = Arrays.stream(sourceConfigs).map(CompositeValuesSourceConfig::format).collect(Collectors.toList()); + final SingleDimensionValuesSource[] sources = + createValuesSources(context.bigArrays(), context.searcher().getIndexReader(), context.query(), sourceConfigs, size); + this.queue = new CompositeValuesCollectorQueue(sources, size); + this.sortedDocsProducer = sources[0].createSortedDocsProducerOrNull(context.searcher().getIndexReader(), context.query()); if (rawAfterKey != null) { - array.setTop(rawAfterKey.values()); + queue.setAfter(rawAfterKey.values()); } - this.keys = new TreeMap<>(array::compare); - this.canEarlyTerminate = Arrays.stream(sources) - .allMatch(CompositeValuesSourceConfig::canEarlyTerminate); + this.entries = new ArrayList<>(); } - boolean canEarlyTerminate() { - return canEarlyTerminate; + @Override + protected void doClose() { + Releasables.close(queue); + } + + @Override + protected void doPreCollection() throws IOException { + List collectors = Arrays.asList(subAggregators); + deferredCollectors = BucketCollector.wrap(collectors); + collectableSubAggregators = BucketCollector.NO_OP_COLLECTOR; } - private int[] getReverseMuls() { - return Arrays.stream(sources).mapToInt(CompositeValuesSourceConfig::reverseMul).toArray(); + @Override + protected void doPostCollection() throws IOException { + finishLeaf(); } @Override public InternalAggregation buildAggregation(long zeroBucket) throws IOException { assert zeroBucket == 0L; - consumeBucketsAndMaybeBreak(keys.size()); + consumeBucketsAndMaybeBreak(queue.size()); - // Replay all documents that contain at least one top bucket (collected during the first pass). - grow(keys.size()+1); - final boolean needsScores = needsScores(); - Weight weight = null; - if (needsScores) { - Query query = context.query(); - weight = context.searcher().createNormalizedWeight(query, true); - } - for (LeafContext context : contexts) { - DocIdSetIterator docIdSetIterator = context.docIdSet.iterator(); - if (docIdSetIterator == null) { - continue; - } - final CompositeValuesSource.Collector collector = - array.getLeafCollector(context.ctx, getSecondPassCollector(context.subCollector)); - int docID; - DocIdSetIterator scorerIt = null; - if (needsScores) { - Scorer scorer = weight.scorer(context.ctx); - // We don't need to check if the scorer is null - // since we are sure that there are documents to replay (docIdSetIterator it not empty). - scorerIt = scorer.iterator(); - context.subCollector.setScorer(scorer); - } - while ((docID = docIdSetIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { - if (needsScores) { - assert scorerIt.docID() < docID; - scorerIt.advance(docID); - // aggregations should only be replayed on matching documents - assert scorerIt.docID() == docID; - } - collector.collect(docID); - } + if (deferredCollectors != NO_OP_COLLECTOR) { + // Replay all documents that contain at least one top bucket (collected during the first pass). + runDeferredCollections(); } - int num = Math.min(size, keys.size()); + int num = Math.min(size, queue.size()); final InternalComposite.InternalBucket[] buckets = new InternalComposite.InternalBucket[num]; - final int[] reverseMuls = getReverseMuls(); int pos = 0; - for (int slot : keys.keySet()) { - CompositeKey key = array.toCompositeKey(slot); + for (int slot : queue.getSortedSlot()) { + CompositeKey key = queue.toCompositeKey(slot); InternalAggregations aggs = bucketAggregations(slot); - int docCount = bucketDocCount(slot); + int docCount = queue.getDocCount(slot); buckets[pos++] = new InternalComposite.InternalBucket(sourceNames, formats, key, reverseMuls, docCount, aggs); } CompositeKey lastBucket = num > 0 ? buckets[num-1].getRawKey() : null; @@ -143,125 +127,179 @@ public InternalAggregation buildAggregation(long zeroBucket) throws IOException @Override public InternalAggregation buildEmptyAggregation() { - final int[] reverseMuls = getReverseMuls(); return new InternalComposite(name, size, sourceNames, formats, Collections.emptyList(), null, reverseMuls, pipelineAggregators(), metaData()); } - @Override - protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { - if (leaf != null) { - leaf.docIdSet = builder.build(); - contexts.add(leaf); + private void finishLeaf() { + if (currentLeaf != null) { + DocIdSet docIdSet = docIdSetBuilder.build(); + entries.add(new Entry(currentLeaf, docIdSet)); + currentLeaf = null; + docIdSetBuilder = null; } - leaf = new LeafContext(ctx, sub); - builder = new RoaringDocIdSet.Builder(ctx.reader().maxDoc()); - final CompositeValuesSource.Collector inner = array.getLeafCollector(ctx, getFirstPassCollector()); - return new LeafBucketCollector() { - @Override - public void collect(int doc, long zeroBucket) throws IOException { - assert zeroBucket == 0L; - inner.collect(doc); - } - }; } @Override - protected void doPostCollection() throws IOException { - if (leaf != null) { - leaf.docIdSet = builder.build(); - contexts.add(leaf); + protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { + finishLeaf(); + boolean fillDocIdSet = deferredCollectors != NO_OP_COLLECTOR; + if (sortedDocsProducer != null) { + /** + * The producer will visit documents sorted by the leading source of the composite definition + * and terminates when the leading source value is guaranteed to be greater than the lowest + * composite bucket in the queue. + */ + DocIdSet docIdSet = sortedDocsProducer.processLeaf(context.query(), queue, ctx, fillDocIdSet); + if (fillDocIdSet) { + entries.add(new Entry(ctx, docIdSet)); + } + + /** + * We can bypass search entirely for this segment, all the processing has been done in the previous call. + * Throwing this exception will terminate the execution of the search for this root aggregation, + * see {@link MultiCollector} for more details on how we handle early termination in aggregations. + */ + throw new CollectionTerminatedException(); + } else { + if (fillDocIdSet) { + currentLeaf = ctx; + docIdSetBuilder = new RoaringDocIdSet.Builder(ctx.reader().maxDoc()); + } + final LeafBucketCollector inner = queue.getLeafCollector(ctx, getFirstPassCollector(docIdSetBuilder)); + return new LeafBucketCollector() { + @Override + public void collect(int doc, long zeroBucket) throws IOException { + assert zeroBucket == 0L; + inner.collect(doc); + } + }; } } /** - * The first pass selects the top N composite buckets from all matching documents. - * It also records all doc ids that contain a top N composite bucket in a {@link RoaringDocIdSet} in order to be - * able to replay the collection filtered on the best buckets only. + * The first pass selects the top composite buckets from all matching documents. */ - private CompositeValuesSource.Collector getFirstPassCollector() { - return new CompositeValuesSource.Collector() { + private LeafBucketCollector getFirstPassCollector(RoaringDocIdSet.Builder builder) { + return new LeafBucketCollector() { int lastDoc = -1; @Override - public void collect(int doc) throws IOException { - - // Checks if the candidate key in slot 0 is competitive. - if (keys.containsKey(0)) { - // This key is already in the top N, skip it for now. - if (doc != lastDoc) { + public void collect(int doc, long bucket) throws IOException { + int slot = queue.addIfCompetitive(); + if (slot != -1) { + if (builder != null && lastDoc != doc) { builder.add(doc); lastDoc = doc; } - return; - } - if (array.hasTop() && array.compareTop(0) <= 0) { - // This key is greater than the top value collected in the previous round. - if (canEarlyTerminate) { - // The index sort matches the composite sort, we can early terminate this segment. - throw new CollectionTerminatedException(); - } - // just skip this key for now - return; - } - if (keys.size() >= size) { - // The tree map is full, check if the candidate key should be kept. - if (array.compare(0, keys.lastKey()) > 0) { - // The candidate key is not competitive - if (canEarlyTerminate) { - // The index sort matches the composite sort, we can early terminate this segment. - throw new CollectionTerminatedException(); - } - // just skip this key - return; - } } + } + }; + } - // The candidate key is competitive - final int newSlot; - if (keys.size() >= size) { - // the tree map is full, we replace the last key with this candidate. - int slot = keys.pollLastEntry().getKey(); - // and we recycle the deleted slot - newSlot = slot; - } else { - newSlot = keys.size() + 1; + /** + * Replay the documents that might contain a top bucket and pass top buckets to + * the {@link this#deferredCollectors}. + */ + private void runDeferredCollections() throws IOException { + final boolean needsScores = needsScores(); + Weight weight = null; + if (needsScores) { + Query query = context.query(); + weight = context.searcher().createNormalizedWeight(query, true); + } + deferredCollectors.preCollection(); + for (Entry entry : entries) { + DocIdSetIterator docIdSetIterator = entry.docIdSet.iterator(); + if (docIdSetIterator == null) { + continue; + } + final LeafBucketCollector subCollector = deferredCollectors.getLeafCollector(entry.context); + final LeafBucketCollector collector = queue.getLeafCollector(entry.context, getSecondPassCollector(subCollector)); + DocIdSetIterator scorerIt = null; + if (needsScores) { + Scorer scorer = weight.scorer(entry.context); + if (scorer != null) { + scorerIt = scorer.iterator(); + subCollector.setScorer(scorer); } - // move the candidate key to its new slot. - array.move(0, newSlot); - keys.put(newSlot, newSlot); - if (doc != lastDoc) { - builder.add(doc); - lastDoc = doc; + } + int docID; + while ((docID = docIdSetIterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) { + if (needsScores) { + assert scorerIt != null && scorerIt.docID() < docID; + scorerIt.advance(docID); + // aggregations should only be replayed on matching documents + assert scorerIt.docID() == docID; } + collector.collect(docID); } - }; + } + deferredCollectors.postCollection(); } - /** - * The second pass delegates the collection to sub-aggregations but only if the collected composite bucket is a top bucket (selected - * in the first pass). + * Replay the top buckets from the matching documents. */ - private CompositeValuesSource.Collector getSecondPassCollector(LeafBucketCollector subCollector) throws IOException { - return doc -> { - Integer bucket = keys.get(0); - if (bucket != null) { - // The candidate key in slot 0 is a top bucket. - // We can defer the collection of this document/bucket to the sub collector - collectExistingBucket(subCollector, doc, bucket); + private LeafBucketCollector getSecondPassCollector(LeafBucketCollector subCollector) { + return new LeafBucketCollector() { + @Override + public void collect(int doc, long zeroBucket) throws IOException { + assert zeroBucket == 0; + Integer slot = queue.compareCurrent(); + if (slot != null) { + // The candidate key is a top bucket. + // We can defer the collection of this document/bucket to the sub collector + subCollector.collect(doc, slot); + } } }; } - static class LeafContext { - final LeafReaderContext ctx; - final LeafBucketCollector subCollector; - DocIdSet docIdSet; + private static SingleDimensionValuesSource[] createValuesSources(BigArrays bigArrays, IndexReader reader, Query query, + CompositeValuesSourceConfig[] configs, int size) { + final SingleDimensionValuesSource[] sources = new SingleDimensionValuesSource[configs.length]; + for (int i = 0; i < sources.length; i++) { + final int reverseMul = configs[i].reverseMul(); + if (configs[i].valuesSource() instanceof ValuesSource.Bytes.WithOrdinals && reader instanceof DirectoryReader) { + ValuesSource.Bytes.WithOrdinals vs = (ValuesSource.Bytes.WithOrdinals) configs[i].valuesSource(); + sources[i] = new GlobalOrdinalValuesSource(bigArrays, configs[i].fieldType(), vs::globalOrdinalsValues, size, reverseMul); + if (i == 0 && sources[i].createSortedDocsProducerOrNull(reader, query) != null) { + // this the leading source and we can optimize it with the sorted docs producer but + // we don't want to use global ordinals because the number of visited documents + // should be low and global ordinals need one lookup per visited term. + Releasables.close(sources[i]); + sources[i] = new BinaryValuesSource(configs[i].fieldType(), vs::bytesValues, size, reverseMul); + } + } else if (configs[i].valuesSource() instanceof ValuesSource.Bytes) { + ValuesSource.Bytes vs = (ValuesSource.Bytes) configs[i].valuesSource(); + sources[i] = new BinaryValuesSource(configs[i].fieldType(), vs::bytesValues, size, reverseMul); + } else if (configs[i].valuesSource() instanceof ValuesSource.Numeric) { + final ValuesSource.Numeric vs = (ValuesSource.Numeric) configs[i].valuesSource(); + if (vs.isFloatingPoint()) { + sources[i] = new DoubleValuesSource(bigArrays, configs[i].fieldType(), vs::doubleValues, size, reverseMul); + } else { + if (vs instanceof RoundingValuesSource) { + sources[i] = new LongValuesSource(bigArrays, configs[i].fieldType(), vs::longValues, + ((RoundingValuesSource) vs)::round, configs[i].format(), size, reverseMul); + } else { + sources[i] = new LongValuesSource(bigArrays, configs[i].fieldType(), vs::longValues, + (value) -> value, configs[i].format(), size, reverseMul); + } + } + } + } + return sources; + } + + private static class Entry { + final LeafReaderContext context; + final DocIdSet docIdSet; - LeafContext(LeafReaderContext ctx, LeafBucketCollector subCollector) { - this.ctx = ctx; - this.subCollector = subCollector; + Entry(LeafReaderContext context, DocIdSet docIdSet) { + this.context = context; + this.docIdSet = docIdSet; } } } + diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueue.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueue.java new file mode 100644 index 0000000000000..5be4508612ece --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueue.java @@ -0,0 +1,247 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.index.LeafReaderContext; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.search.aggregations.LeafBucketCollector; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Set; +import java.util.TreeMap; + +/** + * A specialized queue implementation for composite buckets + */ +final class CompositeValuesCollectorQueue implements Releasable { + // the slot for the current candidate + private static final int CANDIDATE_SLOT = Integer.MAX_VALUE; + + private final int maxSize; + private final TreeMap keys; + private final SingleDimensionValuesSource[] arrays; + private final int[] docCounts; + private boolean afterValueSet = false; + + /** + * Constructs a composite queue with the specified size and sources. + * + * @param sources The list of {@link CompositeValuesSourceConfig} to build the composite buckets. + * @param size The number of composite buckets to keep. + */ + CompositeValuesCollectorQueue(SingleDimensionValuesSource[] sources, int size) { + this.maxSize = size; + this.arrays = sources; + this.docCounts = new int[size]; + this.keys = new TreeMap<>(this::compare); + } + + void clear() { + keys.clear(); + Arrays.fill(docCounts, 0); + afterValueSet = false; + } + + /** + * The current size of the queue. + */ + int size() { + return keys.size(); + } + + /** + * Whether the queue is full or not. + */ + boolean isFull() { + return keys.size() == maxSize; + } + + /** + * Returns a sorted {@link Set} view of the slots contained in this queue. + */ + Set getSortedSlot() { + return keys.keySet(); + } + + /** + * Compares the current candidate with the values in the queue and returns + * the slot if the candidate is already in the queue or null if the candidate is not present. + */ + Integer compareCurrent() { + return keys.get(CANDIDATE_SLOT); + } + + /** + * Returns the lowest value (exclusive) of the leading source. + */ + Comparable getLowerValueLeadSource() { + return afterValueSet ? arrays[0].getAfter() : null; + } + + /** + * Returns the upper value (inclusive) of the leading source. + */ + Comparable getUpperValueLeadSource() throws IOException { + return size() >= maxSize ? arrays[0].toComparable(keys.lastKey()) : null; + } + /** + * Returns the document count in slot. + */ + int getDocCount(int slot) { + return docCounts[slot]; + } + + /** + * Copies the current value in slot. + */ + private void copyCurrent(int slot) { + for (int i = 0; i < arrays.length; i++) { + arrays[i].copyCurrent(slot); + } + docCounts[slot] = 1; + } + + /** + * Compares the values in slot1 with slot2. + */ + int compare(int slot1, int slot2) { + for (int i = 0; i < arrays.length; i++) { + int cmp = (slot1 == CANDIDATE_SLOT) ? arrays[i].compareCurrent(slot2) : + arrays[i].compare(slot1, slot2); + if (cmp != 0) { + return cmp; + } + } + return 0; + } + + /** + * Sets the after values for this comparator. + */ + void setAfter(Comparable[] values) { + assert values.length == arrays.length; + afterValueSet = true; + for (int i = 0; i < arrays.length; i++) { + arrays[i].setAfter(values[i]); + } + } + + /** + * Compares the after values with the values in slot. + */ + private int compareCurrentWithAfter() { + for (int i = 0; i < arrays.length; i++) { + int cmp = arrays[i].compareCurrentWithAfter(); + if (cmp != 0) { + return cmp; + } + } + return 0; + } + + /** + * Builds the {@link CompositeKey} for slot. + */ + CompositeKey toCompositeKey(int slot) throws IOException { + assert slot < maxSize; + Comparable[] values = new Comparable[arrays.length]; + for (int i = 0; i < values.length; i++) { + values[i] = arrays[i].toComparable(slot); + } + return new CompositeKey(values); + } + + /** + * Creates the collector that will visit the composite buckets of the matching documents. + * The provided collector in is called on each composite bucket. + */ + LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector in) throws IOException { + return getLeafCollector(null, context, in); + } + /** + * Creates the collector that will visit the composite buckets of the matching documents. + * If forceLeadSourceValue is not null, the leading source will use this value + * for each document. + * The provided collector in is called on each composite bucket. + */ + LeafBucketCollector getLeafCollector(Comparable forceLeadSourceValue, + LeafReaderContext context, LeafBucketCollector in) throws IOException { + int last = arrays.length - 1; + LeafBucketCollector collector = in; + while (last > 0) { + collector = arrays[last--].getLeafCollector(context, collector); + } + if (forceLeadSourceValue != null) { + collector = arrays[last].getLeafCollector(forceLeadSourceValue, context, collector); + } else { + collector = arrays[last].getLeafCollector(context, collector); + } + return collector; + } + + /** + * Check if the current candidate should be added in the queue. + * @return The target slot of the candidate or -1 is the candidate is not competitive. + */ + int addIfCompetitive() { + // checks if the candidate key is competitive + Integer topSlot = compareCurrent(); + if (topSlot != null) { + // this key is already in the top N, skip it + docCounts[topSlot] += 1; + return topSlot; + } + if (afterValueSet && compareCurrentWithAfter() <= 0) { + // this key is greater than the top value collected in the previous round, skip it + return -1; + } + if (keys.size() >= maxSize) { + // the tree map is full, check if the candidate key should be kept + if (compare(CANDIDATE_SLOT, keys.lastKey()) > 0) { + // the candidate key is not competitive, skip it + return -1; + } + } + + // the candidate key is competitive + final int newSlot; + if (keys.size() >= maxSize) { + // the tree map is full, we replace the last key with this candidate + int slot = keys.pollLastEntry().getKey(); + // and we recycle the deleted slot + newSlot = slot; + } else { + newSlot = keys.size(); + assert newSlot < maxSize; + } + // move the candidate key to its new slot + copyCurrent(newSlot); + keys.put(newSlot, newSlot); + return newSlot; + } + + + @Override + public void close() { + Releasables.close(arrays); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesComparator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesComparator.java deleted file mode 100644 index 0ce87460a5429..0000000000000 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesComparator.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.search.aggregations.bucket.composite; - -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.LeafReaderContext; -import org.elasticsearch.search.aggregations.LeafBucketCollector; - -import java.io.IOException; - -import static org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; -import static org.elasticsearch.search.aggregations.support.ValuesSource.Bytes; -import static org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.WithOrdinals; - -final class CompositeValuesComparator { - private final int size; - private final CompositeValuesSource[] arrays; - private boolean topValueSet = false; - - /** - * - * @param sources The list of {@link CompositeValuesSourceConfig} to build the composite buckets. - * @param size The number of composite buckets to keep. - */ - CompositeValuesComparator(IndexReader reader, CompositeValuesSourceConfig[] sources, int size) { - this.size = size; - this.arrays = new CompositeValuesSource[sources.length]; - for (int i = 0; i < sources.length; i++) { - final int reverseMul = sources[i].reverseMul(); - if (sources[i].valuesSource() instanceof WithOrdinals && reader instanceof DirectoryReader) { - WithOrdinals vs = (WithOrdinals) sources[i].valuesSource(); - arrays[i] = CompositeValuesSource.wrapGlobalOrdinals(vs, size, reverseMul); - } else if (sources[i].valuesSource() instanceof Bytes) { - Bytes vs = (Bytes) sources[i].valuesSource(); - arrays[i] = CompositeValuesSource.wrapBinary(vs, size, reverseMul); - } else if (sources[i].valuesSource() instanceof Numeric) { - final Numeric vs = (Numeric) sources[i].valuesSource(); - if (vs.isFloatingPoint()) { - arrays[i] = CompositeValuesSource.wrapDouble(vs, size, reverseMul); - } else { - arrays[i] = CompositeValuesSource.wrapLong(vs, sources[i].format(), size, reverseMul); - } - } - } - } - - /** - * Moves the values in slot1 to slot2. - */ - void move(int slot1, int slot2) { - assert slot1 < size && slot2 < size; - for (int i = 0; i < arrays.length; i++) { - arrays[i].move(slot1, slot2); - } - } - - /** - * Compares the values in slot1 with slot2. - */ - int compare(int slot1, int slot2) { - assert slot1 < size && slot2 < size; - for (int i = 0; i < arrays.length; i++) { - int cmp = arrays[i].compare(slot1, slot2); - if (cmp != 0) { - return cmp; - } - } - return 0; - } - - /** - * Returns true if a top value has been set for this comparator. - */ - boolean hasTop() { - return topValueSet; - } - - /** - * Sets the top values for this comparator. - */ - void setTop(Comparable[] values) { - assert values.length == arrays.length; - topValueSet = true; - for (int i = 0; i < arrays.length; i++) { - arrays[i].setTop(values[i]); - } - } - - /** - * Compares the top values with the values in slot. - */ - int compareTop(int slot) { - assert slot < size; - for (int i = 0; i < arrays.length; i++) { - int cmp = arrays[i].compareTop(slot); - if (cmp != 0) { - return cmp; - } - } - return 0; - } - - /** - * Builds the {@link CompositeKey} for slot. - */ - CompositeKey toCompositeKey(int slot) throws IOException { - assert slot < size; - Comparable[] values = new Comparable[arrays.length]; - for (int i = 0; i < values.length; i++) { - values[i] = arrays[i].toComparable(slot); - } - return new CompositeKey(values); - } - - /** - * Gets the {@link LeafBucketCollector} that will record the composite buckets of the visited documents. - */ - CompositeValuesSource.Collector getLeafCollector(LeafReaderContext context, CompositeValuesSource.Collector in) throws IOException { - int last = arrays.length - 1; - CompositeValuesSource.Collector next = arrays[last].getLeafCollector(context, in); - for (int i = last - 1; i >= 0; i--) { - next = arrays[i].getLeafCollector(context, next); - } - return next; - } -} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSource.java deleted file mode 100644 index 2d0368dfd4d28..0000000000000 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSource.java +++ /dev/null @@ -1,400 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.search.aggregations.bucket.composite; - -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.index.SortedSetDocValues; -import org.apache.lucene.search.LeafCollector; -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.joda.FormatDateTimeFormatter; -import org.elasticsearch.index.fielddata.SortedBinaryDocValues; -import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; -import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.sort.SortOrder; - -import java.io.IOException; - -import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS; - -/** - * A wrapper for {@link ValuesSource} that can record and compare values produced during a collection. - */ -abstract class CompositeValuesSource> { - interface Collector { - void collect(int doc) throws IOException; - } - - protected final VS vs; - protected final int size; - protected final int reverseMul; - protected T topValue; - - /** - * - * @param vs The original {@link ValuesSource}. - * @param size The number of values to record. - * @param reverseMul -1 if the natural order ({@link SortOrder#ASC} should be reversed. - */ - CompositeValuesSource(VS vs, int size, int reverseMul) { - this.vs = vs; - this.size = size; - this.reverseMul = reverseMul; - } - - /** - * The type of this source. - */ - abstract String type(); - - /** - * Moves the value in from in to. - * The value present in to is overridden. - */ - abstract void move(int from, int to); - - /** - * Compares the value in from with the value in to. - */ - abstract int compare(int from, int to); - - /** - * Compares the value in slot with the top value in this source. - */ - abstract int compareTop(int slot); - - /** - * Sets the top value for this source. Values that compares smaller should not be recorded. - */ - abstract void setTop(Comparable value); - - /** - * Transforms the value in slot to a {@link Comparable} object. - */ - abstract Comparable toComparable(int slot) throws IOException; - - /** - * Gets the {@link LeafCollector} that will record the values of the visited documents. - */ - abstract Collector getLeafCollector(LeafReaderContext context, Collector next) throws IOException; - - /** - * Creates a {@link CompositeValuesSource} that generates long values. - */ - static CompositeValuesSource wrapLong(ValuesSource.Numeric vs, DocValueFormat format, - int size, int reverseMul) { - return new LongValuesSource(vs, format, size, reverseMul); - } - - /** - * Creates a {@link CompositeValuesSource} that generates double values. - */ - static CompositeValuesSource wrapDouble(ValuesSource.Numeric vs, int size, int reverseMul) { - return new DoubleValuesSource(vs, size, reverseMul); - } - - /** - * Creates a {@link CompositeValuesSource} that generates binary values. - */ - static CompositeValuesSource wrapBinary(ValuesSource.Bytes vs, int size, int reverseMul) { - return new BinaryValuesSource(vs, size, reverseMul); - } - - /** - * Creates a {@link CompositeValuesSource} that generates global ordinal values. - */ - static CompositeValuesSource wrapGlobalOrdinals(ValuesSource.Bytes.WithOrdinals vs, - int size, - int reverseMul) { - return new GlobalOrdinalValuesSource(vs, size, reverseMul); - } - - /** - * A {@link CompositeValuesSource} for global ordinals - */ - private static class GlobalOrdinalValuesSource extends CompositeValuesSource { - private final long[] values; - private SortedSetDocValues lookup; - private Long topValueGlobalOrd; - private boolean isTopValueInsertionPoint; - - GlobalOrdinalValuesSource(ValuesSource.Bytes.WithOrdinals vs, int size, int reverseMul) { - super(vs, size, reverseMul); - this.values = new long[size]; - } - - @Override - String type() { - return "global_ordinals"; - } - - @Override - void move(int from, int to) { - values[to] = values[from]; - } - - @Override - int compare(int from, int to) { - return Long.compare(values[from], values[to]) * reverseMul; - } - - @Override - int compareTop(int slot) { - int cmp = Long.compare(values[slot], topValueGlobalOrd); - if (cmp == 0 && isTopValueInsertionPoint) { - // the top value is missing in this shard, the comparison is against - // the insertion point of the top value so equality means that the value - // is "after" the insertion point. - return reverseMul; - } - return cmp * reverseMul; - } - - @Override - void setTop(Comparable value) { - if (value instanceof BytesRef) { - topValue = (BytesRef) value; - } else if (value instanceof String) { - topValue = new BytesRef(value.toString()); - } else { - throw new IllegalArgumentException("invalid value, expected string, got " + value.getClass().getSimpleName()); - } - } - - @Override - Comparable toComparable(int slot) throws IOException { - return BytesRef.deepCopyOf(lookup.lookupOrd(values[slot])); - } - - @Override - Collector getLeafCollector(LeafReaderContext context, Collector next) throws IOException { - final SortedSetDocValues dvs = vs.globalOrdinalsValues(context); - if (lookup == null) { - lookup = dvs; - if (topValue != null && topValueGlobalOrd == null) { - topValueGlobalOrd = lookup.lookupTerm(topValue); - if (topValueGlobalOrd < 0) { - // convert negative insert position - topValueGlobalOrd = -topValueGlobalOrd - 1; - isTopValueInsertionPoint = true; - } - } - } - return doc -> { - if (dvs.advanceExact(doc)) { - long ord; - while ((ord = dvs.nextOrd()) != NO_MORE_ORDS) { - values[0] = ord; - next.collect(doc); - } - } - }; - } - } - - /** - * A {@link CompositeValuesSource} for binary source ({@link BytesRef}) - */ - private static class BinaryValuesSource extends CompositeValuesSource { - private final BytesRef[] values; - - BinaryValuesSource(ValuesSource.Bytes vs, int size, int reverseMul) { - super(vs, size, reverseMul); - this.values = new BytesRef[size]; - } - - @Override - String type() { - return "binary"; - } - - @Override - public void move(int from, int to) { - values[to] = BytesRef.deepCopyOf(values[from]); - } - - @Override - public int compare(int from, int to) { - return values[from].compareTo(values[to]) * reverseMul; - } - - @Override - int compareTop(int slot) { - return values[slot].compareTo(topValue) * reverseMul; - } - - @Override - void setTop(Comparable value) { - if (value.getClass() == BytesRef.class) { - topValue = (BytesRef) value; - } else if (value.getClass() == String.class) { - topValue = new BytesRef((String) value); - } else { - throw new IllegalArgumentException("invalid value, expected string, got " + value.getClass().getSimpleName()); - } - } - - @Override - Comparable toComparable(int slot) { - return values[slot]; - } - - @Override - Collector getLeafCollector(LeafReaderContext context, Collector next) throws IOException { - final SortedBinaryDocValues dvs = vs.bytesValues(context); - return doc -> { - if (dvs.advanceExact(doc)) { - int num = dvs.docValueCount(); - for (int i = 0; i < num; i++) { - values[0] = dvs.nextValue(); - next.collect(doc); - } - } - }; - } - } - - /** - * A {@link CompositeValuesSource} for longs. - */ - private static class LongValuesSource extends CompositeValuesSource { - private final long[] values; - // handles "format" for date histogram source - private final DocValueFormat format; - - LongValuesSource(ValuesSource.Numeric vs, DocValueFormat format, int size, int reverseMul) { - super(vs, size, reverseMul); - this.format = format; - this.values = new long[size]; - } - - @Override - String type() { - return "long"; - } - - @Override - void move(int from, int to) { - values[to] = values[from]; - } - - @Override - int compare(int from, int to) { - return Long.compare(values[from], values[to]) * reverseMul; - } - - @Override - int compareTop(int slot) { - return Long.compare(values[slot], topValue) * reverseMul; - } - - @Override - void setTop(Comparable value) { - if (value instanceof Number) { - topValue = ((Number) value).longValue(); - } else { - // for date histogram source with "format", the after value is formatted - // as a string so we need to retrieve the original value in milliseconds. - topValue = format.parseLong(value.toString(), false, () -> { - throw new IllegalArgumentException("now() is not supported in [after] key"); - }); - } - } - - @Override - Comparable toComparable(int slot) { - return values[slot]; - } - - @Override - Collector getLeafCollector(LeafReaderContext context, Collector next) throws IOException { - final SortedNumericDocValues dvs = vs.longValues(context); - return doc -> { - if (dvs.advanceExact(doc)) { - int num = dvs.docValueCount(); - for (int i = 0; i < num; i++) { - values[0] = dvs.nextValue(); - next.collect(doc); - } - } - }; - } - } - - /** - * A {@link CompositeValuesSource} for doubles. - */ - private static class DoubleValuesSource extends CompositeValuesSource { - private final double[] values; - - DoubleValuesSource(ValuesSource.Numeric vs, int size, int reverseMul) { - super(vs, size, reverseMul); - this.values = new double[size]; - } - - @Override - String type() { - return "long"; - } - - @Override - void move(int from, int to) { - values[to] = values[from]; - } - - @Override - int compare(int from, int to) { - return Double.compare(values[from], values[to]) * reverseMul; - } - - @Override - int compareTop(int slot) { - return Double.compare(values[slot], topValue) * reverseMul; - } - - @Override - void setTop(Comparable value) { - if (value instanceof Number) { - topValue = ((Number) value).doubleValue(); - } else { - topValue = Double.parseDouble(value.toString()); - } - } - - @Override - Comparable toComparable(int slot) { - return values[slot]; - } - - @Override - Collector getLeafCollector(LeafReaderContext context, Collector next) throws IOException { - final SortedNumericDoubleValues dvs = vs.doubleValues(context); - return doc -> { - if (dvs.advanceExact(doc)) { - int num = dvs.docValueCount(); - for (int i = 0; i < num; i++) { - values[0] = dvs.nextValue(); - next.collect(doc); - } - } - }; - } - } -} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceBuilder.java index 2e06d7c9fe30b..d19729293a912 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceBuilder.java @@ -19,19 +19,13 @@ package org.elasticsearch.search.aggregations.bucket.composite; -import org.apache.lucene.index.DocValues; -import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.index.SortedSetDocValues; -import org.apache.lucene.search.SortField; import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.index.IndexSortConfig; +import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.script.Script; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; @@ -291,46 +285,18 @@ public String format() { * * @param context The search context for this source. * @param config The {@link ValuesSourceConfig} for this source. - * @param pos The position of this source in the composite key. - * @param numPos The total number of positions in the composite key. - * @param sortField The {@link SortField} of the index sort at this position or null if not present. */ - protected abstract CompositeValuesSourceConfig innerBuild(SearchContext context, - ValuesSourceConfig config, - int pos, - int numPos, - SortField sortField) throws IOException; + protected abstract CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig config) throws IOException; - public final CompositeValuesSourceConfig build(SearchContext context, int pos, int numPos, SortField sortField) throws IOException { + public final CompositeValuesSourceConfig build(SearchContext context) throws IOException { ValuesSourceConfig config = ValuesSourceConfig.resolve(context.getQueryShardContext(), valueType, field, script, missing, null, format); - return innerBuild(context, config, pos, numPos, sortField); - } - - protected boolean checkCanEarlyTerminate(IndexReader reader, - String fieldName, - boolean reverse, - SortField sortField) throws IOException { - return sortField.getField().equals(fieldName) && - sortField.getReverse() == reverse && - isSingleValued(reader, sortField); - } - - private static boolean isSingleValued(IndexReader reader, SortField field) throws IOException { - SortField.Type type = IndexSortConfig.getSortFieldType(field); - for (LeafReaderContext context : reader.leaves()) { - if (type == SortField.Type.STRING) { - final SortedSetDocValues values = DocValues.getSortedSet(context.reader(), field.getField()); - if (values.cost() > 0 && DocValues.unwrapSingleton(values) == null) { - return false; - } - } else { - final SortedNumericDocValues values = DocValues.getSortedNumeric(context.reader(), field.getField()); - if (values.cost() > 0 && DocValues.unwrapSingleton(values) == null) { - return false; - } - } + if (config.unmapped() && field != null && config.missing() == null) { + // this source cannot produce any values so we refuse to build + // since composite buckets are not created on null values + throw new QueryShardException(context.getQueryShardContext(), + "failed to find field [" + field + "] and [missing] is not provided"); } - return true; + return innerBuild(context, config); } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceConfig.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceConfig.java index ee70d3f39a550..ca4b38dc1f4d5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceConfig.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesSourceConfig.java @@ -19,22 +19,25 @@ package org.elasticsearch.search.aggregations.bucket.composite; +import org.elasticsearch.common.inject.internal.Nullable; +import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.sort.SortOrder; class CompositeValuesSourceConfig { private final String name; + @Nullable + private final MappedFieldType fieldType; private final ValuesSource vs; private final DocValueFormat format; private final int reverseMul; - private final boolean canEarlyTerminate; - CompositeValuesSourceConfig(String name, ValuesSource vs, DocValueFormat format, SortOrder order, boolean canEarlyTerminate) { + CompositeValuesSourceConfig(String name, @Nullable MappedFieldType fieldType, ValuesSource vs, DocValueFormat format, SortOrder order) { this.name = name; + this.fieldType = fieldType; this.vs = vs; this.format = format; - this.canEarlyTerminate = canEarlyTerminate; this.reverseMul = order == SortOrder.ASC ? 1 : -1; } @@ -45,6 +48,13 @@ String name() { return name; } + /** + * Returns the {@link MappedFieldType} for this config. + */ + MappedFieldType fieldType() { + return fieldType; + } + /** * Returns the {@link ValuesSource} for this configuration. */ @@ -67,11 +77,4 @@ int reverseMul() { assert reverseMul == -1 || reverseMul == 1; return reverseMul; } - - /** - * Returns whether this {@link ValuesSource} is used to sort the index. - */ - boolean canEarlyTerminate() { - return canEarlyTerminate; - } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DateHistogramValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DateHistogramValuesSourceBuilder.java index caa16b7a9d57f..fb2999bbd0b33 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DateHistogramValuesSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DateHistogramValuesSourceBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.composite; -import org.apache.lucene.search.SortField; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -29,9 +28,9 @@ import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.script.Script; import org.elasticsearch.search.DocValueFormat; -import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; import org.elasticsearch.search.aggregations.support.FieldContext; @@ -39,7 +38,6 @@ import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.sort.SortOrder; import org.joda.time.DateTimeZone; import java.io.IOException; @@ -217,11 +215,7 @@ private Rounding createRounding() { } @Override - protected CompositeValuesSourceConfig innerBuild(SearchContext context, - ValuesSourceConfig config, - int pos, - int numPos, - SortField sortField) throws IOException { + protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig config) throws IOException { Rounding rounding = createRounding(); ValuesSource orig = config.toValuesSource(context.getQueryShardContext()); if (orig == null) { @@ -230,19 +224,10 @@ protected CompositeValuesSourceConfig innerBuild(SearchContext context, if (orig instanceof ValuesSource.Numeric) { ValuesSource.Numeric numeric = (ValuesSource.Numeric) orig; RoundingValuesSource vs = new RoundingValuesSource(numeric, rounding); - boolean canEarlyTerminate = false; - final FieldContext fieldContext = config.fieldContext(); - if (sortField != null && - pos == numPos-1 && - fieldContext != null) { - canEarlyTerminate = checkCanEarlyTerminate(context.searcher().getIndexReader(), - fieldContext.field(), order() == SortOrder.ASC ? false : true, sortField); - } - // dates are returned as timestamp in milliseconds-since-the-epoch unless a specific date format // is specified in the builder. final DocValueFormat docValueFormat = format() == null ? DocValueFormat.RAW : config.format(); - return new CompositeValuesSourceConfig(name, vs, docValueFormat, - order(), canEarlyTerminate); + final MappedFieldType fieldType = config.fieldContext() != null ? config.fieldContext().fieldType() : null; + return new CompositeValuesSourceConfig(name, fieldType, vs, docValueFormat, order()); } else { throw new IllegalArgumentException("invalid source, expected numeric, got " + orig.getClass().getSimpleName()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DoubleValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DoubleValuesSource.java new file mode 100644 index 0000000000000..baf63a8d65fee --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/DoubleValuesSource.java @@ -0,0 +1,129 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Query; +import org.elasticsearch.common.CheckedFunction; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.DoubleArray; +import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.aggregations.LeafBucketCollector; + +import java.io.IOException; + +/** + * A {@link SingleDimensionValuesSource} for doubles. + */ +class DoubleValuesSource extends SingleDimensionValuesSource { + private final CheckedFunction docValuesFunc; + private final DoubleArray values; + private double currentValue; + + DoubleValuesSource(BigArrays bigArrays, MappedFieldType fieldType, + CheckedFunction docValuesFunc, + int size, int reverseMul) { + super(fieldType, size, reverseMul); + this.docValuesFunc = docValuesFunc; + this.values = bigArrays.newDoubleArray(size, false); + } + + @Override + void copyCurrent(int slot) { + values.set(slot, currentValue); + } + + @Override + int compare(int from, int to) { + return compareValues(values.get(from), values.get(to)); + } + + @Override + int compareCurrent(int slot) { + return compareValues(currentValue, values.get(slot)); + } + + @Override + int compareCurrentWithAfter() { + return compareValues(currentValue, afterValue); + } + + private int compareValues(double v1, double v2) { + return Double.compare(v1, v2) * reverseMul; + } + + @Override + void setAfter(Comparable value) { + if (value instanceof Number) { + afterValue = ((Number) value).doubleValue(); + } else { + afterValue = Double.parseDouble(value.toString()); + } + } + + @Override + Double toComparable(int slot) { + return values.get(slot); + } + + @Override + LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector next) throws IOException { + final SortedNumericDoubleValues dvs = docValuesFunc.apply(context); + return new LeafBucketCollector() { + @Override + public void collect(int doc, long bucket) throws IOException { + if (dvs.advanceExact(doc)) { + int num = dvs.docValueCount(); + for (int i = 0; i < num; i++) { + currentValue = dvs.nextValue(); + next.collect(doc, bucket); + } + } + } + }; + } + + @Override + LeafBucketCollector getLeafCollector(Comparable value, LeafReaderContext context, LeafBucketCollector next) { + if (value.getClass() != Double.class) { + throw new IllegalArgumentException("Expected Double, got " + value.getClass()); + } + currentValue = (Double) value; + return new LeafBucketCollector() { + @Override + public void collect(int doc, long bucket) throws IOException { + next.collect(doc, bucket); + } + }; + } + + @Override + SortedDocsProducer createSortedDocsProducerOrNull(IndexReader reader, Query query) { + return null; + } + + @Override + public void close() { + Releasables.close(values); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GlobalOrdinalValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GlobalOrdinalValuesSource.java new file mode 100644 index 0000000000000..e3ae3dca1bd63 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/GlobalOrdinalValuesSource.java @@ -0,0 +1,189 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedSetDocValues; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.CheckedFunction; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.aggregations.LeafBucketCollector; + +import java.io.IOException; + +import static org.apache.lucene.index.SortedSetDocValues.NO_MORE_ORDS; + +/** + * A {@link SingleDimensionValuesSource} for global ordinals. + */ +class GlobalOrdinalValuesSource extends SingleDimensionValuesSource { + private final CheckedFunction docValuesFunc; + private final LongArray values; + private SortedSetDocValues lookup; + private long currentValue; + private Long afterValueGlobalOrd; + private boolean isTopValueInsertionPoint; + + private long lastLookupOrd = -1; + private BytesRef lastLookupValue; + + GlobalOrdinalValuesSource(BigArrays bigArrays, + MappedFieldType type, CheckedFunction docValuesFunc, + int size, int reverseMul) { + super(type, size, reverseMul); + this.docValuesFunc = docValuesFunc; + this.values = bigArrays.newLongArray(size, false); + } + + @Override + void copyCurrent(int slot) { + values.set(slot, currentValue); + } + + @Override + int compare(int from, int to) { + return Long.compare(values.get(from), values.get(to)) * reverseMul; + } + + @Override + int compareCurrent(int slot) { + return Long.compare(currentValue, values.get(slot)) * reverseMul; + } + + @Override + int compareCurrentWithAfter() { + int cmp = Long.compare(currentValue, afterValueGlobalOrd); + if (cmp == 0 && isTopValueInsertionPoint) { + // the top value is missing in this shard, the comparison is against + // the insertion point of the top value so equality means that the value + // is "after" the insertion point. + return reverseMul; + } + return cmp * reverseMul; + } + + @Override + void setAfter(Comparable value) { + if (value instanceof BytesRef) { + afterValue = (BytesRef) value; + } else if (value instanceof String) { + afterValue = new BytesRef(value.toString()); + } else { + throw new IllegalArgumentException("invalid value, expected string, got " + value.getClass().getSimpleName()); + } + } + + @Override + BytesRef toComparable(int slot) throws IOException { + long globalOrd = values.get(slot); + if (globalOrd == lastLookupOrd) { + return lastLookupValue; + } else { + lastLookupOrd= globalOrd; + lastLookupValue = BytesRef.deepCopyOf(lookup.lookupOrd(values.get(slot))); + return lastLookupValue; + } + } + + @Override + LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector next) throws IOException { + final SortedSetDocValues dvs = docValuesFunc.apply(context); + if (lookup == null) { + initLookup(dvs); + } + return new LeafBucketCollector() { + @Override + public void collect(int doc, long bucket) throws IOException { + if (dvs.advanceExact(doc)) { + long ord; + while ((ord = dvs.nextOrd()) != NO_MORE_ORDS) { + currentValue = ord; + next.collect(doc, bucket); + } + } + } + }; + } + + @Override + LeafBucketCollector getLeafCollector(Comparable value, LeafReaderContext context, LeafBucketCollector next) throws IOException { + if (value.getClass() != BytesRef.class) { + throw new IllegalArgumentException("Expected BytesRef, got " + value.getClass()); + } + BytesRef term = (BytesRef) value; + final SortedSetDocValues dvs = docValuesFunc.apply(context); + if (lookup == null) { + initLookup(dvs); + } + return new LeafBucketCollector() { + boolean currentValueIsSet = false; + + @Override + public void collect(int doc, long bucket) throws IOException { + if (!currentValueIsSet) { + if (dvs.advanceExact(doc)) { + long ord; + while ((ord = dvs.nextOrd()) != NO_MORE_ORDS) { + if (term.equals(lookup.lookupOrd(ord))) { + currentValueIsSet = true; + currentValue = ord; + break; + } + } + } + } + assert currentValueIsSet; + next.collect(doc, bucket); + } + }; + } + + @Override + SortedDocsProducer createSortedDocsProducerOrNull(IndexReader reader, Query query) { + if (checkIfSortedDocsIsApplicable(reader, fieldType) == false || + (query != null && query.getClass() != MatchAllDocsQuery.class)) { + return null; + } + return new TermsSortedDocsProducer(fieldType.name()); + } + + @Override + public void close() { + Releasables.close(values); + } + + private void initLookup(SortedSetDocValues dvs) throws IOException { + lookup = dvs; + if (afterValue != null && afterValueGlobalOrd == null) { + afterValueGlobalOrd = lookup.lookupTerm(afterValue); + if (afterValueGlobalOrd < 0) { + // convert negative insert position + afterValueGlobalOrd = -afterValueGlobalOrd - 1; + isTopValueInsertionPoint = true; + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/HistogramValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/HistogramValuesSourceBuilder.java index 83ada5dbbc3c3..1dc0aa596d790 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/HistogramValuesSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/HistogramValuesSourceBuilder.java @@ -19,19 +19,17 @@ package org.elasticsearch.search.aggregations.bucket.composite; -import org.apache.lucene.search.SortField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; -import org.elasticsearch.search.aggregations.support.FieldContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.sort.SortOrder; import java.io.IOException; import java.util.Objects; @@ -108,27 +106,16 @@ public HistogramValuesSourceBuilder interval(double interval) { } @Override - protected CompositeValuesSourceConfig innerBuild(SearchContext context, - ValuesSourceConfig config, - int pos, - int numPos, - SortField sortField) throws IOException { + protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig config) throws IOException { ValuesSource orig = config.toValuesSource(context.getQueryShardContext()); if (orig == null) { orig = ValuesSource.Numeric.EMPTY; } if (orig instanceof ValuesSource.Numeric) { ValuesSource.Numeric numeric = (ValuesSource.Numeric) orig; - HistogramValuesSource vs = new HistogramValuesSource(numeric, interval); - boolean canEarlyTerminate = false; - final FieldContext fieldContext = config.fieldContext(); - if (sortField != null && - pos == numPos-1 && - fieldContext != null) { - canEarlyTerminate = checkCanEarlyTerminate(context.searcher().getIndexReader(), - fieldContext.field(), order() == SortOrder.ASC ? false : true, sortField); - } - return new CompositeValuesSourceConfig(name, vs, config.format(), order(), canEarlyTerminate); + final HistogramValuesSource vs = new HistogramValuesSource(numeric, interval); + final MappedFieldType fieldType = config.fieldContext() != null ? config.fieldContext().fieldType() : null; + return new CompositeValuesSourceConfig(name, fieldType, vs, config.format(), order()); } else { throw new IllegalArgumentException("invalid source, expected numeric, got " + orig.getClass().getSimpleName()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/LongValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/LongValuesSource.java new file mode 100644 index 0000000000000..96d0b02780948 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/LongValuesSource.java @@ -0,0 +1,190 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.document.IntPoint; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.PointRangeQuery; +import org.apache.lucene.search.Query; +import org.elasticsearch.common.CheckedFunction; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.LongArray; +import org.elasticsearch.index.mapper.DateFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.LeafBucketCollector; + +import java.io.IOException; +import java.util.function.LongUnaryOperator; +import java.util.function.ToLongFunction; + +/** + * A {@link SingleDimensionValuesSource} for longs. + */ +class LongValuesSource extends SingleDimensionValuesSource { + private final CheckedFunction docValuesFunc; + private final LongUnaryOperator rounding; + // handles "format" for date histogram source + private final DocValueFormat format; + + private final LongArray values; + private long currentValue; + + LongValuesSource(BigArrays bigArrays, MappedFieldType fieldType, + CheckedFunction docValuesFunc, + LongUnaryOperator rounding, DocValueFormat format, int size, int reverseMul) { + super(fieldType, size, reverseMul); + this.docValuesFunc = docValuesFunc; + this.rounding = rounding; + this.format = format; + this.values = bigArrays.newLongArray(size, false); + } + + @Override + void copyCurrent(int slot) { + values.set(slot, currentValue); + } + + @Override + int compare(int from, int to) { + return compareValues(values.get(from), values.get(to)); + } + + @Override + int compareCurrent(int slot) { + return compareValues(currentValue, values.get(slot)); + } + + @Override + int compareCurrentWithAfter() { + return compareValues(currentValue, afterValue); + } + + private int compareValues(long v1, long v2) { + return Long.compare(v1, v2) * reverseMul; + } + + @Override + void setAfter(Comparable value) { + if (value instanceof Number) { + afterValue = ((Number) value).longValue(); + } else { + // for date histogram source with "format", the after value is formatted + // as a string so we need to retrieve the original value in milliseconds. + afterValue = format.parseLong(value.toString(), false, () -> { + throw new IllegalArgumentException("now() is not supported in [after] key"); + }); + } + } + + @Override + Long toComparable(int slot) { + return values.get(slot); + } + + @Override + LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector next) throws IOException { + final SortedNumericDocValues dvs = docValuesFunc.apply(context); + return new LeafBucketCollector() { + @Override + public void collect(int doc, long bucket) throws IOException { + if (dvs.advanceExact(doc)) { + int num = dvs.docValueCount(); + for (int i = 0; i < num; i++) { + currentValue = dvs.nextValue(); + next.collect(doc, bucket); + } + } + } + }; + } + + @Override + LeafBucketCollector getLeafCollector(Comparable value, LeafReaderContext context, LeafBucketCollector next) { + if (value.getClass() != Long.class) { + throw new IllegalArgumentException("Expected Long, got " + value.getClass()); + } + currentValue = (Long) value; + return new LeafBucketCollector() { + @Override + public void collect(int doc, long bucket) throws IOException { + next.collect(doc, bucket); + } + }; + } + + @Override + SortedDocsProducer createSortedDocsProducerOrNull(IndexReader reader, Query query) { + if (checkIfSortedDocsIsApplicable(reader, fieldType) == false || + (query != null && + query.getClass() != MatchAllDocsQuery.class && + // if the query is a range query over the same field + (query instanceof PointRangeQuery && fieldType.name().equals((((PointRangeQuery) query).getField()))) == false)) { + return null; + } + final byte[] lowerPoint; + final byte[] upperPoint; + if (query instanceof PointRangeQuery) { + final PointRangeQuery rangeQuery = (PointRangeQuery) query; + lowerPoint = rangeQuery.getLowerPoint(); + upperPoint = rangeQuery.getUpperPoint(); + } else { + lowerPoint = null; + upperPoint = null; + } + + if (fieldType instanceof NumberFieldMapper.NumberFieldType) { + NumberFieldMapper.NumberFieldType ft = (NumberFieldMapper.NumberFieldType) fieldType; + final ToLongFunction toBucketFunction; + + switch (ft.typeName()) { + case "long": + toBucketFunction = (value) -> rounding.applyAsLong(LongPoint.decodeDimension(value, 0)); + break; + + case "int": + case "short": + case "byte": + toBucketFunction = (value) -> rounding.applyAsLong(IntPoint.decodeDimension(value, 0)); + break; + + default: + return null; + } + return new PointsSortedDocsProducer(fieldType.name(), toBucketFunction, lowerPoint, upperPoint); + } else if (fieldType instanceof DateFieldMapper.DateFieldType) { + final ToLongFunction toBucketFunction = (value) -> rounding.applyAsLong(LongPoint.decodeDimension(value, 0)); + return new PointsSortedDocsProducer(fieldType.name(), toBucketFunction, lowerPoint, upperPoint); + } else { + return null; + } + } + + @Override + public void close() { + Releasables.close(values); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/PointsSortedDocsProducer.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/PointsSortedDocsProducer.java new file mode 100644 index 0000000000000..d0f2d6ef9461a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/PointsSortedDocsProducer.java @@ -0,0 +1,181 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PointValues; +import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.search.Query; +import org.apache.lucene.util.DocIdSetBuilder; +import org.apache.lucene.util.StringHelper; + +import java.io.IOException; +import java.util.function.ToLongFunction; + +/** + * A {@link SortedDocsProducer} that can sort documents based on numerics indexed in the provided field. + */ +class PointsSortedDocsProducer extends SortedDocsProducer { + private final ToLongFunction bucketFunction; + private final byte[] lowerPointQuery; + private final byte[] upperPointQuery; + + PointsSortedDocsProducer(String field, ToLongFunction bucketFunction, byte[] lowerPointQuery, byte[] upperPointQuery) { + super(field); + this.bucketFunction = bucketFunction; + this.lowerPointQuery = lowerPointQuery; + this.upperPointQuery = upperPointQuery; + } + + @Override + DocIdSet processLeaf(Query query, CompositeValuesCollectorQueue queue, + LeafReaderContext context, boolean fillDocIdSet) throws IOException { + final PointValues values = context.reader().getPointValues(field); + if (values == null) { + // no value for the field + return DocIdSet.EMPTY; + } + long lowerBucket = Long.MIN_VALUE; + Comparable lowerValue = queue.getLowerValueLeadSource(); + if (lowerValue != null) { + if (lowerValue.getClass() != Long.class) { + throw new IllegalStateException("expected Long, got " + lowerValue.getClass()); + } + lowerBucket = (Long) lowerValue; + } + + long upperBucket = Long.MAX_VALUE; + Comparable upperValue = queue.getUpperValueLeadSource(); + if (upperValue != null) { + if (upperValue.getClass() != Long.class) { + throw new IllegalStateException("expected Long, got " + upperValue.getClass()); + } + upperBucket = (Long) upperValue; + } + DocIdSetBuilder builder = fillDocIdSet ? new DocIdSetBuilder(context.reader().maxDoc(), values, field) : null; + Visitor visitor = new Visitor(context, queue, builder, values.getBytesPerDimension(), lowerBucket, upperBucket); + try { + values.intersect(visitor); + visitor.flush(); + } catch (CollectionTerminatedException exc) {} + return fillDocIdSet ? builder.build() : DocIdSet.EMPTY; + } + + private class Visitor implements PointValues.IntersectVisitor { + final LeafReaderContext context; + final CompositeValuesCollectorQueue queue; + final DocIdSetBuilder builder; + final int maxDoc; + final int bytesPerDim; + final long lowerBucket; + final long upperBucket; + + DocIdSetBuilder bucketDocsBuilder; + DocIdSetBuilder.BulkAdder adder; + int remaining; + long lastBucket; + boolean first = true; + + Visitor(LeafReaderContext context, CompositeValuesCollectorQueue queue, DocIdSetBuilder builder, + int bytesPerDim, long lowerBucket, long upperBucket) { + this.context = context; + this.maxDoc = context.reader().maxDoc(); + this.queue = queue; + this.builder = builder; + this.lowerBucket = lowerBucket; + this.upperBucket = upperBucket; + this.bucketDocsBuilder = new DocIdSetBuilder(maxDoc); + this.bytesPerDim = bytesPerDim; + } + + @Override + public void grow(int count) { + remaining = count; + adder = bucketDocsBuilder.grow(count); + } + + @Override + public void visit(int docID) throws IOException { + throw new IllegalStateException("should never be called"); + } + + @Override + public void visit(int docID, byte[] packedValue) throws IOException { + if (compare(packedValue, packedValue) != PointValues.Relation.CELL_CROSSES_QUERY) { + remaining --; + return; + } + + long bucket = bucketFunction.applyAsLong(packedValue); + if (first == false && bucket != lastBucket) { + final DocIdSet docIdSet = bucketDocsBuilder.build(); + if (processBucket(queue, context, docIdSet.iterator(), lastBucket, builder) && + // lower bucket is inclusive + lowerBucket != lastBucket) { + // this bucket does not have any competitive composite buckets, + // we can early terminate the collection because the remaining buckets are guaranteed + // to be greater than this bucket. + throw new CollectionTerminatedException(); + } + bucketDocsBuilder = new DocIdSetBuilder(maxDoc); + assert remaining > 0; + adder = bucketDocsBuilder.grow(remaining); + } + lastBucket = bucket; + first = false; + adder.add(docID); + remaining --; + } + + @Override + public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { + if ((upperPointQuery != null && StringHelper.compare(bytesPerDim, minPackedValue, 0, upperPointQuery, 0) > 0) || + (lowerPointQuery != null && StringHelper.compare(bytesPerDim, maxPackedValue, 0, lowerPointQuery, 0) < 0)) { + // does not match the query + return PointValues.Relation.CELL_OUTSIDE_QUERY; + } + + // check the current bounds + if (lowerBucket != Long.MIN_VALUE) { + long maxBucket = bucketFunction.applyAsLong(maxPackedValue); + if (maxBucket < lowerBucket) { + return PointValues.Relation.CELL_OUTSIDE_QUERY; + } + } + + if (upperBucket != Long.MAX_VALUE) { + long minBucket = bucketFunction.applyAsLong(minPackedValue); + if (minBucket > upperBucket) { + return PointValues.Relation.CELL_OUTSIDE_QUERY; + } + } + return PointValues.Relation.CELL_CROSSES_QUERY; + } + + public void flush() throws IOException { + if (first == false) { + final DocIdSet docIdSet = bucketDocsBuilder.build(); + processBucket(queue, context, docIdSet.iterator(), lastBucket, builder); + bucketDocsBuilder = null; + } + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/RoundingValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/RoundingValuesSource.java index 099f2e5e0fd5a..635690c44f49e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/RoundingValuesSource.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/RoundingValuesSource.java @@ -51,13 +51,17 @@ public boolean isFloatingPoint() { return false; } + public long round(long value) { + return rounding.round(value); + } + @Override public SortedNumericDocValues longValues(LeafReaderContext context) throws IOException { SortedNumericDocValues values = vs.longValues(context); return new SortedNumericDocValues() { @Override public long nextValue() throws IOException { - return rounding.round(values.nextValue()); + return round(values.nextValue()); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/SingleDimensionValuesSource.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/SingleDimensionValuesSource.java new file mode 100644 index 0000000000000..efedce7db2afa --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/SingleDimensionValuesSource.java @@ -0,0 +1,143 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Query; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.sort.SortOrder; + +import java.io.IOException; + +/** + * A source that can record and compare values of similar type. + */ +abstract class SingleDimensionValuesSource> implements Releasable { + protected final int size; + protected final int reverseMul; + protected T afterValue; + @Nullable + protected MappedFieldType fieldType; + + /** + * Ctr + * + * @param fieldType The fieldType associated with the source. + * @param size The number of values to record. + * @param reverseMul -1 if the natural order ({@link SortOrder#ASC} should be reversed. + */ + SingleDimensionValuesSource(@Nullable MappedFieldType fieldType, int size, int reverseMul) { + this.fieldType = fieldType; + this.size = size; + this.reverseMul = reverseMul; + this.afterValue = null; + } + + /** + * The current value is filled by a {@link LeafBucketCollector} that visits all the + * values of each document. This method saves this current value in a slot and should only be used + * in the context of a collection. + * See {@link this#getLeafCollector}. + */ + abstract void copyCurrent(int slot); + + /** + * Compares the value in from with the value in to. + */ + abstract int compare(int from, int to); + + /** + * The current value is filled by a {@link LeafBucketCollector} that visits all the + * values of each document. This method compares this current value with the value present in + * the provided slot and should only be used in the context of a collection. + * See {@link this#getLeafCollector}. + */ + abstract int compareCurrent(int slot); + + /** + * The current value is filled by a {@link LeafBucketCollector} that visits all the + * values of each document. This method compares this current value with the after value + * set on this source and should only be used in the context of a collection. + * See {@link this#getLeafCollector}. + */ + abstract int compareCurrentWithAfter(); + + /** + * Sets the after value for this source. Values that compares smaller are filtered. + */ + abstract void setAfter(Comparable value); + + /** + * Returns the after value set for this source. + */ + T getAfter() { + return afterValue; + } + + /** + * Transforms the value in slot to a {@link Comparable} object. + */ + abstract T toComparable(int slot) throws IOException; + + /** + * Creates a {@link LeafBucketCollector} that extracts all values from a document and invokes + * {@link LeafBucketCollector#collect} on the provided next collector for each of them. + * The current value of this source is set on each call and can be accessed by next via + * the {@link this#copyCurrent(int)} and {@link this#compareCurrent(int)} methods. Note that these methods + * are only valid when invoked from the {@link LeafBucketCollector} created in this source. + */ + abstract LeafBucketCollector getLeafCollector(LeafReaderContext context, LeafBucketCollector next) throws IOException; + + /** + * Creates a {@link LeafBucketCollector} that sets the current value for each document to the provided + * value and invokes {@link LeafBucketCollector#collect} on the provided next collector. + */ + abstract LeafBucketCollector getLeafCollector(Comparable value, + LeafReaderContext context, LeafBucketCollector next) throws IOException; + + /** + * Returns a {@link SortedDocsProducer} or null if this source cannot produce sorted docs. + */ + abstract SortedDocsProducer createSortedDocsProducerOrNull(IndexReader reader, Query query); + + /** + * Returns true if a {@link SortedDocsProducer} should be used to optimize the execution. + */ + protected boolean checkIfSortedDocsIsApplicable(IndexReader reader, MappedFieldType fieldType) { + if (fieldType == null || + fieldType.indexOptions() == IndexOptions.NONE || + // inverse of the natural order + reverseMul == -1) { + return false; + } + + if (reader.hasDeletions() && + (reader.numDocs() == 0 || (double) reader.numDocs() / (double) reader.maxDoc() < 0.5)) { + // do not use the index if it has more than 50% of deleted docs + return false; + } + return true; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/SortedDocsProducer.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/SortedDocsProducer.java new file mode 100644 index 0000000000000..d9d927ff66061 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/SortedDocsProducer.java @@ -0,0 +1,108 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Query; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.DocIdSetBuilder; +import org.elasticsearch.common.inject.internal.Nullable; +import org.elasticsearch.search.aggregations.LeafBucketCollector; + +import java.io.IOException; + +/** + * A producer that visits composite buckets in the order of the value indexed in the leading source of the composite + * definition. It can be used to control which documents should be collected to produce the top composite buckets + * without visiting all documents in an index. + */ +abstract class SortedDocsProducer { + protected final String field; + + SortedDocsProducer(String field) { + this.field = field; + } + + /** + * Visits all non-deleted documents in iterator and fills the provided queue + * with the top composite buckets extracted from the collection. + * Documents that contain a top composite bucket are added in the provided builder if it is not null. + * + * Returns true if the queue is full and the current leadSourceBucket did not produce any competitive + * composite buckets. + */ + protected boolean processBucket(CompositeValuesCollectorQueue queue, LeafReaderContext context, DocIdSetIterator iterator, + Comparable leadSourceBucket, @Nullable DocIdSetBuilder builder) throws IOException { + final int[] topCompositeCollected = new int[1]; + final boolean[] hasCollected = new boolean[1]; + final LeafBucketCollector queueCollector = new LeafBucketCollector() { + int lastDoc = -1; + + // we need to add the matching document in the builder + // so we build a bulk adder from the approximate cost of the iterator + // and rebuild the adder during the collection if needed + int remainingBits = (int) Math.min(iterator.cost(), Integer.MAX_VALUE); + DocIdSetBuilder.BulkAdder adder = builder == null ? null : builder.grow(remainingBits); + + @Override + public void collect(int doc, long bucket) throws IOException { + hasCollected[0] = true; + int slot = queue.addIfCompetitive(); + if (slot != -1) { + topCompositeCollected[0]++; + if (adder != null && doc != lastDoc) { + if (remainingBits == 0) { + // the cost approximation was lower than the real size, we need to grow the adder + // by some numbers (128) to ensure that we can add the extra documents + adder = builder.grow(128); + remainingBits = 128; + } + adder.add(doc); + remainingBits --; + lastDoc = doc; + } + } + } + }; + final Bits liveDocs = context.reader().getLiveDocs(); + final LeafBucketCollector collector = queue.getLeafCollector(leadSourceBucket, context, queueCollector); + while (iterator.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) { + if (liveDocs == null || liveDocs.get(iterator.docID())) { + collector.collect(iterator.docID()); + } + } + if (queue.isFull() && + hasCollected[0] && + topCompositeCollected[0] == 0) { + return true; + } + return false; + } + + /** + * Populates the queue with the composite buckets present in the context. + * Returns the {@link DocIdSet} of the documents that contain a top composite bucket in this leaf or + * {@link DocIdSet#EMPTY} if fillDocIdSet is false. + */ + abstract DocIdSet processLeaf(Query query, CompositeValuesCollectorQueue queue, + LeafReaderContext context, boolean fillDocIdSet) throws IOException; +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsSortedDocsProducer.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsSortedDocsProducer.java new file mode 100644 index 0000000000000..f9d9877e320b4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsSortedDocsProducer.java @@ -0,0 +1,79 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.PostingsEnum; +import org.apache.lucene.index.Terms; +import org.apache.lucene.index.TermsEnum; +import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.search.Query; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.DocIdSetBuilder; + +import java.io.IOException; + +/** + * A {@link SortedDocsProducer} that can sort documents based on terms indexed in the provided field. + */ +class TermsSortedDocsProducer extends SortedDocsProducer { + TermsSortedDocsProducer(String field) { + super(field); + } + + @Override + DocIdSet processLeaf(Query query, CompositeValuesCollectorQueue queue, + LeafReaderContext context, boolean fillDocIdSet) throws IOException { + final Terms terms = context.reader().terms(field); + if (terms == null) { + // no value for the field + return DocIdSet.EMPTY; + } + BytesRef lowerValue = (BytesRef) queue.getLowerValueLeadSource(); + BytesRef upperValue = (BytesRef) queue.getUpperValueLeadSource(); + final TermsEnum te = terms.iterator(); + if (lowerValue != null) { + if (te.seekCeil(lowerValue) == TermsEnum.SeekStatus.END) { + return DocIdSet.EMPTY ; + } + } else { + if (te.next() == null) { + return DocIdSet.EMPTY; + } + } + DocIdSetBuilder builder = fillDocIdSet ? new DocIdSetBuilder(context.reader().maxDoc(), terms) : null; + PostingsEnum reuse = null; + boolean first = true; + do { + if (upperValue != null && upperValue.compareTo(te.term()) < 0) { + break; + } + reuse = te.postings(reuse, PostingsEnum.NONE); + if (processBucket(queue, context, reuse, te.term(), builder) && !first) { + // this bucket does not have any competitive composite buckets, + // we can early terminate the collection because the remaining buckets are guaranteed + // to be greater than this bucket. + break; + } + first = false; + } while (te.next() != null); + return fillDocIdSet ? builder.build() : DocIdSet.EMPTY; + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java index 6ca5cdbcb6230..21ab14fe27e21 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/composite/TermsValuesSourceBuilder.java @@ -19,18 +19,16 @@ package org.elasticsearch.search.aggregations.bucket.composite; -import org.apache.lucene.search.SortField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.search.aggregations.support.FieldContext; +import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.script.Script; -import org.elasticsearch.search.sort.SortOrder; import java.io.IOException; @@ -80,21 +78,12 @@ public String type() { } @Override - protected CompositeValuesSourceConfig innerBuild(SearchContext context, - ValuesSourceConfig config, - int pos, - int numPos, - SortField sortField) throws IOException { + protected CompositeValuesSourceConfig innerBuild(SearchContext context, ValuesSourceConfig config) throws IOException { ValuesSource vs = config.toValuesSource(context.getQueryShardContext()); if (vs == null) { vs = ValuesSource.Numeric.EMPTY; } - boolean canEarlyTerminate = false; - final FieldContext fieldContext = config.fieldContext(); - if (sortField != null && config.fieldContext() != null) { - canEarlyTerminate = checkCanEarlyTerminate(context.searcher().getIndexReader(), - fieldContext.field(), order() == SortOrder.ASC ? false : true, sortField); - } - return new CompositeValuesSourceConfig(name, vs, config.format(), order(), canEarlyTerminate); + final MappedFieldType fieldType = config.fieldContext() != null ? config.fieldContext().fieldType() : null; + return new CompositeValuesSourceConfig(name, fieldType, vs, config.format(), order()); } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregatorTests.java index 094457a8bf4f6..e52a4b7bbbc9e 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeAggregatorTests.java @@ -19,42 +19,44 @@ package org.elasticsearch.search.aggregations.bucket.composite; -import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.document.Document; +import org.apache.lucene.document.DoublePoint; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.IntPoint; +import org.apache.lucene.document.LongPoint; import org.apache.lucene.document.SortedNumericDocValuesField; import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.document.StringField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; -import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.MultiReader; import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.DocValuesFieldExistsQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.Sort; -import org.apache.lucene.search.SortField; -import org.apache.lucene.search.SortedNumericSortField; -import org.apache.lucene.search.SortedSetSortField; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.NumericUtils; -import org.apache.lucene.util.TestUtil; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.DateFieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.index.query.QueryShardException; +import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.AggregatorTestCase; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; +import org.elasticsearch.search.aggregations.bucket.terms.StringTerms; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.max.InternalMax; +import org.elasticsearch.search.aggregations.metrics.max.MaxAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.tophits.TopHits; import org.elasticsearch.search.aggregations.metrics.tophits.TopHitsAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.sort.SortOrder; -import org.elasticsearch.test.IndexSettingsModule; import org.joda.time.DateTimeZone; import org.junit.After; import org.junit.Before; @@ -64,12 +66,18 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; public class CompositeAggregatorTests extends AggregatorTestCase { @@ -79,7 +87,7 @@ public class CompositeAggregatorTests extends AggregatorTestCase { @Before public void setUp() throws Exception { super.setUp(); - FIELD_TYPES = new MappedFieldType[5]; + FIELD_TYPES = new MappedFieldType[6]; FIELD_TYPES[0] = new KeywordFieldMapper.KeywordFieldType(); FIELD_TYPES[0].setName("keyword"); FIELD_TYPES[0].setHasDocValues(true); @@ -101,6 +109,10 @@ public void setUp() throws Exception { FIELD_TYPES[4] = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.INTEGER); FIELD_TYPES[4].setName("price"); FIELD_TYPES[4].setHasDocValues(true); + + FIELD_TYPES[5] = new KeywordFieldMapper.KeywordFieldType(); + FIELD_TYPES[5].setName("terms"); + FIELD_TYPES[5].setHasDocValues(true); } @Override @@ -110,6 +122,19 @@ public void tearDown() throws Exception { FIELD_TYPES = null; } + public void testUnmappedField() throws Exception { + TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder(randomAlphaOfLengthBetween(5, 10)) + .field("unknown"); + CompositeAggregationBuilder builder = new CompositeAggregationBuilder("test", Collections.singletonList(terms)); + IndexSearcher searcher = new IndexSearcher(new MultiReader()); + QueryShardException exc = + expectThrows(QueryShardException.class, () -> createAggregatorFactory(builder, searcher)); + assertThat(exc.getMessage(), containsString("failed to find field [unknown] and [missing] is not provided")); + // should work when missing is provided + terms.missing("missing"); + createAggregatorFactory(builder, searcher); + } + public void testWithKeyword() throws Exception { final List>> dataset = new ArrayList<>(); dataset.addAll( @@ -121,8 +146,7 @@ public void testWithKeyword() throws Exception { createDocument("keyword", "c") ) ); - final Sort sort = new Sort(new SortedSetSortField("keyword", false)); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword"); @@ -139,7 +163,7 @@ public void testWithKeyword() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword"); @@ -168,8 +192,7 @@ public void testWithKeywordMissingAfter() throws Exception { createDocument("keyword", "delta") ) ); - final Sort sort = new Sort(new SortedSetSortField("keyword", false)); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword"); @@ -188,7 +211,7 @@ public void testWithKeywordMissingAfter() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword"); @@ -206,7 +229,7 @@ public void testWithKeywordMissingAfter() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword").order(SortOrder.DESC); @@ -236,8 +259,7 @@ public void testWithKeywordDesc() throws Exception { createDocument("keyword", "c") ) ); - final Sort sort = new Sort(new SortedSetSortField("keyword", true)); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword") @@ -255,7 +277,7 @@ public void testWithKeywordDesc() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword") @@ -285,7 +307,7 @@ public void testMultiValuedWithKeyword() throws Exception { ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword"); @@ -307,7 +329,7 @@ public void testMultiValuedWithKeyword() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword"); @@ -339,7 +361,7 @@ public void testMultiValuedWithKeywordDesc() throws Exception { ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword") @@ -362,7 +384,7 @@ public void testMultiValuedWithKeywordDesc() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword") @@ -394,11 +416,7 @@ public void testWithKeywordAndLong() throws Exception { createDocument("long", 100L) ) ); - final Sort sort = new Sort( - new SortedSetSortField("keyword", false), - new SortedNumericSortField("long", SortField.Type.LONG) - ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( new TermsValuesSourceBuilder("keyword").field("keyword"), @@ -419,7 +437,7 @@ public void testWithKeywordAndLong() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( new TermsValuesSourceBuilder("keyword").field("keyword"), @@ -451,11 +469,7 @@ public void testWithKeywordAndLongDesc() throws Exception { createDocument("long", 100L) ) ); - final Sort sort = new Sort( - new SortedSetSortField("keyword", true), - new SortedNumericSortField("long", SortField.Type.LONG, true) - ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -477,7 +491,7 @@ public void testWithKeywordAndLongDesc() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -510,7 +524,7 @@ public void testMultiValuedWithKeywordAndLong() throws Exception { ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -543,7 +557,7 @@ public void testMultiValuedWithKeywordAndLong() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -580,11 +594,10 @@ public void testMultiValuedWithKeywordAndLongDesc() throws Exception { createDocument("keyword", Arrays.asList("d", "d"), "long", Arrays.asList(10L, 100L, 1000L)), createDocument("keyword", "c"), createDocument("long", 100L) - ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -619,7 +632,7 @@ public void testMultiValuedWithKeywordAndLongDesc() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -653,7 +666,7 @@ public void testMultiValuedWithKeywordLongAndDouble() throws Exception { ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -688,7 +701,7 @@ public void testMultiValuedWithKeywordLongAndDouble() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -723,7 +736,7 @@ public void testMultiValuedWithKeywordLongAndDouble() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -751,8 +764,12 @@ public void testWithDateHistogram() throws IOException { createDocument("long", 4L) ) ); - final Sort sort = new Sort(new SortedNumericSortField("date", SortField.Type.LONG)); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date"), + LongPoint.newRangeQuery( + "date", + asLong("2016-09-20T09:00:34"), + asLong("2017-10-20T06:09:24") + )), dataset, () -> { DateHistogramValuesSourceBuilder histo = new DateHistogramValuesSourceBuilder("date") .field("date") @@ -771,7 +788,12 @@ public void testWithDateHistogram() throws IOException { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date"), + LongPoint.newRangeQuery( + "date", + asLong("2016-09-20T11:34:00"), + asLong("2017-10-20T06:09:24") + )), dataset, () -> { DateHistogramValuesSourceBuilder histo = new DateHistogramValuesSourceBuilder("date") .field("date") @@ -802,8 +824,7 @@ public void testWithDateHistogramAndFormat() throws IOException { createDocument("long", 4L) ) ); - final Sort sort = new Sort(new SortedNumericSortField("date", SortField.Type.LONG)); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date")), dataset, () -> { DateHistogramValuesSourceBuilder histo = new DateHistogramValuesSourceBuilder("date") .field("date") @@ -823,7 +844,7 @@ public void testWithDateHistogramAndFormat() throws IOException { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date")), dataset, () -> { DateHistogramValuesSourceBuilder histo = new DateHistogramValuesSourceBuilder("date") .field("date") @@ -845,7 +866,7 @@ public void testWithDateHistogramAndFormat() throws IOException { public void testThatDateHistogramFailsFormatAfter() throws IOException { ElasticsearchParseException exc = expectThrows(ElasticsearchParseException.class, - () -> testSearchCase(new MatchAllDocsQuery(), null, Collections.emptyList(), + () -> testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date")), Collections.emptyList(), () -> { DateHistogramValuesSourceBuilder histo = new DateHistogramValuesSourceBuilder("date") .field("date") @@ -860,7 +881,7 @@ public void testThatDateHistogramFailsFormatAfter() throws IOException { assertThat(exc.getCause().getMessage(), containsString("now() is not supported in [after] key")); exc = expectThrows(ElasticsearchParseException.class, - () -> testSearchCase(new MatchAllDocsQuery(), null, Collections.emptyList(), + () -> testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date")), Collections.emptyList(), () -> { DateHistogramValuesSourceBuilder histo = new DateHistogramValuesSourceBuilder("date") .field("date") @@ -887,8 +908,7 @@ public void testWithDateHistogramAndTimeZone() throws IOException { createDocument("long", 4L) ) ); - final Sort sort = new Sort(new SortedNumericSortField("date", SortField.Type.LONG)); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date")), dataset, () -> { DateHistogramValuesSourceBuilder histo = new DateHistogramValuesSourceBuilder("date") .field("date") @@ -908,7 +928,7 @@ public void testWithDateHistogramAndTimeZone() throws IOException { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date")), dataset, () -> { DateHistogramValuesSourceBuilder histo = new DateHistogramValuesSourceBuilder("date") .field("date") @@ -940,7 +960,12 @@ public void testWithDateHistogramAndKeyword() throws IOException { createDocument("long", 4L) ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date"), + LongPoint.newRangeQuery( + "date", + asLong("2016-09-20T09:00:34"), + asLong("2017-10-20T06:09:24") + )), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -971,7 +996,12 @@ public void testWithDateHistogramAndKeyword() throws IOException { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("date"), + LongPoint.newRangeQuery( + "date", + asLong("2016-09-20T11:34:00"), + asLong("2017-10-20T06:09:24") + )), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -1007,7 +1037,7 @@ public void testWithKeywordAndHistogram() throws IOException { createDocument("long", 4L) ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("price")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -1035,7 +1065,7 @@ public void testWithKeywordAndHistogram() throws IOException { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("price")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -1075,7 +1105,7 @@ public void testWithHistogramAndKeyword() throws IOException { createDocument("long", 4L) ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("double")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -1105,7 +1135,7 @@ public void testWithHistogramAndKeyword() throws IOException { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("double")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -1138,7 +1168,7 @@ public void testWithKeywordAndDateHistogram() throws IOException { createDocument("long", 4L) ) ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -1167,7 +1197,7 @@ public void testWithKeywordAndDateHistogram() throws IOException { } ); - testSearchCase(new MatchAllDocsQuery(), null, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> new CompositeAggregationBuilder("name", Arrays.asList( @@ -1202,8 +1232,7 @@ public void testWithKeywordAndTopHits() throws Exception { createDocument("keyword", "c") ) ); - final Sort sort = new Sort(new SortedSetSortField("keyword", false)); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword"); @@ -1232,7 +1261,7 @@ public void testWithKeywordAndTopHits() throws Exception { } ); - testSearchCase(new MatchAllDocsQuery(), sort, dataset, + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, () -> { TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") .field("keyword"); @@ -1257,36 +1286,174 @@ public void testWithKeywordAndTopHits() throws Exception { ); } - private void testSearchCase(Query query, Sort sort, + public void testWithTermsSubAggExecutionMode() throws Exception { + // test with no bucket + for (Aggregator.SubAggCollectionMode mode : Aggregator.SubAggCollectionMode.values()) { + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), + Collections.singletonList(createDocument()), + () -> { + TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") + .field("keyword"); + return new CompositeAggregationBuilder("name", Collections.singletonList(terms)) + .subAggregation( + new TermsAggregationBuilder("terms", ValueType.STRING) + .field("terms") + .collectMode(mode) + .subAggregation(new MaxAggregationBuilder("max").field("long")) + ); + }, (result) -> { + assertEquals(0, result.getBuckets().size()); + } + ); + } + + final List>> dataset = new ArrayList<>(); + dataset.addAll( + Arrays.asList( + createDocument("keyword", "a", "terms", "a", "long", 50L), + createDocument("keyword", "c", "terms", "d", "long", 78L), + createDocument("keyword", "a", "terms", "w", "long", 78L), + createDocument("keyword", "d", "terms", "y", "long", 76L), + createDocument("keyword", "c", "terms", "y", "long", 70L) + ) + ); + for (Aggregator.SubAggCollectionMode mode : Aggregator.SubAggCollectionMode.values()) { + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery("keyword")), dataset, + () -> { + TermsValuesSourceBuilder terms = new TermsValuesSourceBuilder("keyword") + .field("keyword"); + return new CompositeAggregationBuilder("name", Collections.singletonList(terms)) + .subAggregation( + new TermsAggregationBuilder("terms", ValueType.STRING) + .field("terms") + .collectMode(mode) + .subAggregation(new MaxAggregationBuilder("max").field("long")) + ); + }, (result) -> { + assertEquals(3, result.getBuckets().size()); + + assertEquals("{keyword=a}", result.getBuckets().get(0).getKeyAsString()); + assertEquals(2L, result.getBuckets().get(0).getDocCount()); + StringTerms subTerms = result.getBuckets().get(0).getAggregations().get("terms"); + assertEquals(2, subTerms.getBuckets().size()); + assertEquals("a", subTerms.getBuckets().get(0).getKeyAsString()); + assertEquals("w", subTerms.getBuckets().get(1).getKeyAsString()); + InternalMax max = subTerms.getBuckets().get(0).getAggregations().get("max"); + assertEquals(50L, (long) max.getValue()); + max = subTerms.getBuckets().get(1).getAggregations().get("max"); + assertEquals(78L, (long) max.getValue()); + + assertEquals("{keyword=c}", result.getBuckets().get(1).getKeyAsString()); + assertEquals(2L, result.getBuckets().get(1).getDocCount()); + subTerms = result.getBuckets().get(1).getAggregations().get("terms"); + assertEquals(2, subTerms.getBuckets().size()); + assertEquals("d", subTerms.getBuckets().get(0).getKeyAsString()); + assertEquals("y", subTerms.getBuckets().get(1).getKeyAsString()); + max = subTerms.getBuckets().get(0).getAggregations().get("max"); + assertEquals(78L, (long) max.getValue()); + max = subTerms.getBuckets().get(1).getAggregations().get("max"); + assertEquals(70L, (long) max.getValue()); + + assertEquals("{keyword=d}", result.getBuckets().get(2).getKeyAsString()); + assertEquals(1L, result.getBuckets().get(2).getDocCount()); + subTerms = result.getBuckets().get(2).getAggregations().get("terms"); + assertEquals(1, subTerms.getBuckets().size()); + assertEquals("y", subTerms.getBuckets().get(0).getKeyAsString()); + max = subTerms.getBuckets().get(0).getAggregations().get("max"); + assertEquals(76L, (long) max.getValue()); + } + ); + } + } + + public void testRandomStrings() throws IOException { + testRandomTerms("keyword", () -> randomAlphaOfLengthBetween(5, 50), (v) -> (String) v); + } + + public void testRandomLongs() throws IOException { + testRandomTerms("long", () -> randomLong(), (v) -> (long) v); + } + + public void testRandomInts() throws IOException { + testRandomTerms("price", () -> randomInt(), (v) -> ((Number) v).intValue()); + } + + private , V extends Comparable> void testRandomTerms(String field, + Supplier randomSupplier, + Function transformKey) throws IOException { + int numTerms = randomIntBetween(10, 500); + List terms = new ArrayList<>(); + for (int i = 0; i < numTerms; i++) { + terms.add(randomSupplier.get()); + } + int numDocs = randomIntBetween(100, 200); + List>> dataset = new ArrayList<>(); + + Set valuesSet = new HashSet<>(); + Map, AtomicLong> expectedDocCounts = new HashMap<> (); + for (int i = 0; i < numDocs; i++) { + int numValues = randomIntBetween(1, 5); + Set values = new HashSet<>(); + for (int j = 0; j < numValues; j++) { + int rand = randomIntBetween(0, terms.size() - 1); + if (values.add(terms.get(rand))) { + AtomicLong count = expectedDocCounts.computeIfAbsent(terms.get(rand), + (k) -> new AtomicLong(0)); + count.incrementAndGet(); + valuesSet.add(terms.get(rand)); + } + } + dataset.add(Collections.singletonMap(field, new ArrayList<>(values))); + } + List expected = new ArrayList<>(valuesSet); + Collections.sort(expected); + + List> seen = new ArrayList<>(); + AtomicBoolean finish = new AtomicBoolean(false); + int size = randomIntBetween(1, expected.size()); + while (finish.get() == false) { + testSearchCase(Arrays.asList(new MatchAllDocsQuery(), new DocValuesFieldExistsQuery(field)), dataset, + () -> { + Map afterKey = null; + if (seen.size() > 0) { + afterKey = Collections.singletonMap(field, seen.get(seen.size()-1)); + } + TermsValuesSourceBuilder source = new TermsValuesSourceBuilder(field).field(field); + return new CompositeAggregationBuilder("name", Collections.singletonList(source)) + .subAggregation(new TopHitsAggregationBuilder("top_hits").storedField("_none_")) + .aggregateAfter(afterKey) + .size(size); + }, (result) -> { + if (result.getBuckets().size() == 0) { + finish.set(true); + } + for (InternalComposite.InternalBucket bucket : result.getBuckets()) { + V term = transformKey.apply(bucket.getKey().get(field)); + seen.add(term); + assertThat(bucket.getDocCount(), equalTo(expectedDocCounts.get(term).get())); + } + }); + } + assertEquals(expected, seen); + } + + private void testSearchCase(List queries, List>> dataset, Supplier create, Consumer verify) throws IOException { - executeTestCase(false, null, query, dataset, create, verify); - executeTestCase(true, null, query, dataset, create, verify); - if (sort != null) { - executeTestCase(false, sort, query, dataset, create, verify); - executeTestCase(true, sort, query, dataset, create, verify); + for (Query query : queries) { + executeTestCase(false, query, dataset, create, verify); + executeTestCase(true, query, dataset, create, verify); } } private void executeTestCase(boolean reduced, - Sort sort, Query query, List>> dataset, Supplier create, Consumer verify) throws IOException { - IndexSettings indexSettings = createIndexSettings(sort); try (Directory directory = newDirectory()) { - IndexWriterConfig config = LuceneTestCase.newIndexWriterConfig(random(), new MockAnalyzer(random())); - if (sort != null) { - config.setIndexSort(sort); - /** - * Forces the default codec because {@link CompositeValuesSourceBuilder#checkCanEarlyTerminate} - * cannot detect single-valued field with the asserting-codec. - **/ - config.setCodec(TestUtil.getDefaultCodec()); - } - try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory, config)) { + try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory)) { Document document = new Document(); for (Map> fields : dataset) { addToDocument(document, fields); @@ -1295,12 +1462,8 @@ private void executeTestCase(boolean reduced, } } try (IndexReader indexReader = DirectoryReader.open(directory)) { - IndexSearcher indexSearcher = newSearcher(indexReader, sort == null, sort == null); + IndexSearcher indexSearcher = new IndexSearcher(indexReader); CompositeAggregationBuilder aggregationBuilder = create.get(); - if (sort != null) { - CompositeAggregator aggregator = createAggregator(query, aggregationBuilder, indexSearcher, indexSettings, FIELD_TYPES); - assertTrue(aggregator.canEarlyTerminate()); - } final InternalComposite composite; if (reduced) { composite = searchAndReduce(indexSearcher, query, aggregationBuilder, FIELD_TYPES); @@ -1312,31 +1475,22 @@ private void executeTestCase(boolean reduced, } } - private static IndexSettings createIndexSettings(Sort sort) { - Settings.Builder builder = Settings.builder(); - if (sort != null) { - String[] fields = Arrays.stream(sort.getSort()) - .map(SortField::getField) - .toArray(String[]::new); - String[] orders = Arrays.stream(sort.getSort()) - .map((o) -> o.getReverse() ? "desc" : "asc") - .toArray(String[]::new); - builder.putList("index.sort.field", fields); - builder.putList("index.sort.order", orders); - } - return IndexSettingsModule.newIndexSettings(new Index("_index", "0"), builder.build()); - } - private void addToDocument(Document doc, Map> keys) { for (Map.Entry> entry : keys.entrySet()) { final String name = entry.getKey(); for (Object value : entry.getValue()) { - if (value instanceof Long) { + if (value instanceof Integer) { + doc.add(new SortedNumericDocValuesField(name, (int) value)); + doc.add(new IntPoint(name, (int) value)); + } else if (value instanceof Long) { doc.add(new SortedNumericDocValuesField(name, (long) value)); + doc.add(new LongPoint(name, (long) value)); } else if (value instanceof Double) { doc.add(new SortedNumericDocValuesField(name, NumericUtils.doubleToSortableLong((double) value))); + doc.add(new DoublePoint(name, (double) value)); } else if (value instanceof String) { doc.add(new SortedSetDocValuesField(name, new BytesRef((String) value))); + doc.add(new StringField(name, new BytesRef((String) value), Field.Store.NO)); } else { throw new AssertionError("invalid object: " + value.getClass().getSimpleName()); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueueTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueueTests.java new file mode 100644 index 0000000000000..122c8185e2e70 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueueTests.java @@ -0,0 +1,330 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.analysis.core.KeywordAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.document.SortedSetDocValuesField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.NumericUtils; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.index.fielddata.FieldData; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.LeafBucketCollector; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import static org.elasticsearch.index.mapper.NumberFieldMapper.NumberType.DOUBLE; +import static org.elasticsearch.index.mapper.NumberFieldMapper.NumberType.LONG; +import static org.hamcrest.Matchers.equalTo; + +public class CompositeValuesCollectorQueueTests extends AggregatorTestCase { + static class ClassAndName { + final MappedFieldType fieldType; + final Class> clazz; + + ClassAndName(MappedFieldType fieldType, Class> clazz) { + this.fieldType = fieldType; + this.clazz = clazz; + } + } + + public void testRandomLong() throws IOException { + testRandomCase(new ClassAndName(createNumber("long", LONG) , Long.class)); + } + + public void testRandomDouble() throws IOException { + testRandomCase(new ClassAndName(createNumber("double", DOUBLE) , Double.class)); + } + + public void testRandomDoubleAndLong() throws IOException { + testRandomCase(new ClassAndName(createNumber("double", DOUBLE), Double.class), + new ClassAndName(createNumber("long", LONG), Long.class)); + } + + public void testRandomDoubleAndKeyword() throws IOException { + testRandomCase(new ClassAndName(createNumber("double", DOUBLE), Double.class), + new ClassAndName(createKeyword("keyword"), BytesRef.class)); + } + + public void testRandomKeyword() throws IOException { + testRandomCase(new ClassAndName(createKeyword("keyword"), BytesRef.class)); + } + + public void testRandomLongAndKeyword() throws IOException { + testRandomCase(new ClassAndName(createNumber("long", LONG), Long.class), + new ClassAndName(createKeyword("keyword"), BytesRef.class)); + } + + public void testRandomLongAndDouble() throws IOException { + testRandomCase(new ClassAndName(createNumber("long", LONG), Long.class), + new ClassAndName(createNumber("double", DOUBLE) , Double.class)); + } + + public void testRandomKeywordAndLong() throws IOException { + testRandomCase(new ClassAndName(createKeyword("keyword"), BytesRef.class), + new ClassAndName(createNumber("long", LONG), Long.class)); + } + + public void testRandomKeywordAndDouble() throws IOException { + testRandomCase(new ClassAndName(createKeyword("keyword"), BytesRef.class), + new ClassAndName(createNumber("double", DOUBLE), Double.class)); + } + + public void testRandom() throws IOException { + int numTypes = randomIntBetween(3, 8); + ClassAndName[] types = new ClassAndName[numTypes]; + for (int i = 0; i < numTypes; i++) { + int rand = randomIntBetween(0, 2); + switch (rand) { + case 0: + types[i] = new ClassAndName(createNumber(Integer.toString(i), LONG), Long.class); + break; + case 1: + types[i] = new ClassAndName(createNumber(Integer.toString(i), DOUBLE), Double.class); + break; + case 2: + types[i] = new ClassAndName(createKeyword(Integer.toString(i)), BytesRef.class); + break; + default: + assert(false); + } + } + testRandomCase(true, types); + } + + private void testRandomCase(ClassAndName... types) throws IOException { + testRandomCase(true, types); + testRandomCase(false, types); + } + + private void testRandomCase(boolean forceMerge, ClassAndName... types) throws IOException { + final BigArrays bigArrays = BigArrays.NON_RECYCLING_INSTANCE; + int numDocs = randomIntBetween(50, 100); + List[]> possibleValues = new ArrayList<>(); + for (ClassAndName type : types) { + int numValues = randomIntBetween(1, numDocs*2); + Comparable[] values = new Comparable[numValues]; + if (type.clazz == Long.class) { + for (int i = 0; i < numValues; i++) { + values[i] = randomLong(); + } + } else if (type.clazz == Double.class) { + for (int i = 0; i < numValues; i++) { + values[i] = randomDouble(); + } + } else if (type.clazz == BytesRef.class) { + for (int i = 0; i < numValues; i++) { + values[i] = new BytesRef(randomAlphaOfLengthBetween(5, 50)); + } + } else { + assert(false); + } + possibleValues.add(values); + } + + Set keys = new HashSet<>(); + try (Directory directory = newDirectory()) { + try (RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory, new KeywordAnalyzer())) { + for (int i = 0; i < numDocs; i++) { + Document document = new Document(); + List>> docValues = new ArrayList<>(); + boolean hasAllField = true; + for (int j = 0; j < types.length; j++) { + int numValues = randomIntBetween(0, 5); + if (numValues == 0) { + hasAllField = false; + } + List> values = new ArrayList<>(); + for (int k = 0; k < numValues; k++) { + values.add(possibleValues.get(j)[randomIntBetween(0, possibleValues.get(j).length-1)]); + if (types[j].clazz == Long.class) { + long value = (Long) values.get(k); + document.add(new SortedNumericDocValuesField(types[j].fieldType.name(), value)); + document.add(new LongPoint(types[j].fieldType.name(), value)); + } else if (types[j].clazz == Double.class) { + document.add(new SortedNumericDocValuesField(types[j].fieldType.name(), + NumericUtils.doubleToSortableLong((Double) values.get(k)))); + } else if (types[j].clazz == BytesRef.class) { + BytesRef value = (BytesRef) values.get(k); + document.add(new SortedSetDocValuesField(types[j].fieldType.name(), (BytesRef) values.get(k))); + document.add(new TextField(types[j].fieldType.name(), value.utf8ToString(), Field.Store.NO)); + } else { + assert(false); + } + } + docValues.add(values); + } + if (hasAllField) { + List comb = createListCombinations(docValues); + keys.addAll(comb); + } + indexWriter.addDocument(document); + } + if (forceMerge) { + indexWriter.forceMerge(1); + } + } + IndexReader reader = DirectoryReader.open(directory); + int size = randomIntBetween(1, keys.size()); + SingleDimensionValuesSource[] sources = new SingleDimensionValuesSource[types.length]; + for (int i = 0; i < types.length; i++) { + final MappedFieldType fieldType = types[i].fieldType; + if (types[i].clazz == Long.class) { + sources[i] = new LongValuesSource(bigArrays, fieldType, + context -> context.reader().getSortedNumericDocValues(fieldType.name()), value -> value, + DocValueFormat.RAW, size, 1); + } else if (types[i].clazz == Double.class) { + sources[i] = new DoubleValuesSource(bigArrays, fieldType, + context -> FieldData.sortableLongBitsToDoubles(context.reader().getSortedNumericDocValues(fieldType.name())), + size, 1); + } else if (types[i].clazz == BytesRef.class) { + if (forceMerge) { + // we don't create global ordinals but we test this mode when the reader has a single segment + // since ordinals are global in this case. + sources[i] = new GlobalOrdinalValuesSource(bigArrays, fieldType, + context -> context.reader().getSortedSetDocValues(fieldType.name()), size, 1); + } else { + sources[i] = new BinaryValuesSource(fieldType, + context -> FieldData.toString(context.reader().getSortedSetDocValues(fieldType.name())), size, 1); + } + } else { + assert(false); + } + } + CompositeKey[] expected = keys.toArray(new CompositeKey[0]); + Arrays.sort(expected, (a, b) -> compareKey(a, b)); + CompositeValuesCollectorQueue queue = new CompositeValuesCollectorQueue(sources, size); + final SortedDocsProducer docsProducer = sources[0].createSortedDocsProducerOrNull(reader, new MatchAllDocsQuery()); + for (boolean withProducer : new boolean[] {true, false}) { + if (withProducer && docsProducer == null) { + continue; + } + int pos = 0; + CompositeKey last = null; + while (pos < size) { + queue.clear(); + if (last != null) { + queue.setAfter(last.values()); + } + + for (LeafReaderContext leafReaderContext : reader.leaves()) { + final LeafBucketCollector leafCollector = new LeafBucketCollector() { + @Override + public void collect(int doc, long bucket) throws IOException { + queue.addIfCompetitive(); + } + }; + if (withProducer) { + assertEquals(DocIdSet.EMPTY, + docsProducer.processLeaf(new MatchAllDocsQuery(), queue, leafReaderContext, false)); + } else { + final LeafBucketCollector queueCollector = queue.getLeafCollector(leafReaderContext, leafCollector); + final Bits liveDocs = leafReaderContext.reader().getLiveDocs(); + for (int i = 0; i < leafReaderContext.reader().maxDoc(); i++) { + if (liveDocs == null || liveDocs.get(i)) { + queueCollector.collect(i); + } + } + } + } + assertEquals(size, Math.min(queue.size(), expected.length - pos)); + int ptr = 0; + for (int slot : queue.getSortedSlot()) { + CompositeKey key = queue.toCompositeKey(slot); + assertThat(key, equalTo(expected[ptr++])); + last = key; + } + pos += queue.size(); + } + } + reader.close(); + } + } + + private static MappedFieldType createNumber(String name, NumberFieldMapper.NumberType type) { + MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(type); + fieldType.setIndexOptions(IndexOptions.DOCS); + fieldType.setName(name); + fieldType.setHasDocValues(true); + fieldType.freeze(); + return fieldType; + } + + private static MappedFieldType createKeyword(String name) { + MappedFieldType fieldType = new KeywordFieldMapper.KeywordFieldType(); + fieldType.setIndexOptions(IndexOptions.DOCS); + fieldType.setName(name); + fieldType.setHasDocValues(true); + fieldType.freeze(); + return fieldType; + } + + private static int compareKey(CompositeKey key1, CompositeKey key2) { + assert key1.size() == key2.size(); + for (int i = 0; i < key1.size(); i++) { + Comparable cmp1 = (Comparable) key1.get(i); + int cmp = cmp1.compareTo(key2.get(i)); + if (cmp != 0) { + return cmp; + } + } + return 0; + } + + private static List createListCombinations(List>> values) { + List keys = new ArrayList<>(); + createListCombinations(new Comparable[values.size()], values, 0, values.size(), keys); + return keys; + } + + private static void createListCombinations(Comparable[] key, List>> values, + int pos, int maxPos, List keys) { + if (pos == maxPos) { + keys.add(new CompositeKey(key.clone())); + } else { + for (Comparable val : values.get(pos)) { + key[pos] = val; + createListCombinations(key, values, pos + 1, maxPos, keys); + } + } + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/SingleDimensionValuesSourceTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/SingleDimensionValuesSourceTests.java new file mode 100644 index 0000000000000..2fd14fe6b697d --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/SingleDimensionValuesSourceTests.java @@ -0,0 +1,106 @@ +/* + * 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.search.aggregations.bucket.composite; + +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.TermQuery; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.test.ESTestCase; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class SingleDimensionValuesSourceTests extends ESTestCase { + public void testBinarySorted() { + MappedFieldType keyword = new KeywordFieldMapper.KeywordFieldType(); + keyword.setName("keyword"); + BinaryValuesSource source = new BinaryValuesSource(keyword, context -> null, 1, 1); + assertNull(source.createSortedDocsProducerOrNull(mockIndexReader(100, 49), null)); + IndexReader reader = mockIndexReader(1, 1); + assertNotNull(source.createSortedDocsProducerOrNull(reader, new MatchAllDocsQuery())); + assertNotNull(source.createSortedDocsProducerOrNull(reader, null)); + assertNull(source.createSortedDocsProducerOrNull(reader, + new TermQuery(new Term("keyword", "toto)")))); + source = new BinaryValuesSource(keyword, context -> null, 0, -1); + assertNull(source.createSortedDocsProducerOrNull(reader, null)); + } + + public void testGlobalOrdinalsSorted() { + MappedFieldType keyword = new KeywordFieldMapper.KeywordFieldType(); + keyword.setName("keyword"); + BinaryValuesSource source = new BinaryValuesSource(keyword, context -> null, 1, 1); + assertNull(source.createSortedDocsProducerOrNull(mockIndexReader(100, 49), null)); + IndexReader reader = mockIndexReader(1, 1); + assertNotNull(source.createSortedDocsProducerOrNull(reader, new MatchAllDocsQuery())); + assertNotNull(source.createSortedDocsProducerOrNull(reader, null)); + assertNull(source.createSortedDocsProducerOrNull(reader, + new TermQuery(new Term("keyword", "toto)")))); + source = new BinaryValuesSource(keyword, context -> null, 1, -1); + assertNull(source.createSortedDocsProducerOrNull(reader, null)); + } + + public void testNumericSorted() { + for (NumberFieldMapper.NumberType numberType : NumberFieldMapper.NumberType.values()) { + MappedFieldType number = new NumberFieldMapper.NumberFieldType(NumberFieldMapper.NumberType.LONG); + number.setName("number"); + final SingleDimensionValuesSource source; + if (numberType == NumberFieldMapper.NumberType.BYTE || + numberType == NumberFieldMapper.NumberType.SHORT || + numberType == NumberFieldMapper.NumberType.INTEGER || + numberType == NumberFieldMapper.NumberType.LONG) { + source = new LongValuesSource(BigArrays.NON_RECYCLING_INSTANCE, + number, context -> null, value -> value, DocValueFormat.RAW, 1, 1); + assertNull(source.createSortedDocsProducerOrNull(mockIndexReader(100, 49), null)); + IndexReader reader = mockIndexReader(1, 1); + assertNotNull(source.createSortedDocsProducerOrNull(reader, new MatchAllDocsQuery())); + assertNotNull(source.createSortedDocsProducerOrNull(reader, null)); + assertNotNull(source.createSortedDocsProducerOrNull(reader, LongPoint.newRangeQuery("number", 0, 1))); + assertNull(source.createSortedDocsProducerOrNull(reader, new TermQuery(new Term("keyword", "toto)")))); + LongValuesSource sourceRev = + new LongValuesSource(BigArrays.NON_RECYCLING_INSTANCE, + number, context -> null, value -> value, DocValueFormat.RAW, 1, -1); + assertNull(sourceRev.createSortedDocsProducerOrNull(reader, null)); + } else if (numberType == NumberFieldMapper.NumberType.HALF_FLOAT || + numberType == NumberFieldMapper.NumberType.FLOAT || + numberType == NumberFieldMapper.NumberType.DOUBLE) { + source = new DoubleValuesSource(BigArrays.NON_RECYCLING_INSTANCE, + number, context -> null, 1, 1); + } else{ + throw new AssertionError ("missing type:" + numberType.typeName()); + } + assertNull(source.createSortedDocsProducerOrNull(mockIndexReader(100, 49), null)); + } + } + + private static IndexReader mockIndexReader(int maxDoc, int numDocs) { + IndexReader reader = mock(IndexReader.class); + when(reader.hasDeletions()).thenReturn(maxDoc - numDocs > 0); + when(reader.maxDoc()).thenReturn(maxDoc); + when(reader.numDocs()).thenReturn(numDocs); + return reader; + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 720d701e64ced..1940c82438839 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -291,7 +291,6 @@ protected A search(IndexSe A internalAgg = (A) a.buildAggregation(0L); InternalAggregationTestCase.assertMultiBucketConsumer(internalAgg, bucketConsumer); return internalAgg; - } protected A searchAndReduce(IndexSearcher searcher, From 318b0af953118bfc2233d7639b200a4bbf88a1fb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 15 Mar 2018 18:38:57 +0100 Subject: [PATCH 02/22] Remove execute mode bit from source files Some source files seem to have the execute bit (a+x) set, which doesn't really seem to hurt but is a bit odd. This change removes those, making the permissions similar to other source files in the repository. --- .../src/main/java/org/elasticsearch/client/Request.java | 0 .../main/java/org/elasticsearch/client/RestHighLevelClient.java | 0 .../src/test/java/org/elasticsearch/client/IndicesClientIT.java | 0 .../src/test/java/org/elasticsearch/client/RequestTests.java | 0 .../java/org/elasticsearch/discovery/ec2/Ec2NameResolver.java | 0 .../main/resources/rest-api-spec/test/cat.aliases/10_basic.yml | 0 .../main/resources/rest-api-spec/test/cat.allocation/10_basic.yml | 0 .../src/main/resources/rest-api-spec/test/cat.count/10_basic.yml | 0 .../main/resources/rest-api-spec/test/cat.nodeattrs/10_basic.yml | 0 .../src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yml | 0 .../main/resources/rest-api-spec/test/cat.recovery/10_basic.yml | 0 .../resources/rest-api-spec/test/cat.repositories/10_basic.yml | 0 .../main/resources/rest-api-spec/test/cat.segments/10_basic.yml | 0 .../src/main/resources/rest-api-spec/test/cat.shards/10_basic.yml | 0 .../main/resources/rest-api-spec/test/cat.snapshots/10_basic.yml | 0 .../src/main/resources/rest-api-spec/test/cat.tasks/10_basic.yml | 0 .../resources/rest-api-spec/test/cat.thread_pool/10_basic.yml | 0 .../cluster/repositories/delete/DeleteRepositoryResponse.java | 0 .../action/admin/indices/delete/DeleteIndexResponse.java | 0 .../elasticsearch/action/support/master/AcknowledgedResponse.java | 0 .../main/java/org/elasticsearch/index/mapper/MapperService.java | 0 .../action/admin/indices/delete/DeleteIndexResponseTests.java | 0 .../test/java/org/elasticsearch/index/mapper/ExternalMapper.java | 0 .../org/elasticsearch/index/mapper/FakeStringFieldMapper.java | 0 .../resources/indices/analyze/conf_dir/hunspell/en_US/en_US.aff | 0 .../resources/indices/analyze/conf_dir/hunspell/en_US/en_US.dic | 0 .../indices/analyze/conf_dir/hunspell/en_US_custom/en_US.aff | 0 .../indices/analyze/conf_dir/hunspell/en_US_custom/en_US.dic | 0 .../indices/analyze/no_aff_conf_dir/hunspell/en_US/en_US.dic | 0 .../indices/analyze/two_aff_conf_dir/hunspell/en_US/en_AU.aff | 0 .../indices/analyze/two_aff_conf_dir/hunspell/en_US/en_US.aff | 0 .../indices/analyze/two_aff_conf_dir/hunspell/en_US/en_US.dic | 0 32 files changed, 0 insertions(+), 0 deletions(-) mode change 100755 => 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java mode change 100755 => 100644 client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java mode change 100755 => 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java mode change 100755 => 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/RequestTests.java mode change 100755 => 100644 plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2NameResolver.java mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.allocation/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.count/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodeattrs/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.recovery/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.repositories/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.segments/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.snapshots/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.tasks/10_basic.yml mode change 100755 => 100644 rest-api-spec/src/main/resources/rest-api-spec/test/cat.thread_pool/10_basic.yml mode change 100755 => 100644 server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryResponse.java mode change 100755 => 100644 server/src/main/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexResponse.java mode change 100755 => 100644 server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedResponse.java mode change 100755 => 100644 server/src/main/java/org/elasticsearch/index/mapper/MapperService.java mode change 100755 => 100644 server/src/test/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexResponseTests.java mode change 100755 => 100644 server/src/test/java/org/elasticsearch/index/mapper/ExternalMapper.java mode change 100755 => 100644 server/src/test/java/org/elasticsearch/index/mapper/FakeStringFieldMapper.java mode change 100755 => 100644 server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US/en_US.aff mode change 100755 => 100644 server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US/en_US.dic mode change 100755 => 100644 server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US_custom/en_US.aff mode change 100755 => 100644 server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US_custom/en_US.dic mode change 100755 => 100644 server/src/test/resources/indices/analyze/no_aff_conf_dir/hunspell/en_US/en_US.dic mode change 100755 => 100644 server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_AU.aff mode change 100755 => 100644 server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_US.aff mode change 100755 => 100644 server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_US.dic diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/Request.java old mode 100755 new mode 100644 diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java old mode 100755 new mode 100644 diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java old mode 100755 new mode 100644 diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestTests.java old mode 100755 new mode 100644 diff --git a/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2NameResolver.java b/plugins/discovery-ec2/src/main/java/org/elasticsearch/discovery/ec2/Ec2NameResolver.java old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.aliases/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.allocation/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.allocation/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.count/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.count/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodeattrs/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodeattrs/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.nodes/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.recovery/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.recovery/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.repositories/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.repositories/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.segments/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.segments/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.snapshots/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.snapshots/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.tasks/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.tasks/10_basic.yml old mode 100755 new mode 100644 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.thread_pool/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.thread_pool/10_basic.yml old mode 100755 new mode 100644 diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/delete/DeleteRepositoryResponse.java old mode 100755 new mode 100644 diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexResponse.java old mode 100755 new mode 100644 diff --git a/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedResponse.java b/server/src/main/java/org/elasticsearch/action/support/master/AcknowledgedResponse.java old mode 100755 new mode 100644 diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java old mode 100755 new mode 100644 diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/delete/DeleteIndexResponseTests.java old mode 100755 new mode 100644 diff --git a/server/src/test/java/org/elasticsearch/index/mapper/ExternalMapper.java b/server/src/test/java/org/elasticsearch/index/mapper/ExternalMapper.java old mode 100755 new mode 100644 diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FakeStringFieldMapper.java b/server/src/test/java/org/elasticsearch/index/mapper/FakeStringFieldMapper.java old mode 100755 new mode 100644 diff --git a/server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US/en_US.aff b/server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US/en_US.aff old mode 100755 new mode 100644 diff --git a/server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US/en_US.dic b/server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US/en_US.dic old mode 100755 new mode 100644 diff --git a/server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US_custom/en_US.aff b/server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US_custom/en_US.aff old mode 100755 new mode 100644 diff --git a/server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US_custom/en_US.dic b/server/src/test/resources/indices/analyze/conf_dir/hunspell/en_US_custom/en_US.dic old mode 100755 new mode 100644 diff --git a/server/src/test/resources/indices/analyze/no_aff_conf_dir/hunspell/en_US/en_US.dic b/server/src/test/resources/indices/analyze/no_aff_conf_dir/hunspell/en_US/en_US.dic old mode 100755 new mode 100644 diff --git a/server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_AU.aff b/server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_AU.aff old mode 100755 new mode 100644 diff --git a/server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_US.aff b/server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_US.aff old mode 100755 new mode 100644 diff --git a/server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_US.dic b/server/src/test/resources/indices/analyze/two_aff_conf_dir/hunspell/en_US/en_US.dic old mode 100755 new mode 100644 From a9392f6d429f08e750a2e43bed4b224a9b0c70e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 23 Mar 2018 16:40:58 +0100 Subject: [PATCH 03/22] Add file permissions checks to precommit task This adds a check for source files that have the execute bit set to the precommit task. --- .../precommit/FilePermissionsTask.groovy | 87 +++++++++++++++++++ .../gradle/precommit/PrecommitTasks.groovy | 1 + 2 files changed, 88 insertions(+) create mode 100644 buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/FilePermissionsTask.groovy diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/FilePermissionsTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/FilePermissionsTask.groovy new file mode 100644 index 0000000000000..d8da9a4207bf7 --- /dev/null +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/FilePermissionsTask.groovy @@ -0,0 +1,87 @@ +/* + * 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.gradle.precommit + +import org.gradle.api.DefaultTask +import org.gradle.api.GradleException +import org.gradle.api.file.FileCollection +import org.gradle.api.tasks.InputFiles +import org.gradle.api.tasks.OutputFile +import org.gradle.api.tasks.SourceSet +import org.gradle.api.tasks.TaskAction +import org.gradle.api.tasks.util.PatternSet +import org.gradle.api.tasks.util.PatternFilterable +import org.apache.tools.ant.taskdefs.condition.Os + +import java.nio.file.Files +import java.nio.file.attribute.PosixFilePermission +import java.nio.file.attribute.PosixFileAttributeView + +import static java.nio.file.attribute.PosixFilePermission.OTHERS_EXECUTE +import static java.nio.file.attribute.PosixFilePermission.GROUP_EXECUTE +import static java.nio.file.attribute.PosixFilePermission.OWNER_EXECUTE + +/** + * Checks source files for correct file permissions. + */ +public class FilePermissionsTask extends DefaultTask { + + /** A pattern set of which files should be checked. */ + private PatternFilterable filesFilter = new PatternSet() + + @OutputFile + File outputMarker = new File(project.buildDir, 'markers/filePermissions') + + FilePermissionsTask() { + onlyIf { !Os.isFamily(Os.FAMILY_WINDOWS) } + description = "Checks java source files for correct file permissions" + // we always include all source files, and exclude what should not be checked + filesFilter.include('**') + // exclude sh files that might have the executable bit set + filesFilter.exclude('**/*.sh') + } + + /** Returns the files this task will check */ + @InputFiles + FileCollection files() { + List collections = new ArrayList<>() + for (SourceSet sourceSet : project.sourceSets) { + collections.add(sourceSet.allSource.matching(filesFilter)) + } + return project.files(collections.toArray()) + } + + @TaskAction + void checkInvalidPermissions() { + List failures = new ArrayList<>() + for (File f : files()) { + PosixFileAttributeView fileAttributeView = Files.getFileAttributeView(f.toPath(), PosixFileAttributeView.class) + Set permissions = fileAttributeView.readAttributes().permissions() + if (permissions.contains(OTHERS_EXECUTE) || permissions.contains(OWNER_EXECUTE) || + permissions.contains(GROUP_EXECUTE)) { + failures.add("Source file is executable: " + f) + } + } + if (failures.isEmpty() == false) { + throw new GradleException('Found invalid file permissions:\n' + failures.join('\n')) + } + outputMarker.setText('done', 'UTF-8') + } + +} diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy index 9e1cdad04fd6c..ef6f24c5acf5a 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy @@ -37,6 +37,7 @@ class PrecommitTasks { configureNamingConventions(project), project.tasks.create('forbiddenPatterns', ForbiddenPatternsTask.class), project.tasks.create('licenseHeaders', LicenseHeadersTask.class), + project.tasks.create('filepermissions', FilePermissionsTask.class), project.tasks.create('jarHell', JarHellTask.class), project.tasks.create('thirdPartyAudit', ThirdPartyAuditTask.class)] From f5d4550e93e4ae84f42b9512a3a3f7344cd62be3 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 26 Mar 2018 14:08:03 +0200 Subject: [PATCH 04/22] Fold EngineDiskUtils into Store, for better lock semantics (#29156) #28245 has introduced the utility class`EngineDiskUtils` with a set of methods to prepare/change translog and lucene commit points. That util class bundled everything that's needed to create and empty shard, bootstrap a shard from a lucene index that was just restored etc. In order to safely do these manipulations, the util methods acquired the IndexWriter's lock. That would sometime fail due to concurrent shard store fetching or other short activities that require the files not to be changed while they read from them. Since there is no way to wait on the index writer lock, the `Store` class has other locks to make sure that once we try to acquire the IW lock, it will succeed. To side step this waiting problem, this PR folds `EngineDiskUtils` into `Store`. Sadly this comes with a price - the store class doesn't and shouldn't know about the translog. As such the logic is slightly less tight and callers have to do the translog manipulations on their own. --- docs/reference/indices/flush.asciidoc | 2 +- .../org/elasticsearch/index/IndexService.java | 2 +- .../index/engine/EngineDiskUtils.java | 144 ------------ .../index/shard/StoreRecovery.java | 21 +- .../org/elasticsearch/index/store/Store.java | 104 ++++++++- .../indices/recovery/RecoveryTarget.java | 8 +- .../index/engine/EngineDiskUtilsTests.java | 207 ------------------ .../index/engine/InternalEngineTests.java | 98 ++++++++- .../index/shard/IndexShardTests.java | 12 +- .../index/shard/RefreshListenersTests.java | 9 +- .../elasticsearch/index/store/StoreTests.java | 58 ++++- .../recovery/RecoverySourceHandlerTests.java | 2 +- .../index/engine/EngineTestCase.java | 12 +- .../index/shard/IndexShardTestCase.java | 2 +- 14 files changed, 293 insertions(+), 388 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java delete mode 100644 server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index 91fac0908ef7f..87b1e90a4d517 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -93,7 +93,7 @@ which returns something similar to: { "commit" : { "id" : "3M3zkw2GHMo2Y4h4/KFKCg==", - "generation" : 3, + "generation" : 4, "user_data" : { "translog_uuid" : "hnOG3xFcTDeoI_kvvvOdNA", "history_uuid" : "XP7KDJGiS1a2fHYiFL5TXQ", diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 4fe50b983dfac..64760629bfd24 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -24,7 +24,6 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.Accountable; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -40,6 +39,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; import org.elasticsearch.env.ShardLockObtainFailedException; diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java deleted file mode 100644 index f7f3aa8e9fe1d..0000000000000 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.NoMergePolicy; -import org.apache.lucene.store.Directory; -import org.elasticsearch.Assertions; -import org.elasticsearch.common.UUIDs; -import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.Translog; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - - -/** - * This class contains utility methods for mutating the shard lucene index and translog as a preparation to be opened. - */ -public abstract class EngineDiskUtils { - - /** - * creates an empty lucene index and a corresponding empty translog. Any existing data will be deleted. - */ - public static void createEmpty(final Directory dir, final Path translogPath, final ShardId shardId) throws IOException { - try (IndexWriter writer = newIndexWriter(true, dir)) { - final String translogUuid = Translog.createEmptyTranslog(translogPath, SequenceNumbers.NO_OPS_PERFORMED, shardId); - final Map map = new HashMap<>(); - map.put(Translog.TRANSLOG_GENERATION_KEY, "1"); - map.put(Translog.TRANSLOG_UUID_KEY, translogUuid); - map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); - map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(SequenceNumbers.NO_OPS_PERFORMED)); - map.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(SequenceNumbers.NO_OPS_PERFORMED)); - map.put(InternalEngine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, "-1"); - updateCommitData(writer, map); - } - } - - - /** - * Converts an existing lucene index and marks it with a new history uuid. Also creates a new empty translog file. - * This is used to make sure no existing shard will recovery from this index using ops based recovery. - */ - public static void bootstrapNewHistoryFromLuceneIndex(final Directory dir, final Path translogPath, final ShardId shardId) - throws IOException { - try (IndexWriter writer = newIndexWriter(false, dir)) { - final Map userData = getUserData(writer); - final long maxSeqNo = Long.parseLong(userData.get(SequenceNumbers.MAX_SEQ_NO)); - final String translogUuid = Translog.createEmptyTranslog(translogPath, maxSeqNo, shardId); - final Map map = new HashMap<>(); - map.put(Translog.TRANSLOG_GENERATION_KEY, "1"); - map.put(Translog.TRANSLOG_UUID_KEY, translogUuid); - map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); - map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(maxSeqNo)); - updateCommitData(writer, map); - } - } - - /** - * Creates a new empty translog and associates it with an existing lucene index. - */ - public static void createNewTranslog(final Directory dir, final Path translogPath, long initialGlobalCheckpoint, final ShardId shardId) - throws IOException { - if (Assertions.ENABLED) { - final List existingCommits = DirectoryReader.listCommits(dir); - assert existingCommits.size() == 1 : "creating a translog translog should have one commit, commits[" + existingCommits + "]"; - SequenceNumbers.CommitInfo commitInfo = Store.loadSeqNoInfo(existingCommits.get(0)); - assert commitInfo.localCheckpoint >= initialGlobalCheckpoint : - "trying to create a shard whose local checkpoint [" + commitInfo.localCheckpoint + "] is < global checkpoint [" - + initialGlobalCheckpoint + "]"; - } - - try (IndexWriter writer = newIndexWriter(false, dir)) { - final String translogUuid = Translog.createEmptyTranslog(translogPath, initialGlobalCheckpoint, shardId); - final Map map = new HashMap<>(); - map.put(Translog.TRANSLOG_GENERATION_KEY, "1"); - map.put(Translog.TRANSLOG_UUID_KEY, translogUuid); - updateCommitData(writer, map); - } - } - - - /** - * Checks that the Lucene index contains a history uuid marker. If not, a new one is generated and committed. - */ - public static void ensureIndexHasHistoryUUID(final Directory dir) throws IOException { - try (IndexWriter writer = newIndexWriter(false, dir)) { - final Map userData = getUserData(writer); - if (userData.containsKey(Engine.HISTORY_UUID_KEY) == false) { - updateCommitData(writer, Collections.singletonMap(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID())); - } - } - } - - private static void updateCommitData(IndexWriter writer, Map keysToUpdate) throws IOException { - final Map userData = getUserData(writer); - userData.putAll(keysToUpdate); - writer.setLiveCommitData(userData.entrySet()); - writer.commit(); - } - - private static Map getUserData(IndexWriter writer) { - final Map userData = new HashMap<>(); - writer.getLiveCommitData().forEach(e -> userData.put(e.getKey(), e.getValue())); - return userData; - } - - private static IndexWriter newIndexWriter(final boolean create, final Directory dir) throws IOException { - IndexWriterConfig iwc = new IndexWriterConfig(null) - .setCommitOnClose(false) - // we don't want merges to happen here - we call maybe merge on the engine - // later once we stared it up otherwise we would need to wait for it here - // we also don't specify a codec here and merges should use the engines for this index - .setMergePolicy(NoMergePolicy.INSTANCE) - .setOpenMode(create ? IndexWriterConfig.OpenMode.CREATE : IndexWriterConfig.OpenMode.APPEND); - return new IndexWriter(dir, iwc); - } -} diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index 224ae60a420d1..3654aeba2bf8d 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -40,13 +40,13 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; -import org.elasticsearch.index.engine.EngineDiskUtils; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.snapshots.IndexShardRestoreFailedException; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; @@ -390,7 +390,11 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe recoveryState.getIndex().updateVersion(version); if (recoveryState.getRecoverySource().getType() == RecoverySource.Type.LOCAL_SHARDS) { assert indexShouldExists; - EngineDiskUtils.bootstrapNewHistoryFromLuceneIndex(store.directory(), indexShard.shardPath().resolveTranslog(), shardId); + store.bootstrapNewHistory(); + final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); + final long maxSeqNo = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.MAX_SEQ_NO)); + final String translogUUID = Translog.createEmptyTranslog(indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId); + store.associateIndexWithNewTranslog(translogUUID); } else if (indexShouldExists) { // since we recover from local, just fill the files and size try { @@ -402,7 +406,10 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe logger.debug("failed to list file details", e); } } else { - EngineDiskUtils.createEmpty(store.directory(), indexShard.shardPath().resolveTranslog(), shardId); + store.createEmpty(); + final String translogUUID = Translog.createEmptyTranslog(indexShard.shardPath().resolveTranslog(), + SequenceNumbers.NO_OPS_PERFORMED, shardId); + store.associateIndexWithNewTranslog(translogUUID); } indexShard.openEngineAndRecoverFromTranslog(); indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm()); @@ -445,8 +452,12 @@ private void restore(final IndexShard indexShard, final Repository repository, f } final IndexId indexId = repository.getRepositoryData().resolveIndexId(indexName); repository.restoreShard(indexShard, restoreSource.snapshot().getSnapshotId(), restoreSource.version(), indexId, snapshotShardId, indexShard.recoveryState()); - EngineDiskUtils.bootstrapNewHistoryFromLuceneIndex(indexShard.store().directory(), indexShard.shardPath().resolveTranslog(), - shardId); + final Store store = indexShard.store(); + store.bootstrapNewHistory(); + final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); + final long maxSeqNo = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.MAX_SEQ_NO)); + final String translogUUID = Translog.createEmptyTranslog(indexShard.shardPath().resolveTranslog(), maxSeqNo, shardId); + store.associateIndexWithNewTranslog(translogUUID); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; indexShard.openEngineAndRecoverFromTranslog(); indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm()); diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index e560a0b040b0b..297790890c1b0 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -30,6 +30,8 @@ import org.apache.lucene.index.IndexFormatTooOldException; import org.apache.lucene.index.IndexNotFoundException; import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.AlreadyClosedException; @@ -46,7 +48,6 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; -import org.elasticsearch.core.internal.io.IOUtils; import org.apache.lucene.util.Version; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; @@ -69,11 +70,13 @@ import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.RefCounted; import org.elasticsearch.common.util.iterable.Iterables; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.IndexShard; @@ -155,7 +158,8 @@ public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService dire this(shardId, indexSettings, directoryService, shardLock, OnClose.EMPTY); } - public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock, OnClose onClose) throws IOException { + public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock, + 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)); @@ -1454,4 +1458,100 @@ private static long estimateSize(Directory directory) throws IOException { } } + /** + * creates an empty lucene index and a corresponding empty translog. Any existing data will be deleted. + */ + public void createEmpty() throws IOException { + metadataLock.writeLock().lock(); + try (IndexWriter writer = newIndexWriter(IndexWriterConfig.OpenMode.CREATE, directory)) { + final Map map = new HashMap<>(); + map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); + map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(SequenceNumbers.NO_OPS_PERFORMED)); + map.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(SequenceNumbers.NO_OPS_PERFORMED)); + map.put(InternalEngine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, "-1"); + updateCommitData(writer, map); + } finally { + metadataLock.writeLock().unlock(); + } + } + + + /** + * Marks an existing lucene index with a new history uuid. + * This is used to make sure no existing shard will recovery from this index using ops based recovery. + */ + public void bootstrapNewHistory() throws IOException { + metadataLock.writeLock().lock(); + try (IndexWriter writer = newIndexWriter(IndexWriterConfig.OpenMode.APPEND, directory)) { + final Map userData = getUserData(writer); + final long maxSeqNo = Long.parseLong(userData.get(SequenceNumbers.MAX_SEQ_NO)); + final Map map = new HashMap<>(); + map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); + map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(maxSeqNo)); + updateCommitData(writer, map); + } finally { + metadataLock.writeLock().unlock(); + } + } + + /** + * Force bakes the given translog generation as recovery information in the lucene index. This is + * used when recovering from a snapshot or peer file based recovery where a new empty translog is + * created and the existing lucene index needs should be changed to use it. + */ + public void associateIndexWithNewTranslog(final String translogUUID) throws IOException { + metadataLock.writeLock().lock(); + try (IndexWriter writer = newIndexWriter(IndexWriterConfig.OpenMode.APPEND, directory)) { + if (translogUUID.equals(getUserData(writer).get(Translog.TRANSLOG_UUID_KEY))) { + throw new IllegalArgumentException("a new translog uuid can't be equal to existing one. got [" + translogUUID + "]"); + } + final Map map = new HashMap<>(); + map.put(Translog.TRANSLOG_GENERATION_KEY, "1"); + map.put(Translog.TRANSLOG_UUID_KEY, translogUUID); + updateCommitData(writer, map); + } finally { + metadataLock.writeLock().unlock(); + } + } + + + /** + * Checks that the Lucene index contains a history uuid marker. If not, a new one is generated and committed. + */ + public void ensureIndexHasHistoryUUID() throws IOException { + metadataLock.writeLock().lock(); + try (IndexWriter writer = newIndexWriter(IndexWriterConfig.OpenMode.APPEND, directory)) { + final Map userData = getUserData(writer); + if (userData.containsKey(Engine.HISTORY_UUID_KEY) == false) { + updateCommitData(writer, Collections.singletonMap(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID())); + } + } finally { + metadataLock.writeLock().unlock(); + } + } + + private void updateCommitData(IndexWriter writer, Map keysToUpdate) throws IOException { + final Map userData = getUserData(writer); + userData.putAll(keysToUpdate); + writer.setLiveCommitData(userData.entrySet()); + writer.commit(); + } + + private Map getUserData(IndexWriter writer) { + final Map userData = new HashMap<>(); + writer.getLiveCommitData().forEach(e -> userData.put(e.getKey(), e.getValue())); + return userData; + } + + private IndexWriter newIndexWriter(IndexWriterConfig.OpenMode openMode, final Directory dir) throws IOException { + IndexWriterConfig iwc = new IndexWriterConfig(null) + .setCommitOnClose(false) + // we don't want merges to happen here - we call maybe merge on the engine + // later once we stared it up otherwise we would need to wait for it here + // we also don't specify a codec here and merges should use the engines for this index + .setMergePolicy(NoMergePolicy.INSTANCE) + .setOpenMode(openMode); + return new IndexWriter(dir, iwc); + } + } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index b28e992d9fd5d..91d3332f8e646 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -40,7 +40,6 @@ import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.engine.EngineDiskUtils; import org.elasticsearch.index.mapper.MapperException; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -439,11 +438,12 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa try { store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); if (indexShard.indexSettings().getIndexVersionCreated().before(Version.V_6_0_0_rc1)) { - EngineDiskUtils.ensureIndexHasHistoryUUID(store.directory()); + store.ensureIndexHasHistoryUUID(); } // TODO: Assign the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2 - EngineDiskUtils.createNewTranslog(store.directory(), indexShard.shardPath().resolveTranslog(), - SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); + final String translogUUID = + Translog.createEmptyTranslog(indexShard.shardPath().resolveTranslog(), SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); + store.associateIndexWithNewTranslog(translogUUID); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. // this means we transferred files from the remote that have not be checksummed and they are diff --git a/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java b/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java deleted file mode 100644 index aca94708af9f8..0000000000000 --- a/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java +++ /dev/null @@ -1,207 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -import org.apache.lucene.index.IndexWriter; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.NoMergePolicy; -import org.apache.lucene.search.IndexSearcher; -import org.elasticsearch.Version; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.VersionType; -import org.elasticsearch.index.codec.CodecService; -import org.elasticsearch.index.mapper.ParsedDocument; -import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -import org.elasticsearch.test.IndexSettingsModule; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; - -import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.notNullValue; - -public class EngineDiskUtilsTests extends EngineTestCase { - - - public void testHistoryUUIDIsSetIfMissing() throws IOException { - final int numDocs = randomIntBetween(0, 3); - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - Engine.IndexResult index = engine.index(firstIndexRequest); - assertThat(index.getVersion(), equalTo(1L)); - } - assertVisibleCount(engine, numDocs); - engine.close(); - - IndexWriterConfig iwc = new IndexWriterConfig(null) - .setCommitOnClose(false) - // we don't want merges to happen here - we call maybe merge on the engine - // later once we stared it up otherwise we would need to wait for it here - // we also don't specify a codec here and merges should use the engines for this index - .setMergePolicy(NoMergePolicy.INSTANCE) - .setOpenMode(IndexWriterConfig.OpenMode.APPEND); - try (IndexWriter writer = new IndexWriter(store.directory(), iwc)) { - Map newCommitData = new HashMap<>(); - for (Map.Entry entry : writer.getLiveCommitData()) { - if (entry.getKey().equals(Engine.HISTORY_UUID_KEY) == false) { - newCommitData.put(entry.getKey(), entry.getValue()); - } - } - writer.setLiveCommitData(newCommitData.entrySet()); - writer.commit(); - } - - EngineDiskUtils.ensureIndexHasHistoryUUID(store.directory()); - - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_6_0_0_beta1) - .build()); - - EngineConfig config = engine.config(); - EngineConfig newConfig = new EngineConfig( - shardId, allocationId.getId(), - threadPool, indexSettings, null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), - new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), - IndexSearcher.getDefaultQueryCachingPolicy(), config.getTranslogConfig(), TimeValue.timeValueMinutes(5), - config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); - engine = new InternalEngine(newConfig); - engine.recoverFromTranslog(); - assertVisibleCount(engine, numDocs, false); - assertThat(engine.getHistoryUUID(), notNullValue()); - } - - public void testCurrentTranslogIDisCommitted() throws IOException { - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - try (Store store = createStore()) { - EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); - - // create - { - EngineDiskUtils.createEmpty(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); - ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - - try (InternalEngine engine = createEngine(config)) { - engine.index(firstIndexRequest); - globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); - expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog()); - Map userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - } - } - // open and recover tlog - { - for (int i = 0; i < 2; i++) { - try (InternalEngine engine = new InternalEngine(config)) { - assertTrue(engine.isRecovering()); - Map userData = engine.getLastCommittedSegmentInfos().getUserData(); - if (i == 0) { - assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - } else { - // creating an empty index will create the first translog gen and commit it - // opening the empty index will make the second translog file but not commit it - // opening the engine again (i=0) will make the third translog file, which then be committed - assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - } - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.recoverFromTranslog(); - userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - } - } - } - // open index with new tlog - { - EngineDiskUtils.createNewTranslog(store.directory(), config.getTranslogConfig().getTranslogPath(), - SequenceNumbers.NO_OPS_PERFORMED, shardId); - try (InternalEngine engine = new InternalEngine(config)) { - Map userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.recoverFromTranslog(); - assertEquals(2, engine.getTranslog().currentFileGeneration()); - assertEquals(0L, engine.getTranslog().stats().getUncommittedOperations()); - } - } - - // open and recover tlog with empty tlog - { - for (int i = 0; i < 2; i++) { - try (InternalEngine engine = new InternalEngine(config)) { - Map userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.recoverFromTranslog(); - userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("no changes - nothing to commit", "1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - } - } - } - } - } - - public void testHistoryUUIDCanBeForced() throws IOException { - final int numDocs = randomIntBetween(0, 3); - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, - Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - Engine.IndexResult index = engine.index(firstIndexRequest); - assertThat(index.getVersion(), equalTo(1L)); - } - assertVisibleCount(engine, numDocs); - final String oldHistoryUUID = engine.getHistoryUUID(); - engine.close(); - EngineConfig config = engine.config(); - EngineDiskUtils.bootstrapNewHistoryFromLuceneIndex(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); - - EngineConfig newConfig = new EngineConfig( - shardId, allocationId.getId(), - threadPool, config.getIndexSettings(), null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), - new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), - IndexSearcher.getDefaultQueryCachingPolicy(), config.getTranslogConfig(), TimeValue.timeValueMinutes(5), - config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); - engine = new InternalEngine(newConfig); - engine.recoverFromTranslog(); - assertVisibleCount(engine, 0, false); - assertThat(engine.getHistoryUUID(), not(equalTo(oldHistoryUUID))); - } -} diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index bc4ecbee4d6a8..c48a9ee8a2d19 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -65,7 +65,6 @@ import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.FixedBitSet; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; @@ -91,6 +90,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.codec.CodecService; @@ -1141,8 +1141,9 @@ public void testSyncedFlushSurvivesEngineRestart() throws IOException { engine.flushAndClose(); } if (randomBoolean()) { - EngineDiskUtils.createNewTranslog(store.directory(), config.getTranslogConfig().getTranslogPath(), + final String translogUUID = Translog.createEmptyTranslog(config.getTranslogConfig().getTranslogPath(), SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); + store.associateIndexWithNewTranslog(translogUUID); } engine = new InternalEngine(config); engine.recoverFromTranslog(); @@ -2354,6 +2355,84 @@ public void testSettings() { assertEquals(currentIndexWriterConfig.getCodec().getName(), codecService.codec(codecName).getName()); } + public void testCurrentTranslogIDisCommitted() throws IOException { + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + try (Store store = createStore()) { + EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); + + // create + { + store.createEmpty(); + final String translogUUID = + Translog.createEmptyTranslog(config.getTranslogConfig().getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId); + store.associateIndexWithNewTranslog(translogUUID); + ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + + try (InternalEngine engine = createEngine(config)) { + engine.index(firstIndexRequest); + globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); + expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog()); + Map userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + } + } + // open and recover tlog + { + for (int i = 0; i < 2; i++) { + try (InternalEngine engine = new InternalEngine(config)) { + assertTrue(engine.isRecovering()); + Map userData = engine.getLastCommittedSegmentInfos().getUserData(); + if (i == 0) { + assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + } else { + // creating an empty index will create the first translog gen and commit it + // opening the empty index will make the second translog file but not commit it + // opening the engine again (i=0) will make the third translog file, which then be committed + assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + } + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + engine.recoverFromTranslog(); + userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + } + } + } + // open index with new tlog + { + final String translogUUID = + Translog.createEmptyTranslog(config.getTranslogConfig().getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId); + store.associateIndexWithNewTranslog(translogUUID); + try (InternalEngine engine = new InternalEngine(config)) { + Map userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + engine.recoverFromTranslog(); + assertEquals(2, engine.getTranslog().currentFileGeneration()); + assertEquals(0L, engine.getTranslog().stats().getUncommittedOperations()); + } + } + + // open and recover tlog with empty tlog + { + for (int i = 0; i < 2; i++) { + try (InternalEngine engine = new InternalEngine(config)) { + Map userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + engine.recoverFromTranslog(); + userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("no changes - nothing to commit", "1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + } + } + } + } + } + public void testMissingTranslog() throws IOException { // test that we can force start the engine , even if the translog is missing. engine.close(); @@ -2369,7 +2448,8 @@ public void testMissingTranslog() throws IOException { // expected } // when a new translog is created it should be ok - EngineDiskUtils.createNewTranslog(store.directory(), primaryTranslogDir, SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); + final String translogUUID = Translog.createEmptyTranslog(primaryTranslogDir, SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); + store.associateIndexWithNewTranslog(translogUUID); EngineConfig config = config(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null); engine = new InternalEngine(config); } @@ -2432,7 +2512,9 @@ public void testTranslogCleanUpPostCommitCrash() throws Exception { final Path translogPath = createTempDir(); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get(); - EngineDiskUtils.createEmpty(store.directory(), translogPath, shardId); + store.createEmpty(); + final String translogUUID = Translog.createEmptyTranslog(translogPath, globalCheckpoint.get(), shardId); + store.associateIndexWithNewTranslog(translogUUID); try (InternalEngine engine = new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier)) { @@ -3223,7 +3305,8 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { } try (Store store = createStore(newFSDirectory(storeDir))) { if (randomBoolean() || true) { - EngineDiskUtils.createNewTranslog(store.directory(), translogDir, SequenceNumbers.NO_OPS_PERFORMED, shardId); + final String translogUUID = Translog.createEmptyTranslog(translogDir, SequenceNumbers.NO_OPS_PERFORMED, shardId); + store.associateIndexWithNewTranslog(translogUUID); } try (Engine engine = new InternalEngine(configSupplier.apply(store))) { assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); @@ -4025,10 +4108,12 @@ public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { final Path translogPath = createTempDir(); store = createStore(); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + store.createEmpty(); + final String translogUUID = Translog.createEmptyTranslog(translogPath, globalCheckpoint.get(), shardId); + store.associateIndexWithNewTranslog(translogUUID); final EngineConfig engineConfig = config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null, null, () -> globalCheckpoint.get()); - EngineDiskUtils.createEmpty(store.directory(), translogPath, shardId); try (Engine engine = new InternalEngine(engineConfig) { @Override protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException { @@ -4042,7 +4127,6 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s }) { engine.recoverFromTranslog(); int numDocs = scaledRandomIntBetween(10, 100); - final String translogUUID = engine.getTranslog().getTranslogUUID(); for (int docId = 0; docId < numDocs; docId++) { ParseContext.Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index f05fdc60c5cf7..822294a9c19f7 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -29,7 +29,6 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IOContext; import org.apache.lucene.util.Constants; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; @@ -70,6 +69,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; @@ -1059,27 +1059,27 @@ public void testSnapshotStore() throws IOException { DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); Store.MetadataSnapshot snapshot = newShard.snapshotStoreMetadata(); - assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_2")); + assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_3")); newShard.markAsRecovering("store", new RecoveryState(newShard.routingEntry(), localNode, null)); snapshot = newShard.snapshotStoreMetadata(); - assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_2")); + assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_3")); assertTrue(newShard.recoverFromStore()); snapshot = newShard.snapshotStoreMetadata(); - assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_2")); + assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_3")); IndexShardTestCase.updateRoutingEntry(newShard, newShard.routingEntry().moveToStarted()); snapshot = newShard.snapshotStoreMetadata(); - assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_2")); + assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_3")); newShard.close("test", false); snapshot = newShard.snapshotStoreMetadata(); - assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_2")); + assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_3")); closeShards(newShard); } diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 1f9c5ae6df359..0609477dda8e5 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -26,7 +26,6 @@ import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.store.Directory; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; @@ -37,12 +36,12 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; -import org.elasticsearch.index.engine.EngineDiskUtils; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -52,6 +51,7 @@ import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.DummyShardLock; @@ -121,7 +121,10 @@ public void onFailedEngine(String reason, @Nullable Exception e) { // we don't need to notify anybody in this test } }; - EngineDiskUtils.createEmpty(store.directory(), translogConfig.getTranslogPath(), shardId); + store.createEmpty(); + final String translogUUID = + Translog.createEmptyTranslog(translogConfig.getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId); + store.associateIndexWithNewTranslog(translogUUID); EngineConfig config = new EngineConfig(shardId, allocationId, threadPool, indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, diff --git a/server/src/test/java/org/elasticsearch/index/store/StoreTests.java b/server/src/test/java/org/elasticsearch/index/store/StoreTests.java index 392227396de15..9352d978e6e46 100644 --- a/server/src/test/java/org/elasticsearch/index/store/StoreTests.java +++ b/server/src/test/java/org/elasticsearch/index/store/StoreTests.java @@ -48,7 +48,6 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.core.internal.io.IOUtils; import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.Version; import org.elasticsearch.ExceptionsHelper; @@ -59,6 +58,7 @@ import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.ShardLock; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; @@ -93,7 +93,9 @@ import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; public class StoreTests extends ESTestCase { @@ -761,7 +763,8 @@ public void testStoreStats() throws IOException { Settings settings = Settings.builder() .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT) .put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMinutes(0)).build(); - Store store = new Store(shardId, IndexSettingsModule.newIndexSettings("index", settings), directoryService, new DummyShardLock(shardId)); + Store store = new Store(shardId, IndexSettingsModule.newIndexSettings("index", settings), directoryService, + new DummyShardLock(shardId)); long initialStoreSize = 0; for (String extraFiles : store.directory().listAll()) { assertTrue("expected extraFS file but got: " + extraFiles, extraFiles.startsWith("extra")); @@ -1071,4 +1074,55 @@ public Directory newDirectory() throws IOException { store.close(); } + public void testEnsureIndexHasHistoryUUID() throws IOException { + final ShardId shardId = new ShardId("index", "_na_", 1); + DirectoryService directoryService = new LuceneManagedDirectoryService(random()); + try (Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId))) { + + store.createEmpty(); + + // remove the history uuid + IndexWriterConfig iwc = new IndexWriterConfig(null) + .setCommitOnClose(false) + // we don't want merges to happen here - we call maybe merge on the engine + // later once we stared it up otherwise we would need to wait for it here + // we also don't specify a codec here and merges should use the engines for this index + .setMergePolicy(NoMergePolicy.INSTANCE) + .setOpenMode(IndexWriterConfig.OpenMode.APPEND); + try (IndexWriter writer = new IndexWriter(store.directory(), iwc)) { + Map newCommitData = new HashMap<>(); + for (Map.Entry entry : writer.getLiveCommitData()) { + if (entry.getKey().equals(Engine.HISTORY_UUID_KEY) == false) { + newCommitData.put(entry.getKey(), entry.getValue()); + } + } + writer.setLiveCommitData(newCommitData.entrySet()); + writer.commit(); + } + + store.ensureIndexHasHistoryUUID(); + + SegmentInfos segmentInfos = Lucene.readSegmentInfos(store.directory()); + assertThat(segmentInfos.getUserData(), hasKey(Engine.HISTORY_UUID_KEY)); + } + } + + public void testHistoryUUIDCanBeForced() throws IOException { + final ShardId shardId = new ShardId("index", "_na_", 1); + DirectoryService directoryService = new LuceneManagedDirectoryService(random()); + try (Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId))) { + + store.createEmpty(); + + SegmentInfos segmentInfos = Lucene.readSegmentInfos(store.directory()); + assertThat(segmentInfos.getUserData(), hasKey(Engine.HISTORY_UUID_KEY)); + final String oldHistoryUUID = segmentInfos.getUserData().get(Engine.HISTORY_UUID_KEY); + + store.bootstrapNewHistory(); + + segmentInfos = Lucene.readSegmentInfos(store.directory()); + assertThat(segmentInfos.getUserData(), hasKey(Engine.HISTORY_UUID_KEY)); + assertThat(segmentInfos.getUserData().get(Engine.HISTORY_UUID_KEY), not(equalTo(oldHistoryUUID))); + } + } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index d1dbaf6bc89fe..661a1f0635430 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -32,7 +32,6 @@ import org.apache.lucene.store.BaseDirectoryWrapper; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; @@ -48,6 +47,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.SegmentsStats; diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index c75e469f7aff4..f0e46cf022344 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -37,8 +37,6 @@ import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -51,6 +49,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; @@ -363,9 +362,14 @@ private InternalEngine createEngine(@Nullable IndexWriterFactory indexWriterFact @Nullable BiFunction localCheckpointTrackerSupplier, @Nullable ToLongBiFunction seqNoForOperation, EngineConfig config) throws IOException { - final Directory directory = config.getStore().directory(); + final Store store = config.getStore(); + final Directory directory = store.directory(); if (Lucene.indexExists(directory) == false) { - EngineDiskUtils.createEmpty(directory, config.getTranslogConfig().getTranslogPath(), config.getShardId()); + store.createEmpty(); + final String translogUuid = + Translog.createEmptyTranslog(config.getTranslogConfig().getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId); + store.associateIndexWithNewTranslog(translogUuid); + } InternalEngine internalEngine = createInternalEngine(indexWriterFactory, localCheckpointTrackerSupplier, seqNoForOperation, config); internalEngine.recoverFromTranslog(); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index b5ea5fd4c0eab..6d6cc36d78b1b 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -25,7 +25,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.Bits; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; @@ -46,6 +45,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; From dd77d7fd0a6bac9acccab9a89850a27780ef46ae Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Mon, 26 Mar 2018 14:32:59 +0200 Subject: [PATCH 05/22] #28745: remove extra option in the composite rest tests `allow_partial_search_results` is not needed for these tests. --- .../rest-api-spec/test/search.aggregation/230_composite.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml index 10fc9f33beb8b..b8c89517ec119 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/230_composite.yml @@ -99,7 +99,6 @@ setup: - do: search: index: test - allow_partial_search_results: false body: aggregations: test: From bca264699a600e228042d859cf77b690ef55b3e5 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 26 Mar 2018 17:06:32 +0200 Subject: [PATCH 06/22] remove testUnassignedShardAndEmptyNodesInRoutingTable testUnassignedShardAndEmptyNodesInRoutingTable and that test is as old as time and does a very bogus thing. it is an IT test which extracts the GatewayAllocator from the node and tells it to allocated unassigned shards, while giving it a conjured cluster state with no nodes in it (it uses the DiscoveryNodes.EMPTY_NODES. This is never a cluster state we want to reroute on (we always have at least master node in it). I'm going to just delete the test as I don't think it adds much value. Closes #21463 --- .../indices/state/RareClusterStateIT.java | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java b/server/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java index bf213b51475fb..afe421a2916b1 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/RareClusterStateIT.java @@ -93,24 +93,6 @@ protected int numberOfReplicas() { return 0; } - public void testUnassignedShardAndEmptyNodesInRoutingTable() throws Exception { - internalCluster().startNode(); - createIndex("a"); - ensureSearchable("a"); - ClusterState current = clusterService().state(); - GatewayAllocator allocator = internalCluster().getInstance(GatewayAllocator.class); - - AllocationDeciders allocationDeciders = new AllocationDeciders(Settings.EMPTY, Collections.emptyList()); - RoutingNodes routingNodes = new RoutingNodes( - ClusterState.builder(current) - .routingTable(RoutingTable.builder(current.routingTable()).remove("a").addAsRecovery(current.metaData().index("a")).build()) - .nodes(DiscoveryNodes.EMPTY_NODES) - .build(), false - ); - RoutingAllocation routingAllocation = new RoutingAllocation(allocationDeciders, routingNodes, current, ClusterInfo.EMPTY, System.nanoTime()); - allocator.allocateUnassigned(routingAllocation); - } - public void testAssignmentWithJustAddedNodes() throws Exception { internalCluster().startNode(); final String index = "index"; From 87957603c030ad0d9d4c755d8ad5920ceac909cb Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 26 Mar 2018 13:42:08 -0400 Subject: [PATCH 07/22] Prune only gc deletes below local checkpoint (#28790) Once a document is deleted and Lucene is refreshed, we will not be able to look up the `version/seq#` associated with that delete in Lucene. As conflicting operations can still be indexed, we need another mechanism to remember these deletes. Therefore deletes should still be stored in the Version Map, even after Lucene is refreshed. Obviously, we can't remember all deletes forever so a trimming mechanism is needed. Currently, we remember deletes for at least 1 minute (the default GC deletes cycle) and clean them periodically. This is, at the moment, the best we can do on the primary for user facing APIs but this arbitrary time limit is problematic for replicas. Furthermore, we can't rely on the primary and replicas doing the trimming in a synchronized manner, and failing to do so results in the replica and primary making different decisions. The following scenario can cause inconsistency between primary and replica. 1. Primary index doc (index, id=1, v2) 2. Network packet issue causes index operation to back off and wait 3. Primary deletes doc (delete, id=1, v3) 4. Replica processes delete (delete, id=1, v3) 5. 1+ minute passes (GC deletes runs replica) 6. Indexing op is finally sent to the replica which no processes it because it forgot about the delete. We can reply on sequence-numbers to prevent this issue. If we prune only deletes whose seqno at most the local checkpoint, a replica will correctly remember what it needs. The correctness is explained as follows: Suppose o1 and o2 are two operations on the same document with seq#(o1) < seq#(o2), and o2 arrives before o1 on the replica. o2 is processed normally since it arrives first; when o1 arrives it should be discarded: 1. If seq#(o1) <= LCP, then it will be not be added to Lucene, as it was already previously added. 2. If seq#(o1) > LCP, then it depends on the nature of o2: - If o2 is a delete then its seq# is recorded in the VersionMap, since seq#(o2) > seq#(o1) > LCP, so a lookup can find it and determine that o1 is stale. - If o2 is an indexing then its seq# is either in Lucene (if refreshed) or the VersionMap (if not refreshed yet), so a real-time lookup can find it and determine that o1 is stale. In this PR, we prefer to deploy a single trimming strategy, which satisfies both requirements, on primary and replicas because: - It's simpler - no need to distinguish if an engine is running at primary mode or replica mode or being promoted. - If a replica subsequently is promoted, user experience is fully maintained as that replica remembers deletes for the last GC cycle. However, the version map may consume less memory if we deploy two different trimming strategies for primary and replicas. --- .../index/engine/InternalEngine.java | 33 +++++++++- .../index/engine/LiveVersionMap.java | 18 +++-- .../index/engine/InternalEngineTests.java | 66 +++++++++++++++++++ .../index/engine/LiveVersionMapTests.java | 38 ++++++----- .../ESIndexLevelReplicationTestCase.java | 32 +++++++-- .../IndexLevelReplicationTests.java | 53 ++++++++++++++- .../index/engine/EngineTestCase.java | 4 ++ 7 files changed, 211 insertions(+), 33 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index cc5c4799479da..70cbebb774e02 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -80,6 +80,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -1540,15 +1541,41 @@ public void trimTranslog() throws EngineException { } private void pruneDeletedTombstones() { + /* + * We need to deploy two different trimming strategies for GC deletes on primary and replicas. Delete operations on primary + * are remembered for at least one GC delete cycle and trimmed periodically. This is, at the moment, the best we can do on + * primary for user facing APIs but this arbitrary time limit is problematic for replicas. On replicas however we should + * trim only deletes whose seqno at most the local checkpoint. This requirement is explained as follows. + * + * Suppose o1 and o2 are two operations on the same document with seq#(o1) < seq#(o2), and o2 arrives before o1 on the replica. + * o2 is processed normally since it arrives first; when o1 arrives it should be discarded: + * - If seq#(o1) <= LCP, then it will be not be added to Lucene, as it was already previously added. + * - If seq#(o1) > LCP, then it depends on the nature of o2: + * *) If o2 is a delete then its seq# is recorded in the VersionMap, since seq#(o2) > seq#(o1) > LCP, + * so a lookup can find it and determine that o1 is stale. + * *) If o2 is an indexing then its seq# is either in Lucene (if refreshed) or the VersionMap (if not refreshed yet), + * so a real-time lookup can find it and determine that o1 is stale. + * + * Here we prefer to deploy a single trimming strategy, which satisfies two constraints, on both primary and replicas because: + * - It's simpler - no need to distinguish if an engine is running at primary mode or replica mode or being promoted. + * - If a replica subsequently is promoted, user experience is maintained as that replica remembers deletes for the last GC cycle. + * + * However, the version map may consume less memory if we deploy two different trimming strategies for primary and replicas. + */ final long timeMSec = engineConfig.getThreadPool().relativeTimeInMillis(); - versionMap.pruneTombstones(timeMSec, engineConfig.getIndexSettings().getGcDeletesInMillis()); + final long maxTimestampToPrune = timeMSec - engineConfig.getIndexSettings().getGcDeletesInMillis(); + versionMap.pruneTombstones(maxTimestampToPrune, localCheckpointTracker.getCheckpoint()); lastDeleteVersionPruneTimeMSec = timeMSec; } // testing void clearDeletedTombstones() { - // clean with current time Long.MAX_VALUE and interval 0 since we use a greater than relationship here. - versionMap.pruneTombstones(Long.MAX_VALUE, 0); + versionMap.pruneTombstones(Long.MAX_VALUE, localCheckpointTracker.getMaxSeqNo()); + } + + // for testing + final Collection getDeletedTombstones() { + return versionMap.getAllTombstones().values(); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java b/server/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java index fc62f1fb32e2b..7c5dcfa5c9050 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java +++ b/server/src/main/java/org/elasticsearch/index/engine/LiveVersionMap.java @@ -375,21 +375,25 @@ void removeTombstoneUnderLock(BytesRef uid) { } } - private boolean canRemoveTombstone(long currentTime, long pruneInterval, DeleteVersionValue versionValue) { - // check if the value is old enough to be removed - final boolean isTooOld = currentTime - versionValue.time > pruneInterval; + private boolean canRemoveTombstone(long maxTimestampToPrune, long maxSeqNoToPrune, DeleteVersionValue versionValue) { + // check if the value is old enough and safe to be removed + final boolean isTooOld = versionValue.time < maxTimestampToPrune; + final boolean isSafeToPrune = versionValue.seqNo <= maxSeqNoToPrune; // version value can't be removed it's // not yet flushed to lucene ie. it's part of this current maps object final boolean isNotTrackedByCurrentMaps = versionValue.time < maps.getMinDeleteTimestamp(); - return isTooOld && isNotTrackedByCurrentMaps; + return isTooOld && isSafeToPrune && isNotTrackedByCurrentMaps; } - void pruneTombstones(long currentTime, long pruneInterval) { + /** + * Try to prune tombstones whose timestamp is less than maxTimestampToPrune and seqno at most the maxSeqNoToPrune. + */ + void pruneTombstones(long maxTimestampToPrune, long maxSeqNoToPrune) { for (Map.Entry entry : tombstones.entrySet()) { // we do check before we actually lock the key - this way we don't need to acquire the lock for tombstones that are not // prune-able. If the tombstone changes concurrently we will re-read and step out below since if we can't collect it now w // we won't collect the tombstone below since it must be newer than this one. - if (canRemoveTombstone(currentTime, pruneInterval, entry.getValue())) { + if (canRemoveTombstone(maxTimestampToPrune, maxSeqNoToPrune, entry.getValue())) { final BytesRef uid = entry.getKey(); try (Releasable lock = keyedLock.tryAcquire(uid)) { // we use tryAcquire here since this is a best effort and we try to be least disruptive @@ -399,7 +403,7 @@ void pruneTombstones(long currentTime, long pruneInterval) { // Must re-get it here, vs using entry.getValue(), in case the uid was indexed/deleted since we pulled the iterator: final DeleteVersionValue versionValue = tombstones.get(uid); if (versionValue != null) { - if (canRemoveTombstone(currentTime, pruneInterval, versionValue)) { + if (canRemoveTombstone(maxTimestampToPrune, maxSeqNoToPrune, versionValue)) { removeTombstoneUnderLock(uid); } } diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index c48a9ee8a2d19..b0d701bdfdaa1 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -76,6 +76,7 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; @@ -163,6 +164,8 @@ import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.sameInstance; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; import static org.hamcrest.Matchers.greaterThan; @@ -173,6 +176,8 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; public class InternalEngineTests extends EngineTestCase { @@ -4464,4 +4469,65 @@ public void testStressUpdateSameDocWhileGettingIt() throws IOException, Interrup } } } + + public void testPruneOnlyDeletesAtMostLocalCheckpoint() throws Exception { + final AtomicLong clock = new AtomicLong(0); + threadPool = spy(threadPool); + when(threadPool.relativeTimeInMillis()).thenAnswer(invocation -> clock.get()); + final EngineConfig config = engine.config(); + final long gcInterval = randomIntBetween(0, 10); + final IndexSettings indexSettings = engine.config().getIndexSettings(); + final IndexMetaData indexMetaData = IndexMetaData.builder(indexSettings.getIndexMetaData()) + .settings(Settings.builder().put(indexSettings.getSettings()) + .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), TimeValue.timeValueMillis(gcInterval).getStringRep())).build(); + indexSettings.updateIndexMetaData(indexMetaData); + + try (Store store = createStore(); + InternalEngine engine = createEngine(new EngineConfig(config.getShardId(), config.getAllocationId(), threadPool, + indexSettings, config.getWarmer(), store, config.getMergePolicy(), config.getAnalyzer(), config.getSimilarity(), + new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), + config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), + config.getIndexSort(), config.getTranslogRecoveryRunner(), config.getCircuitBreakerService(), + config.getGlobalCheckpointSupplier()))) { + engine.config().setEnableGcDeletes(false); + for (int i = 0, docs = scaledRandomIntBetween(0, 10); i < docs; i++) { + index(engine, i); + } + final long deleteBatch = between(10, 20); + final long gapSeqNo = randomLongBetween( + engine.getLocalCheckpointTracker().getMaxSeqNo() + 1, engine.getLocalCheckpointTracker().getMaxSeqNo() + deleteBatch); + for (int i = 0; i < deleteBatch; i++) { + final long seqno = engine.getLocalCheckpointTracker().generateSeqNo(); + if (seqno != gapSeqNo) { + if (randomBoolean()) { + clock.incrementAndGet(); + } + engine.delete(replicaDeleteForDoc(UUIDs.randomBase64UUID(), 1, seqno, threadPool.relativeTimeInMillis())); + } + } + List tombstones = new ArrayList<>(engine.getDeletedTombstones()); + engine.config().setEnableGcDeletes(true); + // Prune tombstones whose seqno < gap_seqno and timestamp < clock-gcInterval. + clock.set(randomLongBetween(gcInterval, deleteBatch + gcInterval)); + engine.refresh("test"); + tombstones.removeIf(v -> v.seqNo < gapSeqNo && v.time < clock.get() - gcInterval); + assertThat(engine.getDeletedTombstones(), containsInAnyOrder(tombstones.toArray())); + // Prune tombstones whose seqno at most the local checkpoint (eg. seqno < gap_seqno). + clock.set(randomLongBetween(deleteBatch + gcInterval * 4/3, 100)); // Need a margin for gcInterval/4. + engine.refresh("test"); + tombstones.removeIf(v -> v.seqNo < gapSeqNo); + assertThat(engine.getDeletedTombstones(), containsInAnyOrder(tombstones.toArray())); + // Fill the seqno gap - should prune all tombstones. + clock.set(between(0, 100)); + if (randomBoolean()) { + engine.index(replicaIndexForDoc(testParsedDocument("d", null, testDocumentWithTextField(), SOURCE, null), 1, gapSeqNo, false)); + } else { + engine.delete(replicaDeleteForDoc(UUIDs.randomBase64UUID(), Versions.MATCH_ANY, gapSeqNo, threadPool.relativeTimeInMillis())); + } + clock.set(randomLongBetween(100 + gcInterval * 4/3, Long.MAX_VALUE)); // Need a margin for gcInterval/4. + engine.refresh("test"); + assertThat(engine.getDeletedTombstones(), empty()); + } + } + } diff --git a/server/src/test/java/org/elasticsearch/index/engine/LiveVersionMapTests.java b/server/src/test/java/org/elasticsearch/index/engine/LiveVersionMapTests.java index 8c5973e8750fd..ce3ddff00dade 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/LiveVersionMapTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/LiveVersionMapTests.java @@ -37,7 +37,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.StreamSupport; + +import static org.hamcrest.Matchers.empty; public class LiveVersionMapTests extends ESTestCase { @@ -106,7 +107,6 @@ public void testBasics() throws IOException { map.afterRefresh(randomBoolean()); assertNull(map.getUnderLock(uid("test"))); - map.putUnderLock(uid("test"), new DeleteVersionValue(1,1,1,1)); assertEquals(new DeleteVersionValue(1,1,1,1), map.getUnderLock(uid("test"))); map.beforeRefresh(); @@ -114,6 +114,8 @@ public void testBasics() throws IOException { map.afterRefresh(randomBoolean()); assertEquals(new DeleteVersionValue(1,1,1,1), map.getUnderLock(uid("test"))); map.pruneTombstones(2, 0); + assertEquals(new DeleteVersionValue(1,1,1,1), map.getUnderLock(uid("test"))); + map.pruneTombstones(2, 1); assertNull(map.getUnderLock(uid("test"))); } } @@ -134,8 +136,10 @@ public void testConcurrently() throws IOException, InterruptedException { CountDownLatch startGun = new CountDownLatch(numThreads); CountDownLatch done = new CountDownLatch(numThreads); int randomValuesPerThread = randomIntBetween(5000, 20000); - AtomicLong clock = new AtomicLong(0); - AtomicLong lastPrunedTimestamp = new AtomicLong(-1); + final AtomicLong clock = new AtomicLong(0); + final AtomicLong lastPrunedTimestamp = new AtomicLong(-1); + final AtomicLong maxSeqNo = new AtomicLong(); + final AtomicLong lastPrunedSeqNo = new AtomicLong(); for (int j = 0; j < threads.length; j++) { threads[j] = new Thread(() -> { startGun.countDown(); @@ -148,29 +152,31 @@ public void testConcurrently() throws IOException, InterruptedException { try { for (int i = 0; i < randomValuesPerThread; ++i) { BytesRef bytesRef = randomFrom(random(), keyList); - final long clockTick = clock.get(); try (Releasable r = map.acquireLock(bytesRef)) { VersionValue versionValue = values.computeIfAbsent(bytesRef, - v -> new VersionValue(randomLong(), randomLong(), randomLong())); + v -> new VersionValue(randomLong(), maxSeqNo.incrementAndGet(), randomLong())); boolean isDelete = versionValue instanceof DeleteVersionValue; if (isDelete) { map.removeTombstoneUnderLock(bytesRef); deletes.remove(bytesRef); } if (isDelete == false && rarely()) { - versionValue = new DeleteVersionValue(versionValue.version + 1, versionValue.seqNo + 1, + versionValue = new DeleteVersionValue(versionValue.version + 1, maxSeqNo.incrementAndGet(), versionValue.term, clock.getAndIncrement()); deletes.put(bytesRef, (DeleteVersionValue) versionValue); } else { - versionValue = new VersionValue(versionValue.version + 1, versionValue.seqNo + 1, versionValue.term); + versionValue = new VersionValue(versionValue.version + 1, maxSeqNo.incrementAndGet(), versionValue.term); } values.put(bytesRef, versionValue); map.putUnderLock(bytesRef, versionValue); } if (rarely()) { - map.pruneTombstones(clockTick, 0); - // timestamp we pruned the deletes - lastPrunedTimestamp.updateAndGet(prev -> Math.max(clockTick, prev)); // make sure we track the latest + final long pruneSeqNo = randomLongBetween(0, maxSeqNo.get()); + final long clockTick = randomLongBetween(0, clock.get()); + map.pruneTombstones(clockTick, pruneSeqNo); + // make sure we track the latest timestamp and seqno we pruned the deletes + lastPrunedTimestamp.updateAndGet(prev -> Math.max(clockTick, prev)); + lastPrunedSeqNo.updateAndGet(prev -> Math.max(pruneSeqNo, prev)); } } } finally { @@ -234,15 +240,17 @@ public void testConcurrently() throws IOException, InterruptedException { VersionValue value = map.getUnderLock(e.getKey()); // here we keep track of the deletes and ensure that all deletes that are not visible anymore ie. not in the map // have a timestamp that is smaller or equal to the maximum timestamp that we pruned on + final DeleteVersionValue delete = e.getValue(); if (value == null) { - assertTrue(e.getValue().time + " > " + lastPrunedTimestamp.get(), e.getValue().time <= lastPrunedTimestamp.get()); + assertTrue(delete.time + " > " + lastPrunedTimestamp.get() + "," + delete.seqNo + " > " + lastPrunedSeqNo.get(), + delete.time <= lastPrunedTimestamp.get() && delete.seqNo <= lastPrunedSeqNo.get()); } else { - assertEquals(value, e.getValue()); + assertEquals(value, delete); } } }); - map.pruneTombstones(clock.incrementAndGet(), 0); - assertEquals(0, StreamSupport.stream(map.getAllTombstones().entrySet().spliterator(), false).count()); + map.pruneTombstones(clock.incrementAndGet(), maxSeqNo.get()); + assertThat(map.getAllTombstones().entrySet(), empty()); } public void testCarryOnSafeAccess() throws IOException { diff --git a/server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index ad046dddc0c27..ba5b43b1d9204 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/server/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -22,6 +22,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.bulk.BulkItemRequest; @@ -30,11 +31,13 @@ import org.elasticsearch.action.bulk.BulkShardResponse; import org.elasticsearch.action.bulk.TransportShardBulkAction; import org.elasticsearch.action.bulk.TransportShardBulkActionTests; +import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.resync.ResyncReplicationRequest; import org.elasticsearch.action.resync.ResyncReplicationResponse; import org.elasticsearch.action.resync.TransportResyncReplicationAction; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.replication.ReplicatedWriteRequest; import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; @@ -617,6 +620,13 @@ private TransportWriteAction.WritePrimaryResult + BulkShardRequest executeReplicationRequestOnPrimary(IndexShard primary, Request request) throws Exception { + final BulkShardRequest bulkShardRequest = new BulkShardRequest(shardId, request.getRefreshPolicy(), + new BulkItemRequest[]{new BulkItemRequest(0, request)}); + return executeShardBulkOnPrimary(primary, bulkShardRequest).replicaRequest(); + } + private void executeShardBulkOnReplica(BulkShardRequest request, IndexShard replica, long operationPrimaryTerm, long globalCheckpointOnPrimary) throws Exception { final PlainActionFuture permitAcquiredFuture = new PlainActionFuture<>(); replica.acquireReplicaOperationPermit(operationPrimaryTerm, globalCheckpointOnPrimary, permitAcquiredFuture, ThreadPool.Names.SAME, request); @@ -631,13 +641,14 @@ private void executeShardBulkOnReplica(BulkShardRequest request, IndexShard repl * indexes the given requests on the supplied primary, modifying it for replicas */ BulkShardRequest indexOnPrimary(IndexRequest request, IndexShard primary) throws Exception { - final BulkItemRequest bulkItemRequest = new BulkItemRequest(0, request); - BulkItemRequest[] bulkItemRequests = new BulkItemRequest[1]; - bulkItemRequests[0] = bulkItemRequest; - final BulkShardRequest bulkShardRequest = new BulkShardRequest(shardId, request.getRefreshPolicy(), bulkItemRequests); - final TransportWriteAction.WritePrimaryResult result = - executeShardBulkOnPrimary(primary, bulkShardRequest); - return result.replicaRequest(); + return executeReplicationRequestOnPrimary(primary, request); + } + + /** + * Executes the delete request on the primary, and modifies it for replicas. + */ + BulkShardRequest deleteOnPrimary(DeleteRequest request, IndexShard primary) throws Exception { + return executeReplicationRequestOnPrimary(primary, request); } /** @@ -647,6 +658,13 @@ void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard executeShardBulkOnReplica(request, replica, group.primary.getPrimaryTerm(), group.primary.getGlobalCheckpoint()); } + /** + * Executes the delete request on the given replica shard. + */ + void deleteOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica) throws Exception { + executeShardBulkOnReplica(request, replica, group.primary.getPrimaryTerm(), group.primary.getGlobalCheckpoint()); + } + class GlobalCheckpointSync extends ReplicationAction< GlobalCheckpointSyncAction.Request, GlobalCheckpointSyncAction.Request, diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index 86436d8d88ac9..4762c23319a4a 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -26,9 +26,14 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkShardRequest; +import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineFactory; @@ -43,6 +48,8 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.recovery.RecoveryTarget; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.Matcher; import java.io.IOException; @@ -52,13 +59,13 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import static org.elasticsearch.index.translog.SnapshotMatchers.containsOperationsInAnyOrder; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; @@ -368,6 +375,50 @@ public void testSeqNoCollision() throws Exception { } } + /** + * This test ensures the consistency between primary and replica with late and out of order delivery on the replica. + * An index operation on the primary is followed by a delete operation. The delete operation is delivered first + * and processed on the replica but the index is delayed with an interval that is even longer the gc deletes cycle. + * This makes sure that that replica still remembers the delete operation and correctly ignores the stale index operation. + */ + public void testLateDeliveryAfterGCTriggeredOnReplica() throws Exception { + ThreadPool.terminate(this.threadPool, 10, TimeUnit.SECONDS); + this.threadPool = new TestThreadPool(getClass().getName(), + Settings.builder().put(threadPoolSettings()).put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0).build()); + + try (ReplicationGroup shards = createGroup(1)) { + shards.startAll(); + final IndexShard primary = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + final TimeValue gcInterval = TimeValue.timeValueMillis(between(1, 10)); + // I think we can just set this to something very small (10ms?) and also set ThreadPool#ESTIMATED_TIME_INTERVAL_SETTING to 0? + + updateGCDeleteCycle(replica, gcInterval); + final BulkShardRequest indexRequest = indexOnPrimary( + new IndexRequest(index.getName(), "type", "d1").source("{}", XContentType.JSON), primary); + final BulkShardRequest deleteRequest = deleteOnPrimary(new DeleteRequest(index.getName(), "type", "d1"), primary); + deleteOnReplica(deleteRequest, shards, replica); // delete arrives on replica first. + final long deleteTimestamp = threadPool.relativeTimeInMillis(); + replica.refresh("test"); + assertBusy(() -> + assertThat(threadPool.relativeTimeInMillis() - deleteTimestamp, greaterThan(gcInterval.millis())) + ); + getEngine(replica).maybePruneDeletes(); + indexOnReplica(indexRequest, shards, replica); // index arrives on replica lately. + shards.assertAllEqual(0); + } + } + + private void updateGCDeleteCycle(IndexShard shard, TimeValue interval) { + IndexMetaData.Builder builder = IndexMetaData.builder(shard.indexSettings().getIndexMetaData()); + builder.settings(Settings.builder() + .put(shard.indexSettings().getSettings()) + .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), interval.getStringRep()) + ); + shard.indexSettings().updateIndexMetaData(builder.build()); + shard.onSettingsChanged(); + } + /** Throws documentFailure on every indexing operation */ static class ThrowingDocumentFailureEngineFactory implements EngineFactory { final String documentFailureMessage; diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index f0e46cf022344..8a9ad3d2a76e1 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -485,4 +485,8 @@ protected Engine.Index replicaIndexForDoc(ParsedDocument doc, long version, long IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, isRetry); } + protected Engine.Delete replicaDeleteForDoc(String id, long version, long seqNo, long startTime) { + return new Engine.Delete("test", id, newUid(id), seqNo, 1, version, VersionType.EXTERNAL, + Engine.Operation.Origin.REPLICA, startTime); + } } From 7bf909194222f851a5e10ba8012e85624c2c124f Mon Sep 17 00:00:00 2001 From: Andy Bristol Date: Mon, 26 Mar 2018 13:43:09 -0700 Subject: [PATCH 08/22] [test] packaging: gradle tasks for groovy tests (#29046) The vagrant test plugin adds tasks for the groovy packaging tests, which run after the bats packaging test tasks.Rename the 'bats' configuration to 'packaging' and remove the option to inherit archives from this configuration. --- TESTING.asciidoc | 6 +- Vagrantfile | 8 +- .../vagrant/VagrantPropertiesExtension.groovy | 7 - .../gradle/vagrant/VagrantTestPlugin.groovy | 166 +++++++++--------- qa/vagrant/build.gradle | 4 +- 5 files changed, 95 insertions(+), 96 deletions(-) diff --git a/TESTING.asciidoc b/TESTING.asciidoc index f7725bd17d78d..4a432d48181ee 100644 --- a/TESTING.asciidoc +++ b/TESTING.asciidoc @@ -414,16 +414,16 @@ and in another window: ---------------------------------------------------- vagrant up centos-7 --provider virtualbox && vagrant ssh centos-7 -cd $BATS_ARCHIVES +cd $PACKAGING_ARCHIVES sudo -E bats $BATS_TESTS/*rpm*.bats ---------------------------------------------------- If you wanted to retest all the release artifacts on a single VM you could: ------------------------------------------------- -./gradlew setupBats +./gradlew setupPackagingTest cd qa/vagrant; vagrant up ubuntu-1404 --provider virtualbox && vagrant ssh ubuntu-1404 -cd $BATS_ARCHIVES +cd $PACKAGING_ARCHIVES sudo -E bats $BATS_TESTS/*.bats ------------------------------------------------- diff --git a/Vagrantfile b/Vagrantfile index 683a5d83732bd..6f81ba0273c9f 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -334,9 +334,9 @@ export TAR=/elasticsearch/distribution/tar/build/distributions export RPM=/elasticsearch/distribution/rpm/build/distributions export DEB=/elasticsearch/distribution/deb/build/distributions export BATS=/project/build/bats -export BATS_UTILS=/project/build/bats/utils -export BATS_TESTS=/project/build/bats/tests -export BATS_ARCHIVES=/project/build/bats/archives +export BATS_UTILS=/project/build/packaging/bats/utils +export BATS_TESTS=/project/build/packaging/bats/tests +export PACKAGING_ARCHIVES=/project/build/packaging/archives VARS cat \<\ /etc/sudoers.d/elasticsearch_vars Defaults env_keep += "ZIP" @@ -346,7 +346,7 @@ Defaults env_keep += "DEB" Defaults env_keep += "BATS" Defaults env_keep += "BATS_UTILS" Defaults env_keep += "BATS_TESTS" -Defaults env_keep += "BATS_ARCHIVES" +Defaults env_keep += "PACKAGING_ARCHIVES" SUDOERS_VARS chmod 0440 /etc/sudoers.d/elasticsearch_vars SHELL diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantPropertiesExtension.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantPropertiesExtension.groovy index e6e7fca62f97e..c6d0f1d0425d0 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantPropertiesExtension.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantPropertiesExtension.groovy @@ -37,9 +37,6 @@ class VagrantPropertiesExtension { @Input Boolean inheritTests - @Input - Boolean inheritTestArchives - @Input Boolean inheritTestUtils @@ -60,10 +57,6 @@ class VagrantPropertiesExtension { this.inheritTests = inheritTests } - void setInheritTestArchives(Boolean inheritTestArchives) { - this.inheritTestArchives = inheritTestArchives - } - void setInheritTestUtils(Boolean inheritTestUtils) { this.inheritTestUtils = inheritTestUtils } diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy index fc15258215e65..d7d1c01e7dd00 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy @@ -1,6 +1,5 @@ package org.elasticsearch.gradle.vagrant -import com.carrotsearch.gradle.junit4.RandomizedTestingPlugin import org.apache.tools.ant.taskdefs.condition.Os import org.elasticsearch.gradle.FileContentsTask import org.elasticsearch.gradle.LoggedExec @@ -43,8 +42,9 @@ class VagrantTestPlugin implements Plugin { /** Packages onboarded for upgrade tests **/ static List UPGRADE_FROM_ARCHIVES = ['rpm', 'deb'] + private static final PACKAGING_CONFIGURATION = 'packaging' private static final BATS = 'bats' - private static final String BATS_TEST_COMMAND ="cd \$BATS_ARCHIVES && sudo bats --tap \$BATS_TESTS/*.$BATS" + private static final String BATS_TEST_COMMAND ="cd \$PACKAGING_ARCHIVES && sudo bats --tap \$BATS_TESTS/*.$BATS" private static final String PLATFORM_TEST_COMMAND ="rm -rf ~/elasticsearch && rsync -r /elasticsearch/ ~/elasticsearch && cd ~/elasticsearch && ./gradlew test integTest" @Override @@ -53,11 +53,11 @@ class VagrantTestPlugin implements Plugin { // Creates the Vagrant extension for the project project.extensions.create('esvagrant', VagrantPropertiesExtension, listVagrantBoxes(project)) - // Add required repositories for Bats tests - configureBatsRepositories(project) + // Add required repositories for packaging tests + configurePackagingArchiveRepositories(project) // Creates custom configurations for Bats testing files (and associated scripts and archives) - createBatsConfiguration(project) + createPackagingConfiguration(project) // Creates all the main Vagrant tasks createVagrantTasks(project) @@ -87,7 +87,7 @@ class VagrantTestPlugin implements Plugin { } } - private static void configureBatsRepositories(Project project) { + private static void configurePackagingArchiveRepositories(Project project) { RepositoryHandler repos = project.repositories // Try maven central first, it'll have releases before 5.0.0 @@ -102,10 +102,10 @@ class VagrantTestPlugin implements Plugin { } } - private static void createBatsConfiguration(Project project) { - project.configurations.create(BATS) + private static void createPackagingConfiguration(Project project) { + project.configurations.create(PACKAGING_CONFIGURATION) - String upgradeFromVersion = System.getProperty("tests.packaging.upgradeVersion"); + String upgradeFromVersion = System.getProperty("tests.packaging.upgradeVersion") if (upgradeFromVersion == null) { String firstPartOfSeed = project.rootProject.testSeed.tokenize(':').get(0) final long seed = Long.parseUnsignedLong(firstPartOfSeed, 16) @@ -120,12 +120,14 @@ class VagrantTestPlugin implements Plugin { } else { it = "packages:${it}" } - project.dependencies.add(BATS, project.dependencies.project(path: ":distribution:${it}", configuration: 'default')) + project.dependencies.add(PACKAGING_CONFIGURATION, + project.dependencies.project(path: ":distribution:${it}", configuration: 'default')) } UPGRADE_FROM_ARCHIVES.each { // The version of elasticsearch that we upgrade *from* - project.dependencies.add(BATS, "org.elasticsearch.distribution.${it}:elasticsearch:${upgradeFromVersion}@${it}") + project.dependencies.add(PACKAGING_CONFIGURATION, + "org.elasticsearch.distribution.${it}:elasticsearch:${upgradeFromVersion}@${it}") } project.extensions.esvagrant.upgradeFromVersion = upgradeFromVersion @@ -154,22 +156,28 @@ class VagrantTestPlugin implements Plugin { } private static void createPrepareVagrantTestEnvTask(Project project) { - File batsDir = new File("${project.buildDir}/${BATS}") + File packagingDir = new File(project.buildDir, PACKAGING_CONFIGURATION) - Task createBatsDirsTask = project.tasks.create('createBatsDirs') - createBatsDirsTask.outputs.dir batsDir - createBatsDirsTask.doLast { - batsDir.mkdirs() + File archivesDir = new File(packagingDir, 'archives') + Copy copyPackagingArchives = project.tasks.create('copyPackagingArchives', Copy) { + into archivesDir + from project.configurations[PACKAGING_CONFIGURATION] } - Copy copyBatsArchives = project.tasks.create('copyBatsArchives', Copy) { - dependsOn createBatsDirsTask - into "${batsDir}/archives" - from project.configurations[BATS] + Task createVersionFile = project.tasks.create('createVersionFile', FileContentsTask) { + dependsOn copyPackagingArchives + file "${archivesDir}/version" + contents project.version + } + + Task createUpgradeFromFile = project.tasks.create('createUpgradeFromFile', FileContentsTask) { + dependsOn copyPackagingArchives + file "${archivesDir}/upgrade_from_version" + contents project.extensions.esvagrant.upgradeFromVersion } + File batsDir = new File(packagingDir, BATS) Copy copyBatsTests = project.tasks.create('copyBatsTests', Copy) { - dependsOn createBatsDirsTask into "${batsDir}/tests" from { "${project.extensions.esvagrant.batsDir}/tests" @@ -177,7 +185,6 @@ class VagrantTestPlugin implements Plugin { } Copy copyBatsUtils = project.tasks.create('copyBatsUtils', Copy) { - dependsOn createBatsDirsTask into "${batsDir}/utils" from { "${project.extensions.esvagrant.batsDir}/utils" @@ -185,42 +192,30 @@ class VagrantTestPlugin implements Plugin { } // Now we iterate over dependencies of the bats configuration. When a project dependency is found, - // we bring back its own archives, test files or test utils. + // we bring back its test files or test utils. project.afterEvaluate { - project.configurations.bats.dependencies.findAll {it.targetConfiguration == BATS }.each { d -> - if (d instanceof DefaultProjectDependency) { - DefaultProjectDependency externalBatsDependency = (DefaultProjectDependency) d - Project externalBatsProject = externalBatsDependency.dependencyProject - String externalBatsDir = externalBatsProject.extensions.esvagrant.batsDir - - if (project.extensions.esvagrant.inheritTests) { - copyBatsTests.from(externalBatsProject.files("${externalBatsDir}/tests")) - } - if (project.extensions.esvagrant.inheritTestArchives) { - copyBatsArchives.from(externalBatsDependency.projectConfiguration.files) - } - if (project.extensions.esvagrant.inheritTestUtils) { - copyBatsUtils.from(externalBatsProject.files("${externalBatsDir}/utils")) + project.configurations[PACKAGING_CONFIGURATION].dependencies + .findAll {it.targetConfiguration == PACKAGING_CONFIGURATION } + .each { d -> + if (d instanceof DefaultProjectDependency) { + DefaultProjectDependency externalBatsDependency = (DefaultProjectDependency) d + Project externalBatsProject = externalBatsDependency.dependencyProject + String externalBatsDir = externalBatsProject.extensions.esvagrant.batsDir + + if (project.extensions.esvagrant.inheritTests) { + copyBatsTests.from(externalBatsProject.files("${externalBatsDir}/tests")) + } + if (project.extensions.esvagrant.inheritTestUtils) { + copyBatsUtils.from(externalBatsProject.files("${externalBatsDir}/utils")) + } } - } } } - Task createVersionFile = project.tasks.create('createVersionFile', FileContentsTask) { - dependsOn createBatsDirsTask - file "${batsDir}/archives/version" - contents project.version - } - - Task createUpgradeFromFile = project.tasks.create('createUpgradeFromFile', FileContentsTask) { - dependsOn createBatsDirsTask - file "${batsDir}/archives/upgrade_from_version" - contents project.extensions.esvagrant.upgradeFromVersion - } - - Task vagrantSetUpTask = project.tasks.create('setupBats') + Task vagrantSetUpTask = project.tasks.create('setupPackagingTest') vagrantSetUpTask.dependsOn 'vagrantCheckVersion' - vagrantSetUpTask.dependsOn copyBatsTests, copyBatsUtils, copyBatsArchives, createVersionFile, createUpgradeFromFile + vagrantSetUpTask.dependsOn copyPackagingArchives, createVersionFile, createUpgradeFromFile + vagrantSetUpTask.dependsOn copyBatsTests, copyBatsUtils } private static void createPackagingTestTask(Project project) { @@ -270,8 +265,8 @@ class VagrantTestPlugin implements Plugin { assert project.tasks.virtualboxCheckVersion != null Task virtualboxCheckVersion = project.tasks.virtualboxCheckVersion - assert project.tasks.setupBats != null - Task setupBats = project.tasks.setupBats + assert project.tasks.setupPackagingTest != null + Task setupPackagingTest = project.tasks.setupPackagingTest assert project.tasks.packagingTest != null Task packagingTest = project.tasks.packagingTest @@ -308,7 +303,7 @@ class VagrantTestPlugin implements Plugin { environmentVars vagrantEnvVars dependsOn vagrantCheckVersion, virtualboxCheckVersion } - update.mustRunAfter(setupBats) + update.mustRunAfter(setupPackagingTest) /* * Destroying before every execution can be annoying while iterating on tests locally. Therefore, we provide a flag @@ -359,32 +354,39 @@ class VagrantTestPlugin implements Plugin { } vagrantSmokeTest.dependsOn(smoke) - Task packaging = project.tasks.create("vagrant${boxTask}#packagingTest", BatsOverVagrantTask) { + Task batsPackagingTest = project.tasks.create("vagrant${boxTask}#batsPackagingTest", BatsOverVagrantTask) { remoteCommand BATS_TEST_COMMAND boxName box environmentVars vagrantEnvVars - dependsOn up, setupBats + dependsOn up, setupPackagingTest finalizedBy halt } - TaskExecutionAdapter packagingReproListener = new TaskExecutionAdapter() { - @Override - void afterExecute(Task task, TaskState state) { - final String gradlew = Os.isFamily(Os.FAMILY_WINDOWS) ? "gradlew" : "./gradlew" - if (state.failure != null) { - println "REPRODUCE WITH: ${gradlew} ${packaging.path} " + - "-Dtests.seed=${project.testSeed} " - } - } + TaskExecutionAdapter batsPackagingReproListener = createReproListener(project, batsPackagingTest.path) + batsPackagingTest.doFirst { + project.gradle.addListener(batsPackagingReproListener) + } + batsPackagingTest.doLast { + project.gradle.removeListener(batsPackagingReproListener) } - packaging.doFirst { - project.gradle.addListener(packagingReproListener) + if (project.extensions.esvagrant.boxes.contains(box)) { + packagingTest.dependsOn(batsPackagingTest) + } + + // This task doesn't do anything yet. In the future it will execute a jar containing tests on the vm + Task groovyPackagingTest = project.tasks.create("vagrant${boxTask}#groovyPackagingTest") + groovyPackagingTest.dependsOn(up) + groovyPackagingTest.finalizedBy(halt) + + TaskExecutionAdapter groovyPackagingReproListener = createReproListener(project, groovyPackagingTest.path) + groovyPackagingTest.doFirst { + project.gradle.addListener(groovyPackagingReproListener) } - packaging.doLast { - project.gradle.removeListener(packagingReproListener) + groovyPackagingTest.doLast { + project.gradle.removeListener(groovyPackagingReproListener) } if (project.extensions.esvagrant.boxes.contains(box)) { - packagingTest.dependsOn(packaging) + packagingTest.dependsOn(groovyPackagingTest) } Task platform = project.tasks.create("vagrant${boxTask}#platformTest", VagrantCommandTask) { @@ -395,15 +397,7 @@ class VagrantTestPlugin implements Plugin { finalizedBy halt args '--command', PLATFORM_TEST_COMMAND + " -Dtests.seed=${-> project.testSeed}" } - TaskExecutionAdapter platformReproListener = new TaskExecutionAdapter() { - @Override - void afterExecute(Task task, TaskState state) { - if (state.failure != null) { - println "REPRODUCE WITH: gradle ${platform.path} " + - "-Dtests.seed=${project.testSeed} " - } - } - } + TaskExecutionAdapter platformReproListener = createReproListener(project, platform.path) platform.doFirst { project.gradle.addListener(platformReproListener) } @@ -415,4 +409,16 @@ class VagrantTestPlugin implements Plugin { } } } + + private static TaskExecutionAdapter createReproListener(Project project, String reproTaskPath) { + return new TaskExecutionAdapter() { + @Override + void afterExecute(Task task, TaskState state) { + final String gradlew = Os.isFamily(Os.FAMILY_WINDOWS) ? "gradlew" : "./gradlew" + if (state.failure != null) { + println "REPRODUCE WITH: ${gradlew} ${reproTaskPath} -Dtests.seed=${project.testSeed} " + } + } + } + } } diff --git a/qa/vagrant/build.gradle b/qa/vagrant/build.gradle index f28f6afd2fc33..4086cf2205785 100644 --- a/qa/vagrant/build.gradle +++ b/qa/vagrant/build.gradle @@ -25,14 +25,14 @@ for (Project subproj : project.rootProject.subprojects) { if (subproj.path.startsWith(':plugins:') || subproj.path.equals(':example-plugins:custom-settings')) { // add plugin as a dep dependencies { - bats project(path: "${subproj.path}", configuration: 'zip') + packaging project(path: "${subproj.path}", configuration: 'zip') } plugins.add(subproj.name) } } plugins = plugins.toSorted() -setupBats { +setupPackagingTest { doFirst { File expectedPlugins = file('build/plugins/expected') expectedPlugins.parentFile.mkdirs() From 0ac89a32cc71742256a0a3471aa19656bcaac901 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 26 Mar 2018 16:56:12 -0400 Subject: [PATCH 09/22] Do not optimize append-only if seen normal op with higher seqno (#28787) When processing an append-only operation, primary knows that operations can only conflict with another instance of the same operation. This is true as the id was freshly generated. However this property doesn't hold for replicas. As soon as an auto-generated ID was indexed into the primary, it can be exposed to a search and users can issue a follow up operation on it. In extremely rare cases, the follow up operation can be arrived and processed on a replica before the original append-only request. In this case we can't simply proceed with the append-only request and blindly add it to the index without consulting the version map. The following scenario can cause difference between primary and replica. 1. Primary indexes an auto-gen-id doc. (id=X, v=1, s#=20) 2. A refresh cycle happens on primary 3. The new doc is picked up and modified - say by a delete by query request - Primary gets a delete doc (id=X, v=2, s#=30) 4. Delete doc is processed first on the replica (id=X, v=2, s#=30) 5. Indexing operation arrives on the replica, since it's an auto-gen-id request and the retry marker is lower, we put it into lucene without any check. Replica has a doc the primary doesn't have. To deal with a potential conflict between an append-only operation and a normal operation on replicas, we need to rely on sequence numbers. This commit maintains the max seqno of non-append-only operations on replica then only apply optimization for an append-only operation only if its seq# is higher than the seq# of all non-append-only. --- .../index/engine/InternalEngine.java | 45 +++++++-- .../index/engine/InternalEngineTests.java | 92 +++++++++++++++++++ .../IndexLevelReplicationTests.java | 22 +++++ 3 files changed, 150 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 70cbebb774e02..0fda2f04ac5a4 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -136,6 +136,7 @@ public class InternalEngine extends Engine { private final AtomicBoolean pendingTranslogRecovery = new AtomicBoolean(false); public static final String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID = "max_unsafe_auto_id_timestamp"; private final AtomicLong maxUnsafeAutoIdTimestamp = new AtomicLong(-1); + private final AtomicLong maxSeqNoOfNonAppendOnlyOperations = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); private final CounterMetric numVersionLookups = new CounterMetric(); private final CounterMetric numIndexVersionsLookups = new CounterMetric(); /** @@ -186,7 +187,7 @@ public InternalEngine(EngineConfig engineConfig) { this.combinedDeletionPolicy = new CombinedDeletionPolicy(logger, translogDeletionPolicy, translog::getLastSyncedGlobalCheckpoint, startingCommit); writer = createWriter(startingCommit); - updateMaxUnsafeAutoIdTimestampFromWriter(writer); + bootstrapAppendOnlyInfoFromWriter(writer); historyUUID = loadOrGenerateHistoryUUID(writer); Objects.requireNonNull(historyUUID, "history uuid should not be null"); indexWriter = writer; @@ -345,15 +346,20 @@ public int fillSeqNoGaps(long primaryTerm) throws IOException { } } - private void updateMaxUnsafeAutoIdTimestampFromWriter(IndexWriter writer) { - long commitMaxUnsafeAutoIdTimestamp = Long.MIN_VALUE; + private void bootstrapAppendOnlyInfoFromWriter(IndexWriter writer) { for (Map.Entry entry : writer.getLiveCommitData()) { - if (entry.getKey().equals(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID)) { - commitMaxUnsafeAutoIdTimestamp = Long.parseLong(entry.getValue()); - break; + final String key = entry.getKey(); + if (key.equals(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID)) { + assert maxUnsafeAutoIdTimestamp.get() == -1 : + "max unsafe timestamp was assigned already [" + maxUnsafeAutoIdTimestamp.get() + "]"; + maxUnsafeAutoIdTimestamp.set(Long.parseLong(entry.getValue())); + } + if (key.equals(SequenceNumbers.MAX_SEQ_NO)) { + assert maxSeqNoOfNonAppendOnlyOperations.get() == -1 : + "max unsafe append-only seq# was assigned already [" + maxSeqNoOfNonAppendOnlyOperations.get() + "]"; + maxSeqNoOfNonAppendOnlyOperations.set(Long.parseLong(entry.getValue())); } } - maxUnsafeAutoIdTimestamp.set(Math.max(maxUnsafeAutoIdTimestamp.get(), commitMaxUnsafeAutoIdTimestamp)); } @Override @@ -803,11 +809,24 @@ public IndexResult index(Index index) throws IOException { private IndexingStrategy planIndexingAsNonPrimary(Index index) throws IOException { final IndexingStrategy plan; - if (canOptimizeAddDocument(index) && mayHaveBeenIndexedBefore(index) == false) { - // no need to deal with out of order delivery - we never saw this one + final boolean appendOnlyRequest = canOptimizeAddDocument(index); + if (appendOnlyRequest && mayHaveBeenIndexedBefore(index) == false && index.seqNo() > maxSeqNoOfNonAppendOnlyOperations.get()) { + /* + * As soon as an append-only request was indexed into the primary, it can be exposed to a search then users can issue + * a follow-up operation on it. In rare cases, the follow up operation can be arrived and processed on a replica before + * the original append-only. In this case we can't simply proceed with the append only without consulting the version map. + * If a replica has seen a non-append-only operation with a higher seqno than the seqno of an append-only, it may have seen + * the document of that append-only request. However if the seqno of an append-only is higher than seqno of any non-append-only + * requests, we can assert the replica have not seen the document of that append-only request, thus we can apply optimization. + */ assert index.version() == 1L : "can optimize on replicas but incoming version is [" + index.version() + "]"; plan = IndexingStrategy.optimizedAppendOnly(index.seqNo()); } else { + if (appendOnlyRequest == false) { + maxSeqNoOfNonAppendOnlyOperations.updateAndGet(curr -> Math.max(index.seqNo(), curr)); + assert maxSeqNoOfNonAppendOnlyOperations.get() >= index.seqNo() : "max_seqno of non-append-only was not updated;" + + "max_seqno non-append-only [" + maxSeqNoOfNonAppendOnlyOperations.get() + "], seqno of index [" + index.seqNo() + "]"; + } versionMap.enforceSafeAccess(); // drop out of order operations assert index.versionType().versionTypeForReplicationAndRecovery() == index.versionType() : @@ -942,6 +961,11 @@ private boolean mayHaveBeenIndexedBefore(Index index) { return mayHaveBeenIndexBefore; } + // for testing + long getMaxSeqNoOfNonAppendOnlyOperations() { + return maxSeqNoOfNonAppendOnlyOperations.get(); + } + private static void index(final List docs, final IndexWriter indexWriter) throws IOException { if (docs.size() > 1) { indexWriter.addDocuments(docs); @@ -1097,6 +1121,9 @@ private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOExcept assert delete.versionType().versionTypeForReplicationAndRecovery() == delete.versionType() : "resolving out of order delivery based on versioning but version type isn't fit for it. got [" + delete.versionType() + "]"; + maxSeqNoOfNonAppendOnlyOperations.updateAndGet(curr -> Math.max(delete.seqNo(), curr)); + assert maxSeqNoOfNonAppendOnlyOperations.get() >= delete.seqNo() : "max_seqno of non-append-only was not updated;" + + "max_seqno non-append-only [" + maxSeqNoOfNonAppendOnlyOperations.get() + "], seqno of delete [" + delete.seqNo() + "]"; // unlike the primary, replicas don't really care to about found status of documents // this allows to ignore the case where a document was found in the live version maps in // a delete state and return true for the found flag in favor of code simplicity diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index b0d701bdfdaa1..e022330b664c3 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -4530,4 +4530,96 @@ public void testPruneOnlyDeletesAtMostLocalCheckpoint() throws Exception { } } + public void testTrackMaxSeqNoOfNonAppendOnlyOperations() throws Exception { + IOUtils.close(engine, store); + store = createStore(); + final Path translogPath = createTempDir(); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + try (InternalEngine engine = createEngine(store, translogPath, globalCheckpoint::get)) { + final CountDownLatch latch = new CountDownLatch(1); + final Thread appendOnlyIndexer = new Thread(() -> { + try { + latch.countDown(); + final int numDocs = scaledRandomIntBetween(100, 1000); + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument("append-only" + i, null, testDocumentWithTextField(), SOURCE, null); + if (randomBoolean()) { + engine.index(appendOnlyReplica(doc, randomBoolean(), 1, engine.getLocalCheckpointTracker().generateSeqNo())); + } else { + engine.index(appendOnlyPrimary(doc, randomBoolean(), randomNonNegativeLong())); + } + } + } catch (Exception ex) { + throw new RuntimeException("Failed to index", ex); + } + }); + appendOnlyIndexer.setName("append-only indexer"); + appendOnlyIndexer.start(); + latch.await(); + long maxSeqNoOfNonAppendOnly = SequenceNumbers.NO_OPS_PERFORMED; + final int numOps = scaledRandomIntBetween(100, 1000); + for (int i = 0; i < numOps; i++) { + ParsedDocument parsedDocument = testParsedDocument(Integer.toString(i), null, testDocumentWithTextField(), SOURCE, null); + if (randomBoolean()) { // On replica - update max_seqno for non-append-only operations + final long seqno = engine.getLocalCheckpointTracker().generateSeqNo(); + final Engine.Index doc = replicaIndexForDoc(parsedDocument, 1, seqno, randomBoolean()); + if (randomBoolean()) { + engine.index(doc); + } else { + engine.delete(new Engine.Delete(doc.type(), doc.id(), doc.uid(), seqno, doc.primaryTerm(), + doc.version(), doc.versionType(), doc.origin(), threadPool.relativeTimeInMillis())); + } + maxSeqNoOfNonAppendOnly = seqno; + } else { // On primary - do not update max_seqno for non-append-only operations + if (randomBoolean()) { + engine.index(indexForDoc(parsedDocument)); + } else { + engine.delete(new Engine.Delete(parsedDocument.type(), parsedDocument.id(), newUid(parsedDocument.id()))); + } + } + } + appendOnlyIndexer.join(120_000); + assertThat(engine.getMaxSeqNoOfNonAppendOnlyOperations(), equalTo(maxSeqNoOfNonAppendOnly)); + globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); + engine.syncTranslog(); + engine.flush(); + } + try (InternalEngine engine = createEngine(store, translogPath, globalCheckpoint::get)) { + assertThat("max_seqno from non-append-only was not bootstrap from the safe commit", + engine.getMaxSeqNoOfNonAppendOnlyOperations(), equalTo(globalCheckpoint.get())); + } + } + + public void testSkipOptimizeForExposedAppendOnlyOperations() throws Exception { + long lookupTimes = 0L; + final LocalCheckpointTracker localCheckpointTracker = engine.getLocalCheckpointTracker(); + final int initDocs = between(0, 10); + for (int i = 0; i < initDocs; i++) { + index(engine, i); + lookupTimes++; + } + // doc1 is delayed and arrived after a non-append-only op. + final long seqNoAppendOnly1 = localCheckpointTracker.generateSeqNo(); + final long seqnoNormalOp = localCheckpointTracker.generateSeqNo(); + if (randomBoolean()) { + engine.index(replicaIndexForDoc( + testParsedDocument("d", null, testDocumentWithTextField(), SOURCE, null), 1, seqnoNormalOp, false)); + } else { + engine.delete(replicaDeleteForDoc("d", 1, seqnoNormalOp, randomNonNegativeLong())); + } + lookupTimes++; + assertThat(engine.getNumVersionLookups(), equalTo(lookupTimes)); + assertThat(engine.getMaxSeqNoOfNonAppendOnlyOperations(), equalTo(seqnoNormalOp)); + + // should not optimize for doc1 and process as a regular doc (eg. look up in version map) + engine.index(appendOnlyReplica(testParsedDocument("append-only-1", null, testDocumentWithTextField(), SOURCE, null), + false, randomNonNegativeLong(), seqNoAppendOnly1)); + lookupTimes++; + assertThat(engine.getNumVersionLookups(), equalTo(lookupTimes)); + + // optimize for other append-only 2 (its seqno > max_seqno of non-append-only) - do not look up in version map. + engine.index(appendOnlyReplica(testParsedDocument("append-only-2", null, testDocumentWithTextField(), SOURCE, null), + false, randomNonNegativeLong(), localCheckpointTracker.generateSeqNo())); + assertThat(engine.getNumVersionLookups(), equalTo(lookupTimes)); + } } diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index 4762c23319a4a..baa56ee9585f6 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; @@ -419,6 +420,27 @@ private void updateGCDeleteCycle(IndexShard shard, TimeValue interval) { shard.onSettingsChanged(); } + /** + * This test ensures the consistency between primary and replica when non-append-only (eg. index request with id or delete) operation + * of the same document is processed before the original append-only request on replicas. The append-only document can be exposed and + * deleted on the primary before it is added to replica. Replicas should treat a late append-only request as a regular index request. + */ + public void testOutOfOrderDeliveryForAppendOnlyOperations() throws Exception { + try (ReplicationGroup shards = createGroup(1)) { + shards.startAll(); + final IndexShard primary = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + // Append-only request - without id + final BulkShardRequest indexRequest = indexOnPrimary( + new IndexRequest(index.getName(), "type", null).source("{}", XContentType.JSON), primary); + final String docId = Iterables.get(getShardDocUIDs(primary), 0); + final BulkShardRequest deleteRequest = deleteOnPrimary(new DeleteRequest(index.getName(), "type", docId), primary); + deleteOnReplica(deleteRequest, shards, replica); + indexOnReplica(indexRequest, shards, replica); + shards.assertAllEqual(0); + } + } + /** Throws documentFailure on every indexing operation */ static class ThrowingDocumentFailureEngineFactory implements EngineFactory { final String documentFailureMessage; From d2baf4b19136a57bd9205a556f753a499e4d357b Mon Sep 17 00:00:00 2001 From: Andrew Banchich Date: Tue, 27 Mar 2018 06:16:18 -0400 Subject: [PATCH 10/22] [Docs] Spelling and grammar changes to reindex.asciidoc (#29232) --- docs/reference/docs/reindex.asciidoc | 124 +++++++++++++-------------- 1 file changed, 62 insertions(+), 62 deletions(-) diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index fe13c0c206eb7..5f34371ab8467 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -136,7 +136,7 @@ POST _reindex // TEST[setup:twitter] You can limit the documents by adding a type to the `source` or by adding a -query. This will only copy ++tweet++'s made by `kimchy` into `new_twitter`: +query. This will only copy tweets made by `kimchy` into `new_twitter`: [source,js] -------------------------------------------------- @@ -161,11 +161,13 @@ POST _reindex `index` and `type` in `source` can both be lists, allowing you to copy from lots of sources in one request. This will copy documents from the `_doc` and -`post` types in the `twitter` and `blog` index. It'd include the `post` type in -the `twitter` index and the `_doc` type in the `blog` index. If you want to be -more specific you'll need to use the `query`. It also makes no effort to handle -ID collisions. The target index will remain valid but it's not easy to predict -which document will survive because the iteration order isn't well defined. +`post` types in the `twitter` and `blog` index. The copied documents would include the +`post` type in the `twitter` index and the `_doc` type in the `blog` index. For more +specific parameters, you can use `query`. + +The Reindex API makes no effort to handle ID collisions. For such issues, the target index +will remain valid, but it's not easy to predict which document will survive because +the iteration order isn't well defined. [source,js] -------------------------------------------------- @@ -203,8 +205,8 @@ POST _reindex // CONSOLE // TEST[setup:twitter] -If you want a particular set of documents from the twitter index you'll -need to sort. Sorting makes the scroll less efficient but in some contexts +If you want a particular set of documents from the `twitter` index you'll +need to use `sort`. Sorting makes the scroll less efficient but in some contexts it's worth it. If possible, prefer a more selective query to `size` and `sort`. This will copy 10000 documents from `twitter` into `new_twitter`: @@ -226,8 +228,8 @@ POST _reindex // TEST[setup:twitter] The `source` section supports all the elements that are supported in a -<>. For instance only a subset of the -fields from the original documents can be reindexed using source filtering +<>. For instance, only a subset of the +fields from the original documents can be reindexed using `source` filtering as follows: [source,js] @@ -286,10 +288,10 @@ Set `ctx.op = "delete"` if your script decides that the document must be deleted from the destination index. The deletion will be reported in the `deleted` counter in the <>. -Setting `ctx.op` to anything else is an error. Setting any -other field in `ctx` is an error. +Setting `ctx.op` to anything else will return an error, as will setting any +other field in `ctx`. -Think of the possibilities! Just be careful! With great power.... You can +Think of the possibilities! Just be careful; you are able to change: * `_id` @@ -299,7 +301,7 @@ change: * `_routing` Setting `_version` to `null` or clearing it from the `ctx` map is just like not -sending the version in an indexing request. It will cause that document to be +sending the version in an indexing request; it will cause the document to be overwritten in the target index regardless of the version on the target or the version type you use in the `_reindex` request. @@ -310,11 +312,11 @@ preserved unless it's changed by the script. You can set `routing` on the `keep`:: Sets the routing on the bulk request sent for each match to the routing on -the match. The default. +the match. This is the default value. `discard`:: -Sets the routing on the bulk request sent for each match to null. +Sets the routing on the bulk request sent for each match to `null`. `=`:: @@ -422,7 +424,7 @@ POST _reindex The `host` parameter must contain a scheme, host, and port (e.g. `https://otherhost:9200`). The `username` and `password` parameters are -optional and when they are present reindex will connect to the remote +optional, and when they are present `_reindex` will connect to the remote Elasticsearch node using basic auth. Be sure to use `https` when using basic auth or the password will be sent in plain text. @@ -446,7 +448,7 @@ NOTE: Reindexing from remote clusters does not support Reindexing from a remote server uses an on-heap buffer that defaults to a maximum size of 100mb. If the remote index includes very large documents you'll -need to use a smaller batch size. The example below sets the batch size `10` +need to use a smaller batch size. The example below sets the batch size to `10` which is very, very small. [source,js] @@ -477,8 +479,8 @@ POST _reindex It is also possible to set the socket read timeout on the remote connection with the `socket_timeout` field and the connection timeout with the -`connect_timeout` field. Both default to thirty seconds. This example -sets the socket read timeout to one minute and the connection timeout to ten +`connect_timeout` field. Both default to 30 seconds. This example +sets the socket read timeout to one minute and the connection timeout to 10 seconds: [source,js] @@ -533,14 +535,14 @@ for details. `timeout` controls how long each write request waits for unavailabl shards to become available. Both work exactly how they work in the <>. As `_reindex` uses scroll search, you can also specify the `scroll` parameter to control how long it keeps the "search context" alive, -eg `?scroll=10m`, by default it's 5 minutes. +(e.g. `?scroll=10m`). The default value is 5 minutes. `requests_per_second` can be set to any positive decimal number (`1.4`, `6`, -`1000`, etc) and throttles rate at which reindex issues batches of index +`1000`, etc) and throttles the rate at which `_reindex` issues batches of index operations by padding each batch with a wait time. The throttling can be disabled by setting `requests_per_second` to `-1`. -The throttling is done by waiting between batches so that scroll that reindex +The throttling is done by waiting between batches so that the `scroll` which `_reindex` uses internally can be given a timeout that takes into account the padding. The padding time is the difference between the batch size divided by the `requests_per_second` and the time spent writing. By default the batch size is @@ -552,9 +554,9 @@ target_time = 1000 / 500 per second = 2 seconds wait_time = target_time - write_time = 2 seconds - .5 seconds = 1.5 seconds -------------------------------------------------- -Since the batch is issued as a single `_bulk` request large batch sizes will +Since the batch is issued as a single `_bulk` request, large batch sizes will cause Elasticsearch to create many requests and then wait for a while before -starting the next set. This is "bursty" instead of "smooth". The default is `-1`. +starting the next set. This is "bursty" instead of "smooth". The default value is `-1`. [float] [[docs-reindex-response-body]] @@ -606,12 +608,12 @@ The JSON response looks like this: `took`:: -The number of milliseconds from start to end of the whole operation. +The total milliseconds the entire operation took. `timed_out`:: This flag is set to `true` if any of the requests executed during the -reindex has timed out. +reindex timed out. `total`:: @@ -657,7 +659,7 @@ The number of requests per second effectively executed during the reindex. `throttled_until_millis`:: -This field should always be equal to zero in a delete by query response. It only +This field should always be equal to zero in a `_delete_by_query` response. It only has meaning when using the <>, where it indicates the next time (in milliseconds since epoch) a throttled request will be executed again in order to conform to `requests_per_second`. @@ -681,7 +683,7 @@ GET _tasks?detailed=true&actions=*reindex -------------------------------------------------- // CONSOLE -The responses looks like: +The response looks like: [source,js] -------------------------------------------------- @@ -726,9 +728,9 @@ The responses looks like: // NOTCONSOLE // We can't test tasks output -<1> this object contains the actual status. It is just like the response json -with the important addition of the `total` field. `total` is the total number -of operations that the reindex expects to perform. You can estimate the +<1> this object contains the actual status. It is identical to the response JSON +except for the important addition of the `total` field. `total` is the total number +of operations that the `_reindex` expects to perform. You can estimate the progress by adding the `updated`, `created`, and `deleted` fields. The request will finish when their sum is equal to the `total` field. @@ -743,7 +745,7 @@ GET /_tasks/taskId:1 The advantage of this API is that it integrates with `wait_for_completion=false` to transparently return the status of completed tasks. If the task is completed -and `wait_for_completion=false` was set on it them it'll come back with a +and `wait_for_completion=false` was set, it will return a `results` or an `error` field. The cost of this feature is the document that `wait_for_completion=false` creates at `.tasks/task/${taskId}`. It is up to you to delete that document. @@ -761,10 +763,10 @@ POST _tasks/task_id:1/_cancel -------------------------------------------------- // CONSOLE -The `task_id` can be found using the tasks API above. +The `task_id` can be found using the Tasks API. -Cancelation should happen quickly but might take a few seconds. The task status -API above will continue to list the task until it is wakes to cancel itself. +Cancelation should happen quickly but might take a few seconds. The Tasks +API will continue to list the task until it wakes to cancel itself. [float] @@ -780,9 +782,9 @@ POST _reindex/task_id:1/_rethrottle?requests_per_second=-1 -------------------------------------------------- // CONSOLE -The `task_id` can be found using the tasks API above. +The `task_id` can be found using the Tasks API above. -Just like when setting it on the `_reindex` API `requests_per_second` +Just like when setting it on the Reindex API, `requests_per_second` can be either `-1` to disable throttling or any decimal number like `1.7` or `12` to throttle to that level. Rethrottling that speeds up the query takes effect immediately but rethrotting that slows down the query will @@ -806,7 +808,7 @@ POST test/_doc/1?refresh -------------------------------------------------- // CONSOLE -But you don't like the name `flag` and want to replace it with `tag`. +but you don't like the name `flag` and want to replace it with `tag`. `_reindex` can create the other index for you: [source,js] @@ -836,7 +838,7 @@ GET test2/_doc/1 // CONSOLE // TEST[continued] -and it'll look like: +which will return: [source,js] -------------------------------------------------- @@ -854,8 +856,6 @@ and it'll look like: -------------------------------------------------- // TESTRESPONSE -Or you can search by `tag` or whatever you want. - [float] [[docs-reindex-slice]] === Slicing @@ -902,7 +902,7 @@ POST _reindex // CONSOLE // TEST[setup:big_twitter] -Which you can verify works with: +You can verify this works by: [source,js] ---------------------------------------------------------------- @@ -912,7 +912,7 @@ POST new_twitter/_search?size=0&filter_path=hits.total // CONSOLE // TEST[continued] -Which results in a sensible `total` like this one: +which results in a sensible `total` like this one: [source,js] ---------------------------------------------------------------- @@ -928,7 +928,7 @@ Which results in a sensible `total` like this one: [[docs-reindex-automatic-slice]] ==== Automatic slicing -You can also let reindex automatically parallelize using <> to +You can also let `_reindex` automatically parallelize using <> to slice on `_uid`. Use `slices` to specify the number of slices to use: [source,js] @@ -946,7 +946,7 @@ POST _reindex?slices=5&refresh // CONSOLE // TEST[setup:big_twitter] -Which you also can verify works with: +You can also this verify works by: [source,js] ---------------------------------------------------------------- @@ -955,7 +955,7 @@ POST new_twitter/_search?size=0&filter_path=hits.total // CONSOLE // TEST[continued] -Which results in a sensible `total` like this one: +which results in a sensible `total` like this one: [source,js] ---------------------------------------------------------------- @@ -979,7 +979,7 @@ section above, creating sub-requests which means it has some quirks: sub-requests are "child" tasks of the task for the request with `slices`. * Fetching the status of the task for the request with `slices` only contains the status of completed slices. -* These sub-requests are individually addressable for things like cancellation +* These sub-requests are individually addressable for things like cancelation and rethrottling. * Rethrottling the request with `slices` will rethrottle the unfinished sub-request proportionally. @@ -992,7 +992,7 @@ are distributed proportionally to each sub-request. Combine that with the point above about distribution being uneven and you should conclude that the using `size` with `slices` might not result in exactly `size` documents being `_reindex`ed. -* Each sub-requests gets a slightly different snapshot of the source index +* Each sub-request gets a slightly different snapshot of the source index, though these are all taken at approximately the same time. [float] @@ -1000,12 +1000,12 @@ though these are all taken at approximately the same time. ===== Picking the number of slices If slicing automatically, setting `slices` to `auto` will choose a reasonable -number for most indices. If you're slicing manually or otherwise tuning +number for most indices. If slicing manually or otherwise tuning automatic slicing, use these guidelines. Query performance is most efficient when the number of `slices` is equal to the -number of shards in the index. If that number is large, (for example, -500) choose a lower number as too many `slices` will hurt performance. Setting +number of shards in the index. If that number is large (e.g. 500), +choose a lower number as too many `slices` will hurt performance. Setting `slices` higher than the number of shards generally does not improve efficiency and adds overhead. @@ -1018,10 +1018,10 @@ documents being reindexed and cluster resources. [float] === Reindex daily indices -You can use `_reindex` in combination with <> - to reindex daily indices to apply a new template to the existing documents. +You can use `_reindex` in combination with <> +to reindex daily indices to apply a new template to the existing documents. -Assuming you have indices consisting of documents as following: +Assuming you have indices consisting of documents as follows: [source,js] ---------------------------------------------------------------- @@ -1032,12 +1032,12 @@ PUT metricbeat-2016.05.31/_doc/1?refresh ---------------------------------------------------------------- // CONSOLE -The new template for the `metricbeat-*` indices is already loaded into Elasticsearch +The new template for the `metricbeat-*` indices is already loaded into Elasticsearch, but it applies only to the newly created indices. Painless can be used to reindex the existing documents and apply the new template. The script below extracts the date from the index name and creates a new index -with `-1` appended. All data from `metricbeat-2016.05.31` will be reindex +with `-1` appended. All data from `metricbeat-2016.05.31` will be reindexed into `metricbeat-2016.05.31-1`. [source,js] @@ -1059,7 +1059,7 @@ POST _reindex // CONSOLE // TEST[continued] -All documents from the previous metricbeat indices now can be found in the `*-1` indices. +All documents from the previous metricbeat indices can now be found in the `*-1` indices. [source,js] ---------------------------------------------------------------- @@ -1069,13 +1069,13 @@ GET metricbeat-2016.05.31-1/_doc/1 // CONSOLE // TEST[continued] -The previous method can also be used in combination with <> -to only load the existing data into the new index, but also rename fields if needed. +The previous method can also be used in conjunction with <> +to load only the existing data into the new index and rename any fields if needed. [float] === Extracting a random subset of an index -Reindex can be used to extract a random subset of an index for testing: +`_reindex` can be used to extract a random subset of an index for testing: [source,js] ---------------------------------------------------------------- @@ -1100,5 +1100,5 @@ POST _reindex // CONSOLE // TEST[setup:big_twitter] -<1> Reindex defaults to sorting by `_doc` so `random_score` won't have any +<1> `_reindex` defaults to sorting by `_doc` so `random_score` will not have any effect unless you override the sort to `_score`. From 8d6832c5ee5298e1befc372ed0b999dfeeeac279 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 27 Mar 2018 15:21:11 +0200 Subject: [PATCH 11/22] Make SearchStats implement Writeable (#29258) Moves another class over from Streamable to Writeable. By this, also some constructors can be removed or made private. --- .../admin/indices/stats/CommonStats.java | 4 +- .../index/search/stats/SearchStats.java | 95 ++++++++----------- .../search/stats/SearchStatsTests.java} | 9 +- 3 files changed, 44 insertions(+), 64 deletions(-) rename server/src/test/java/org/elasticsearch/{search/stats/SearchStatsUnitTests.java => index/search/stats/SearchStatsTests.java} (93%) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java b/server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java index 71360c359d311..6379f8da21aa2 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/stats/CommonStats.java @@ -233,7 +233,7 @@ public CommonStats(StreamInput in) throws IOException { store = in.readOptionalStreamable(StoreStats::new); indexing = in.readOptionalStreamable(IndexingStats::new); get = in.readOptionalStreamable(GetStats::new); - search = in.readOptionalStreamable(SearchStats::new); + search = in.readOptionalWriteable(SearchStats::new); merge = in.readOptionalStreamable(MergeStats::new); refresh = in.readOptionalStreamable(RefreshStats::new); flush = in.readOptionalStreamable(FlushStats::new); @@ -253,7 +253,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalStreamable(store); out.writeOptionalStreamable(indexing); out.writeOptionalStreamable(get); - out.writeOptionalStreamable(search); + out.writeOptionalWriteable(search); out.writeOptionalStreamable(merge); out.writeOptionalStreamable(refresh); out.writeOptionalStreamable(flush); diff --git a/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java b/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java index 519cd9ff9ae71..5f514b89b64a2 100644 --- a/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java +++ b/server/src/main/java/org/elasticsearch/index/search/stats/SearchStats.java @@ -23,19 +23,20 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; -public class SearchStats implements Streamable, ToXContentFragment { +public class SearchStats implements Writeable, ToXContentFragment { - public static class Stats implements Streamable, ToXContentFragment { + public static class Stats implements Writeable, ToXContentFragment { private long queryCount; private long queryTimeInMillis; @@ -53,8 +54,8 @@ public static class Stats implements Streamable, ToXContentFragment { private long suggestTimeInMillis; private long suggestCurrent; - Stats() { - + private Stats() { + // for internal use, initializes all counts to 0 } public Stats( @@ -78,16 +79,24 @@ public Stats( this.suggestCount = suggestCount; this.suggestTimeInMillis = suggestTimeInMillis; this.suggestCurrent = suggestCurrent; - } - public Stats(Stats stats) { - this( - stats.queryCount, stats.queryTimeInMillis, stats.queryCurrent, - stats.fetchCount, stats.fetchTimeInMillis, stats.fetchCurrent, - stats.scrollCount, stats.scrollTimeInMillis, stats.scrollCurrent, - stats.suggestCount, stats.suggestTimeInMillis, stats.suggestCurrent - ); + private Stats(StreamInput in) throws IOException { + queryCount = in.readVLong(); + queryTimeInMillis = in.readVLong(); + queryCurrent = in.readVLong(); + + fetchCount = in.readVLong(); + fetchTimeInMillis = in.readVLong(); + fetchCurrent = in.readVLong(); + + scrollCount = in.readVLong(); + scrollTimeInMillis = in.readVLong(); + scrollCurrent = in.readVLong(); + + suggestCount = in.readVLong(); + suggestTimeInMillis = in.readVLong(); + suggestCurrent = in.readVLong(); } public void add(Stats stats) { @@ -173,28 +182,7 @@ public long getSuggestCurrent() { } public static Stats readStats(StreamInput in) throws IOException { - Stats stats = new Stats(); - stats.readFrom(in); - return stats; - } - - @Override - public void readFrom(StreamInput in) throws IOException { - queryCount = in.readVLong(); - queryTimeInMillis = in.readVLong(); - queryCurrent = in.readVLong(); - - fetchCount = in.readVLong(); - fetchTimeInMillis = in.readVLong(); - fetchCurrent = in.readVLong(); - - scrollCount = in.readVLong(); - scrollTimeInMillis = in.readVLong(); - scrollCurrent = in.readVLong(); - - suggestCount = in.readVLong(); - suggestTimeInMillis = in.readVLong(); - suggestCurrent = in.readVLong(); + return new Stats(in); } @Override @@ -238,11 +226,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } } - Stats totalStats; - long openContexts; + private final Stats totalStats; + private long openContexts; @Nullable - Map groupStats; + private Map groupStats; public SearchStats() { totalStats = new Stats(); @@ -254,27 +242,27 @@ public SearchStats(Stats totalStats, long openContexts, @Nullable Map(searchStats.groupStats.size()); } for (Map.Entry entry : searchStats.groupStats.entrySet()) { - Stats stats = groupStats.get(entry.getKey()); - if (stats == null) { - groupStats.put(entry.getKey(), new Stats(entry.getValue())); - } else { - stats.add(entry.getValue()); - } + groupStats.putIfAbsent(entry.getKey(), new Stats()); + groupStats.get(entry.getKey()).add(entry.getValue()); } } } @@ -296,7 +284,7 @@ public long getOpenContexts() { @Nullable public Map getGroupStats() { - return this.groupStats; + return this.groupStats != null ? Collections.unmodifiableMap(this.groupStats) : null; } @Override @@ -344,15 +332,6 @@ static final class Fields { static final String SUGGEST_CURRENT = "suggest_current"; } - @Override - public void readFrom(StreamInput in) throws IOException { - totalStats = Stats.readStats(in); - openContexts = in.readVLong(); - if (in.readBoolean()) { - groupStats = in.readMap(StreamInput::readString, Stats::readStats); - } - } - @Override public void writeTo(StreamOutput out) throws IOException { totalStats.writeTo(out); diff --git a/server/src/test/java/org/elasticsearch/search/stats/SearchStatsUnitTests.java b/server/src/test/java/org/elasticsearch/index/search/stats/SearchStatsTests.java similarity index 93% rename from server/src/test/java/org/elasticsearch/search/stats/SearchStatsUnitTests.java rename to server/src/test/java/org/elasticsearch/index/search/stats/SearchStatsTests.java index 15fa7e64e3f67..5ec7aeaa0b2be 100644 --- a/server/src/test/java/org/elasticsearch/search/stats/SearchStatsUnitTests.java +++ b/server/src/test/java/org/elasticsearch/index/search/stats/SearchStatsTests.java @@ -17,16 +17,16 @@ * under the License. */ -package org.elasticsearch.search.stats; +package org.elasticsearch.index.search.stats; -import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.SearchStats.Stats; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; import java.util.Map; -public class SearchStatsUnitTests extends ESTestCase { +public class SearchStatsTests extends ESTestCase { + // https://github.com/elastic/elasticsearch/issues/7644 public void testShardLevelSearchGroupStats() throws Exception { // let's create two dummy search stats with groups @@ -52,7 +52,7 @@ public void testShardLevelSearchGroupStats() throws Exception { assertStats(groupStats1.get("group1"), 3); } - private void assertStats(Stats stats, long equalTo) { + private static void assertStats(Stats stats, long equalTo) { assertEquals(equalTo, stats.getQueryCount()); assertEquals(equalTo, stats.getQueryTimeInMillis()); assertEquals(equalTo, stats.getQueryCurrent()); @@ -66,4 +66,5 @@ private void assertStats(Stats stats, long equalTo) { assertEquals(equalTo, stats.getSuggestTimeInMillis()); assertEquals(equalTo, stats.getSuggestCurrent()); } + } From d1d3edf156fe074f26fe78c73d764ec1ebd60ff2 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 27 Mar 2018 09:45:51 -0400 Subject: [PATCH 12/22] TEST: Use different translog dir for a new engine In #testPruneOnlyDeletesAtMostLocalCheckpoint, we create a new engine but mistakenly use the same translog directory of the existing engine. This prevents translog files from cleaning up when closing the engines. ERROR 0.12s J2 | InternalEngineTests.testPruneOnlyDeletesAtMostLocalCheckpoint <<< FAILURES! > Throwable #1: java.io.IOException: could not remove the following files (in the order of attempts): > translog-primary-060/translog-2.tlog: java.io.IOException: access denied: This commit makes sure to use a separate directory for each engine in this tes. --- .../elasticsearch/index/engine/InternalEngineTests.java | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index e022330b664c3..1ecb1829234ab 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -4474,21 +4474,14 @@ public void testPruneOnlyDeletesAtMostLocalCheckpoint() throws Exception { final AtomicLong clock = new AtomicLong(0); threadPool = spy(threadPool); when(threadPool.relativeTimeInMillis()).thenAnswer(invocation -> clock.get()); - final EngineConfig config = engine.config(); final long gcInterval = randomIntBetween(0, 10); final IndexSettings indexSettings = engine.config().getIndexSettings(); final IndexMetaData indexMetaData = IndexMetaData.builder(indexSettings.getIndexMetaData()) .settings(Settings.builder().put(indexSettings.getSettings()) .put(IndexSettings.INDEX_GC_DELETES_SETTING.getKey(), TimeValue.timeValueMillis(gcInterval).getStringRep())).build(); indexSettings.updateIndexMetaData(indexMetaData); - try (Store store = createStore(); - InternalEngine engine = createEngine(new EngineConfig(config.getShardId(), config.getAllocationId(), threadPool, - indexSettings, config.getWarmer(), store, config.getMergePolicy(), config.getAnalyzer(), config.getSimilarity(), - new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), - config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), - config.getIndexSort(), config.getTranslogRecoveryRunner(), config.getCircuitBreakerService(), - config.getGlobalCheckpointSupplier()))) { + InternalEngine engine = createEngine(store, createTempDir())) { engine.config().setEnableGcDeletes(false); for (int i = 0, docs = scaledRandomIntBetween(0, 10); i < docs; i++) { index(engine, i); From 13f9e922f358897caf9f49fa2c9495a96382f48c Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 27 Mar 2018 16:15:44 +0200 Subject: [PATCH 13/22] REST client: hosts marked dead for the first time should not be immediately retried (#29230) This was the plan from day one but due to a silly bug nodes were immediately retried after they were marked as dead for the first time. From the second time on, the expected backoff was applied. --- .../elasticsearch/client/DeadHostState.java | 63 ++++++++-- .../org/elasticsearch/client/RestClient.java | 15 +-- .../client/DeadHostStateTests.java | 118 ++++++++++++++++++ .../client/RestClientSingleHostTests.java | 42 +------ .../elasticsearch/client/RestClientTests.java | 43 ++++++- 5 files changed, 223 insertions(+), 58 deletions(-) create mode 100644 client/rest/src/test/java/org/elasticsearch/client/DeadHostStateTests.java diff --git a/client/rest/src/main/java/org/elasticsearch/client/DeadHostState.java b/client/rest/src/main/java/org/elasticsearch/client/DeadHostState.java index a7b222da70e1d..452e71b14d93a 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/DeadHostState.java +++ b/client/rest/src/main/java/org/elasticsearch/client/DeadHostState.java @@ -26,31 +26,50 @@ * when the host should be retried (based on number of previous failed attempts). * Class is immutable, a new copy of it should be created each time the state has to be changed. */ -final class DeadHostState { +final class DeadHostState implements Comparable { private static final long MIN_CONNECTION_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(1); private static final long MAX_CONNECTION_TIMEOUT_NANOS = TimeUnit.MINUTES.toNanos(30); - static final DeadHostState INITIAL_DEAD_STATE = new DeadHostState(); - private final int failedAttempts; private final long deadUntilNanos; + private final TimeSupplier timeSupplier; - private DeadHostState() { + /** + * Build the initial dead state of a host. Useful when a working host stops functioning + * and needs to be marked dead after its first failure. In such case the host will be retried after a minute or so. + * + * @param timeSupplier a way to supply the current time and allow for unit testing + */ + DeadHostState(TimeSupplier timeSupplier) { this.failedAttempts = 1; - this.deadUntilNanos = System.nanoTime() + MIN_CONNECTION_TIMEOUT_NANOS; + this.deadUntilNanos = timeSupplier.nanoTime() + MIN_CONNECTION_TIMEOUT_NANOS; + this.timeSupplier = timeSupplier; } /** - * We keep track of how many times a certain node fails consecutively. The higher that number is the longer we will wait - * to retry that same node again. Minimum is 1 minute (for a node the only failed once), maximum is 30 minutes (for a node - * that failed many consecutive times). + * Build the dead state of a host given its previous dead state. Useful when a host has been failing before, hence + * it already failed for one or more consecutive times. The more failed attempts we register the longer we wait + * to retry that same host again. Minimum is 1 minute (for a node the only failed once created + * through {@link #DeadHostState(TimeSupplier)}), maximum is 30 minutes (for a node that failed more than 10 consecutive times) + * + * @param previousDeadHostState the previous state of the host which allows us to increase the wait till the next retry attempt */ - DeadHostState(DeadHostState previousDeadHostState) { + DeadHostState(DeadHostState previousDeadHostState, TimeSupplier timeSupplier) { long timeoutNanos = (long)Math.min(MIN_CONNECTION_TIMEOUT_NANOS * 2 * Math.pow(2, previousDeadHostState.failedAttempts * 0.5 - 1), MAX_CONNECTION_TIMEOUT_NANOS); - this.deadUntilNanos = System.nanoTime() + timeoutNanos; + this.deadUntilNanos = timeSupplier.nanoTime() + timeoutNanos; this.failedAttempts = previousDeadHostState.failedAttempts + 1; + this.timeSupplier = timeSupplier; + } + + /** + * Indicates whether it's time to retry to failed host or not. + * + * @return true if the host should be retried, false otherwise + */ + boolean shallBeRetried() { + return timeSupplier.nanoTime() - deadUntilNanos > 0; } /** @@ -61,6 +80,15 @@ long getDeadUntilNanos() { return deadUntilNanos; } + int getFailedAttempts() { + return failedAttempts; + } + + @Override + public int compareTo(DeadHostState other) { + return Long.compare(deadUntilNanos, other.deadUntilNanos); + } + @Override public String toString() { return "DeadHostState{" + @@ -68,4 +96,19 @@ public String toString() { ", deadUntilNanos=" + deadUntilNanos + '}'; } + + /** + * Time supplier that makes timing aspects pluggable to ease testing + */ + interface TimeSupplier { + + TimeSupplier DEFAULT = new TimeSupplier() { + @Override + public long nanoTime() { + return System.nanoTime(); + } + }; + + long nanoTime(); + } } diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index 4aa1a9d815cf4..48349c3858938 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -47,6 +47,7 @@ import org.apache.http.nio.protocol.HttpAsyncRequestProducer; import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; +import javax.net.ssl.SSLHandshakeException; import java.io.Closeable; import java.io.IOException; import java.net.SocketTimeoutException; @@ -72,7 +73,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import javax.net.ssl.SSLHandshakeException; /** * Client that connects to an Elasticsearch cluster through HTTP. @@ -457,18 +457,18 @@ private HostTuple> nextHost() { do { Set filteredHosts = new HashSet<>(hostTuple.hosts); for (Map.Entry entry : blacklist.entrySet()) { - if (System.nanoTime() - entry.getValue().getDeadUntilNanos() < 0) { + if (entry.getValue().shallBeRetried() == false) { filteredHosts.remove(entry.getKey()); } } if (filteredHosts.isEmpty()) { - //last resort: if there are no good host to use, return a single dead one, the one that's closest to being retried + //last resort: if there are no good hosts to use, return a single dead one, the one that's closest to being retried List> sortedHosts = new ArrayList<>(blacklist.entrySet()); if (sortedHosts.size() > 0) { Collections.sort(sortedHosts, new Comparator>() { @Override public int compare(Map.Entry o1, Map.Entry o2) { - return Long.compare(o1.getValue().getDeadUntilNanos(), o2.getValue().getDeadUntilNanos()); + return o1.getValue().compareTo(o2.getValue()); } }); HttpHost deadHost = sortedHosts.get(0).getKey(); @@ -499,14 +499,15 @@ private void onResponse(HttpHost host) { * Called after each failed attempt. * Receives as an argument the host that was used for the failed attempt. */ - private void onFailure(HttpHost host) throws IOException { + private void onFailure(HttpHost host) { while(true) { - DeadHostState previousDeadHostState = blacklist.putIfAbsent(host, DeadHostState.INITIAL_DEAD_STATE); + DeadHostState previousDeadHostState = blacklist.putIfAbsent(host, new DeadHostState(DeadHostState.TimeSupplier.DEFAULT)); if (previousDeadHostState == null) { logger.debug("added host [" + host + "] to blacklist"); break; } - if (blacklist.replace(host, previousDeadHostState, new DeadHostState(previousDeadHostState))) { + if (blacklist.replace(host, previousDeadHostState, + new DeadHostState(previousDeadHostState, DeadHostState.TimeSupplier.DEFAULT))) { logger.debug("updated host [" + host + "] already in blacklist"); break; } diff --git a/client/rest/src/test/java/org/elasticsearch/client/DeadHostStateTests.java b/client/rest/src/test/java/org/elasticsearch/client/DeadHostStateTests.java new file mode 100644 index 0000000000000..75fbafd88f83c --- /dev/null +++ b/client/rest/src/test/java/org/elasticsearch/client/DeadHostStateTests.java @@ -0,0 +1,118 @@ +/* + * 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.client; + +import java.util.concurrent.TimeUnit; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; + +public class DeadHostStateTests extends RestClientTestCase { + + private static long[] EXPECTED_TIMEOUTS_SECONDS = new long[]{60, 84, 120, 169, 240, 339, 480, 678, 960, 1357, 1800}; + + public void testInitialDeadHostStateDefaultTimeSupplier() { + DeadHostState deadHostState = new DeadHostState(DeadHostState.TimeSupplier.DEFAULT); + long currentTime = System.nanoTime(); + assertThat(deadHostState.getDeadUntilNanos(), greaterThan(currentTime)); + assertThat(deadHostState.getFailedAttempts(), equalTo(1)); + } + + public void testDeadHostStateFromPreviousDefaultTimeSupplier() { + DeadHostState previous = new DeadHostState(DeadHostState.TimeSupplier.DEFAULT); + int iters = randomIntBetween(5, 30); + for (int i = 0; i < iters; i++) { + DeadHostState deadHostState = new DeadHostState(previous, DeadHostState.TimeSupplier.DEFAULT); + assertThat(deadHostState.getDeadUntilNanos(), greaterThan(previous.getDeadUntilNanos())); + assertThat(deadHostState.getFailedAttempts(), equalTo(previous.getFailedAttempts() + 1)); + previous = deadHostState; + } + } + + public void testCompareToDefaultTimeSupplier() { + int numObjects = randomIntBetween(EXPECTED_TIMEOUTS_SECONDS.length, 30); + DeadHostState[] deadHostStates = new DeadHostState[numObjects]; + for (int i = 0; i < numObjects; i++) { + if (i == 0) { + deadHostStates[i] = new DeadHostState(DeadHostState.TimeSupplier.DEFAULT); + } else { + deadHostStates[i] = new DeadHostState(deadHostStates[i - 1], DeadHostState.TimeSupplier.DEFAULT); + } + } + for (int k = 1; k < deadHostStates.length; k++) { + assertThat(deadHostStates[k - 1].getDeadUntilNanos(), lessThan(deadHostStates[k].getDeadUntilNanos())); + assertThat(deadHostStates[k - 1], lessThan(deadHostStates[k])); + } + } + + public void testShallBeRetried() { + ConfigurableTimeSupplier timeSupplier = new ConfigurableTimeSupplier(); + DeadHostState deadHostState = null; + for (int i = 0; i < EXPECTED_TIMEOUTS_SECONDS.length; i++) { + long expectedTimeoutSecond = EXPECTED_TIMEOUTS_SECONDS[i]; + timeSupplier.nanoTime = 0; + if (i == 0) { + deadHostState = new DeadHostState(timeSupplier); + } else { + deadHostState = new DeadHostState(deadHostState, timeSupplier); + } + for (int j = 0; j < expectedTimeoutSecond; j++) { + timeSupplier.nanoTime += TimeUnit.SECONDS.toNanos(1); + assertThat(deadHostState.shallBeRetried(), is(false)); + } + int iters = randomIntBetween(5, 30); + for (int j = 0; j < iters; j++) { + timeSupplier.nanoTime += TimeUnit.SECONDS.toNanos(1); + assertThat(deadHostState.shallBeRetried(), is(true)); + } + } + } + + public void testDeadHostStateTimeouts() { + ConfigurableTimeSupplier zeroTimeSupplier = new ConfigurableTimeSupplier(); + zeroTimeSupplier.nanoTime = 0L; + DeadHostState previous = new DeadHostState(zeroTimeSupplier); + for (long expectedTimeoutsSecond : EXPECTED_TIMEOUTS_SECONDS) { + assertThat(TimeUnit.NANOSECONDS.toSeconds(previous.getDeadUntilNanos()), equalTo(expectedTimeoutsSecond)); + previous = new DeadHostState(previous, zeroTimeSupplier); + } + //check that from here on the timeout does not increase + int iters = randomIntBetween(5, 30); + for (int i = 0; i < iters; i++) { + DeadHostState deadHostState = new DeadHostState(previous, zeroTimeSupplier); + assertThat(TimeUnit.NANOSECONDS.toSeconds(deadHostState.getDeadUntilNanos()), + equalTo(EXPECTED_TIMEOUTS_SECONDS[EXPECTED_TIMEOUTS_SECONDS.length - 1])); + previous = deadHostState; + } + } + + private static class ConfigurableTimeSupplier implements DeadHostState.TimeSupplier { + + long nanoTime; + + @Override + public long nanoTime() { + return nanoTime; + } + } +} diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java index caf9ce6be2e07..7786eefb97f01 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientSingleHostTests.java @@ -101,7 +101,7 @@ public class RestClientSingleHostTests extends RestClientTestCase { @Before @SuppressWarnings("unchecked") - public void createRestClient() throws IOException { + public void createRestClient() { httpClient = mock(CloseableHttpAsyncClient.class); when(httpClient.execute(any(HttpAsyncRequestProducer.class), any(HttpAsyncResponseConsumer.class), any(HttpClientContext.class), any(FutureCallback.class))).thenAnswer(new Answer>() { @@ -160,17 +160,6 @@ public void shutdownExec() { exec.shutdown(); } - public void testNullPath() throws IOException { - for (String method : getHttpMethods()) { - try { - restClient.performRequest(method, null); - fail("path set to null should fail!"); - } catch (NullPointerException e) { - assertEquals("path must not be null", e.getMessage()); - } - } - } - /** * Verifies the content of the {@link HttpRequest} that's internally created and passed through to the http client */ @@ -196,33 +185,6 @@ public void testInternalHttpRequest() throws Exception { } } - public void testSetHosts() throws IOException { - try { - restClient.setHosts((HttpHost[]) null); - fail("setHosts should have failed"); - } catch (IllegalArgumentException e) { - assertEquals("hosts must not be null nor empty", e.getMessage()); - } - try { - restClient.setHosts(); - fail("setHosts should have failed"); - } catch (IllegalArgumentException e) { - assertEquals("hosts must not be null nor empty", e.getMessage()); - } - try { - restClient.setHosts((HttpHost) null); - fail("setHosts should have failed"); - } catch (NullPointerException e) { - assertEquals("host cannot be null", e.getMessage()); - } - try { - restClient.setHosts(new HttpHost("localhost", 9200), null, new HttpHost("localhost", 9201)); - fail("setHosts should have failed"); - } catch (NullPointerException e) { - assertEquals("host cannot be null", e.getMessage()); - } - } - /** * End to end test for ok status codes */ @@ -289,7 +251,7 @@ public void testErrorStatusCodes() throws IOException { } } - public void testIOExceptions() throws IOException { + public void testIOExceptions() { for (String method : getHttpMethods()) { //IOExceptions should be let bubble up try { diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java index 33323d39663e2..ee6dbf449bd56 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java @@ -28,6 +28,7 @@ import java.util.Collections; import java.util.concurrent.CountDownLatch; +import static org.elasticsearch.client.RestClientTestUtil.getHttpMethods; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -147,8 +148,48 @@ public void testBuildUriLeavesPathUntouched() { } } + public void testSetHostsWrongArguments() throws IOException { + try (RestClient restClient = createRestClient()) { + restClient.setHosts((HttpHost[]) null); + fail("setHosts should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("hosts must not be null nor empty", e.getMessage()); + } + try (RestClient restClient = createRestClient()) { + restClient.setHosts(); + fail("setHosts should have failed"); + } catch (IllegalArgumentException e) { + assertEquals("hosts must not be null nor empty", e.getMessage()); + } + try (RestClient restClient = createRestClient()) { + restClient.setHosts((HttpHost) null); + fail("setHosts should have failed"); + } catch (NullPointerException e) { + assertEquals("host cannot be null", e.getMessage()); + } + try (RestClient restClient = createRestClient()) { + restClient.setHosts(new HttpHost("localhost", 9200), null, new HttpHost("localhost", 9201)); + fail("setHosts should have failed"); + } catch (NullPointerException e) { + assertEquals("host cannot be null", e.getMessage()); + } + } + + public void testNullPath() throws IOException { + try (RestClient restClient = createRestClient()) { + for (String method : getHttpMethods()) { + try { + restClient.performRequest(method, null); + fail("path set to null should fail!"); + } catch (NullPointerException e) { + assertEquals("path must not be null", e.getMessage()); + } + } + } + } + private static RestClient createRestClient() { HttpHost[] hosts = new HttpHost[]{new HttpHost("localhost", 9200)}; - return new RestClient(mock(CloseableHttpAsyncClient.class), randomLongBetween(1_000, 30_000), new Header[]{}, hosts, null, null); + return new RestClient(mock(CloseableHttpAsyncClient.class), randomIntBetween(1_000, 30_000), new Header[]{}, hosts, null, null); } } From dfc9e721d8b540193f9a99641edaf577f1b8c8f7 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 27 Mar 2018 12:17:57 -0400 Subject: [PATCH 14/22] TEST: Increase timeout for testPrimaryReplicaResyncFailed The default timeout (eg. 10 seconds) may not be enough for CI to re-allocate shards after the partion is healed. This commit increases the timeout to 30 seconds and enables logging in order to have more detailed information in case this test failed again. Closes #29060 --- .../elasticsearch/cluster/routing/PrimaryAllocationIT.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index d7a91c988e9da..407212936d1d6 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -43,6 +43,7 @@ import org.elasticsearch.test.disruption.NetworkDisruption; import org.elasticsearch.test.disruption.NetworkDisruption.NetworkDisconnect; import org.elasticsearch.test.disruption.NetworkDisruption.TwoPartitions; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; import java.util.Arrays; @@ -326,6 +327,8 @@ public void testForceAllocatePrimaryOnNoDecision() throws Exception { /** * This test asserts that replicas failed to execute resync operations will be failed but not marked as stale. */ + @TestLogging("_root:DEBUG, org.elasticsearch.cluster.routing.allocation:TRACE, org.elasticsearch.cluster.action.shard:TRACE," + + "org.elasticsearch.indices.recovery:TRACE, org.elasticsearch.cluster.routing.allocation.allocator:TRACE") public void testPrimaryReplicaResyncFailed() throws Exception { String master = internalCluster().startMasterOnlyNode(Settings.EMPTY); final int numberOfReplicas = between(2, 3); @@ -377,6 +380,7 @@ public void testPrimaryReplicaResyncFailed() throws Exception { client(master).admin().cluster().prepareUpdateSettings() .setTransientSettings(Settings.builder().put("cluster.routing.allocation.enable", "all")).get()); partition.stopDisrupting(); + partition.ensureHealthy(internalCluster()); logger.info("--> stop disrupting network and re-enable allocation"); assertBusy(() -> { ClusterState state = client(master).admin().cluster().prepareState().get().getState(); @@ -386,7 +390,7 @@ public void testPrimaryReplicaResyncFailed() throws Exception { IndexShard shard = internalCluster().getInstance(IndicesService.class, node).getShardOrNull(shardId); assertThat(shard.getLocalCheckpoint(), equalTo(numDocs + moreDocs)); } - }); + }, 30, TimeUnit.SECONDS); } } From 2aaa0573875632efef83e79f60f3f5aa882d2331 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Tue, 27 Mar 2018 18:55:42 +0200 Subject: [PATCH 15/22] Propagate ignore_unmapped to inner_hits (#29261) In 5.2 `ignore_unmapped` was added to `inner_hits` in order to ignore invalid mapping. This value was automatically set to the value defined in the parent query (`nested`, `has_child`, `has_parent`) but the refactoring of the parent/child in 5.6 removed this behavior unintentionally. This commit restores this behavior but also makes sure that we always automatically enforce this value when the query builder is used directly (previously this was only done by the XContent deserialization). Closes #29071 --- .../elasticsearch/join/query/HasChildQueryBuilder.java | 5 ++++- .../elasticsearch/join/query/HasParentQueryBuilder.java | 4 ++++ .../join/query/HasChildQueryBuilderTests.java | 9 +++++++-- .../join/query/HasParentQueryBuilderTests.java | 9 +++++++-- .../elasticsearch/index/query/NestedQueryBuilder.java | 4 ++++ .../index/query/NestedQueryBuilderTests.java | 3 ++- 6 files changed, 28 insertions(+), 6 deletions(-) diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java index 65a02b9c83e5d..0577aa01ebd8f 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasChildQueryBuilder.java @@ -167,6 +167,7 @@ public InnerHitBuilder innerHit() { public HasChildQueryBuilder innerHit(InnerHitBuilder innerHit) { this.innerHitBuilder = innerHit; + innerHitBuilder.setIgnoreUnmapped(ignoreUnmapped); return this; } @@ -212,6 +213,9 @@ public int minChildren() { */ public HasChildQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) { this.ignoreUnmapped = ignoreUnmapped; + if (innerHitBuilder!= null ){ + innerHitBuilder.setIgnoreUnmapped(ignoreUnmapped); + } return this; } @@ -291,7 +295,6 @@ public static HasChildQueryBuilder fromXContent(XContentParser parser) throws IO hasChildQueryBuilder.ignoreUnmapped(ignoreUnmapped); if (innerHitBuilder != null) { hasChildQueryBuilder.innerHit(innerHitBuilder); - hasChildQueryBuilder.ignoreUnmapped(ignoreUnmapped); } return hasChildQueryBuilder; } diff --git a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java index cce6cdc840479..5e2dd4206f2f7 100644 --- a/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java +++ b/modules/parent-join/src/main/java/org/elasticsearch/join/query/HasParentQueryBuilder.java @@ -145,6 +145,7 @@ public InnerHitBuilder innerHit() { public HasParentQueryBuilder innerHit(InnerHitBuilder innerHit) { this.innerHitBuilder = innerHit; + innerHitBuilder.setIgnoreUnmapped(ignoreUnmapped); return this; } @@ -155,6 +156,9 @@ public HasParentQueryBuilder innerHit(InnerHitBuilder innerHit) { */ public HasParentQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) { this.ignoreUnmapped = ignoreUnmapped; + if (innerHitBuilder != null) { + innerHitBuilder.setIgnoreUnmapped(ignoreUnmapped); + } return this; } diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java index 0dcf5933f4f23..4f4d965d59433 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasChildQueryBuilderTests.java @@ -158,8 +158,7 @@ protected HasChildQueryBuilder doCreateTestQueryBuilder() { hqb.innerHit(new InnerHitBuilder() .setName(randomAlphaOfLengthBetween(1, 10)) .setSize(randomIntBetween(0, 100)) - .addSort(new FieldSortBuilder(STRING_FIELD_NAME_2).order(SortOrder.ASC)) - .setIgnoreUnmapped(hqb.ignoreUnmapped())); + .addSort(new FieldSortBuilder(STRING_FIELD_NAME_2).order(SortOrder.ASC))); } return hqb; } @@ -345,13 +344,19 @@ public void testNonDefaultSimilarity() throws Exception { public void testIgnoreUnmapped() throws IOException { final HasChildQueryBuilder queryBuilder = new HasChildQueryBuilder("unmapped", new MatchAllQueryBuilder(), ScoreMode.None); + queryBuilder.innerHit(new InnerHitBuilder()); + assertFalse(queryBuilder.innerHit().isIgnoreUnmapped()); queryBuilder.ignoreUnmapped(true); + assertTrue(queryBuilder.innerHit().isIgnoreUnmapped()); Query query = queryBuilder.toQuery(createShardContext()); assertThat(query, notNullValue()); assertThat(query, instanceOf(MatchNoDocsQuery.class)); final HasChildQueryBuilder failingQueryBuilder = new HasChildQueryBuilder("unmapped", new MatchAllQueryBuilder(), ScoreMode.None); + failingQueryBuilder.innerHit(new InnerHitBuilder()); + assertFalse(failingQueryBuilder.innerHit().isIgnoreUnmapped()); failingQueryBuilder.ignoreUnmapped(false); + assertFalse(failingQueryBuilder.innerHit().isIgnoreUnmapped()); QueryShardException e = expectThrows(QueryShardException.class, () -> failingQueryBuilder.toQuery(createShardContext())); assertThat(e.getMessage(), containsString("[" + HasChildQueryBuilder.NAME + "] join field [join_field] doesn't hold [unmapped] as a child")); diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java index c7ded186c9aee..e2d45d22ab25d 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/HasParentQueryBuilderTests.java @@ -132,8 +132,7 @@ protected HasParentQueryBuilder doCreateTestQueryBuilder() { hqb.innerHit(new InnerHitBuilder() .setName(randomAlphaOfLengthBetween(1, 10)) .setSize(randomIntBetween(0, 100)) - .addSort(new FieldSortBuilder(STRING_FIELD_NAME_2).order(SortOrder.ASC)) - .setIgnoreUnmapped(hqb.ignoreUnmapped())); + .addSort(new FieldSortBuilder(STRING_FIELD_NAME_2).order(SortOrder.ASC))); } return hqb; } @@ -245,13 +244,19 @@ public void testFromJson() throws IOException { public void testIgnoreUnmapped() throws IOException { final HasParentQueryBuilder queryBuilder = new HasParentQueryBuilder("unmapped", new MatchAllQueryBuilder(), false); + queryBuilder.innerHit(new InnerHitBuilder()); + assertFalse(queryBuilder.innerHit().isIgnoreUnmapped()); queryBuilder.ignoreUnmapped(true); + assertTrue(queryBuilder.innerHit().isIgnoreUnmapped()); Query query = queryBuilder.toQuery(createShardContext()); assertThat(query, notNullValue()); assertThat(query, instanceOf(MatchNoDocsQuery.class)); final HasParentQueryBuilder failingQueryBuilder = new HasParentQueryBuilder("unmapped", new MatchAllQueryBuilder(), false); + failingQueryBuilder.innerHit(new InnerHitBuilder()); + assertFalse(failingQueryBuilder.innerHit().isIgnoreUnmapped()); failingQueryBuilder.ignoreUnmapped(false); + assertFalse(failingQueryBuilder.innerHit().isIgnoreUnmapped()); QueryShardException e = expectThrows(QueryShardException.class, () -> failingQueryBuilder.toQuery(createShardContext())); assertThat(e.getMessage(), containsString("[has_parent] join field [join_field] doesn't hold [unmapped] as a parent")); diff --git a/server/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java index 9ebd548cae1f0..889f41a037f86 100644 --- a/server/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/NestedQueryBuilder.java @@ -132,6 +132,7 @@ public InnerHitBuilder innerHit() { public NestedQueryBuilder innerHit(InnerHitBuilder innerHitBuilder) { this.innerHitBuilder = innerHitBuilder; + innerHitBuilder.setIgnoreUnmapped(ignoreUnmapped); return this; } @@ -149,6 +150,9 @@ public ScoreMode scoreMode() { */ public NestedQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) { this.ignoreUnmapped = ignoreUnmapped; + if (innerHitBuilder != null) { + innerHitBuilder.setIgnoreUnmapped(ignoreUnmapped); + } return this; } diff --git a/server/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java index 46e10bc7f224c..a2e6018d0ef6b 100644 --- a/server/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/NestedQueryBuilderTests.java @@ -347,7 +347,8 @@ public void testBuildIgnoreUnmappedNestQuery() throws Exception { }); innerHitBuilders.clear(); NestedQueryBuilder query2 = new NestedQueryBuilder("path", new MatchAllQueryBuilder(), ScoreMode.None); - query2.innerHit(leafInnerHits.setIgnoreUnmapped(true)); + query2.ignoreUnmapped(true); + query2.innerHit(leafInnerHits); query2.extractInnerHitBuilders(innerHitBuilders); assertThat(innerHitBuilders.size(), Matchers.equalTo(1)); assertTrue(innerHitBuilders.containsKey(leafInnerHits.getName())); From 3db6f1c9d5cde40e166870fd724ae85183544811 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Tue, 27 Mar 2018 20:11:21 +0200 Subject: [PATCH 16/22] Fix sporadic failure in CompositeValuesCollectorQueueTests This commit fixes a test bug that causes an NPE on empty segments. Closes #29269 --- .../composite/CompositeValuesCollectorQueueTests.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueueTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueueTests.java index 122c8185e2e70..edf732ce24a41 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueueTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueueTests.java @@ -26,6 +26,7 @@ import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.document.TextField; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.DocValues; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; @@ -210,21 +211,21 @@ private void testRandomCase(boolean forceMerge, ClassAndName... types) throws IO final MappedFieldType fieldType = types[i].fieldType; if (types[i].clazz == Long.class) { sources[i] = new LongValuesSource(bigArrays, fieldType, - context -> context.reader().getSortedNumericDocValues(fieldType.name()), value -> value, + context -> DocValues.getSortedNumeric(context.reader(), fieldType.name()), value -> value, DocValueFormat.RAW, size, 1); } else if (types[i].clazz == Double.class) { sources[i] = new DoubleValuesSource(bigArrays, fieldType, - context -> FieldData.sortableLongBitsToDoubles(context.reader().getSortedNumericDocValues(fieldType.name())), + context -> FieldData.sortableLongBitsToDoubles(DocValues.getSortedNumeric(context.reader(), fieldType.name())), size, 1); } else if (types[i].clazz == BytesRef.class) { if (forceMerge) { // we don't create global ordinals but we test this mode when the reader has a single segment // since ordinals are global in this case. sources[i] = new GlobalOrdinalValuesSource(bigArrays, fieldType, - context -> context.reader().getSortedSetDocValues(fieldType.name()), size, 1); + context -> DocValues.getSortedSet(context.reader(), fieldType.name()), size, 1); } else { sources[i] = new BinaryValuesSource(fieldType, - context -> FieldData.toString(context.reader().getSortedSetDocValues(fieldType.name())), size, 1); + context -> FieldData.toString(DocValues.getSortedSet(context.reader(), fieldType.name())), size, 1); } } else { assert(false); From b4c78019b02313bd1eaa61e0752ef3c2cf34f5a4 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 27 Mar 2018 12:58:22 -0600 Subject: [PATCH 17/22] Remove all dependencies from XContentBuilder (#29225) * Remove all dependencies from XContentBuilder This commit removes all of the non-JDK dependencies from XContentBuilder, with the exception of `CollectionUtils.ensureNoSelfReferences`. It adds a third extension point around dealing with time-based fields and formatters to work around the Joda dependency. This decoupling allows us to be able to move XContentBuilder to a separate lib so it can be available for things like the high level rest client. Relates to #28504 --- .../documentation/CRUDDocumentationIT.java | 4 +- .../snapshots/status/SnapshotStats.java | 5 +- .../cluster/stats/ClusterStatsNodes.java | 4 +- .../segments/IndicesSegmentResponse.java | 4 +- .../upgrade/get/UpgradeStatusResponse.java | 24 ++-- .../elasticsearch/cluster/ClusterInfo.java | 3 +- .../org/elasticsearch/cluster/DiskUsage.java | 6 +- .../cluster/metadata/IndexGraveyard.java | 2 +- .../allocation/NodeAllocationResult.java | 3 +- .../common/FieldMemoryStats.java | 3 +- .../common/xcontent/XContentBuilder.java | 126 +++++++----------- .../xcontent/XContentBuilderExtension.java | 17 +++ .../XContentElasticsearchExtension.java | 28 ++++ .../common/xcontent/XContentParser.java | 2 +- .../java/org/elasticsearch/http/HttpInfo.java | 2 +- .../index/cache/query/QueryCacheStats.java | 2 +- .../cache/request/RequestCacheStats.java | 2 +- .../index/engine/SegmentsStats.java | 22 +-- .../index/fielddata/FieldDataStats.java | 2 +- .../elasticsearch/index/merge/MergeStats.java | 9 +- .../elasticsearch/index/store/StoreStats.java | 2 +- .../index/translog/TranslogStats.java | 5 +- .../indices/recovery/RecoveryState.java | 15 ++- .../org/elasticsearch/monitor/fs/FsInfo.java | 15 ++- .../elasticsearch/monitor/jvm/JvmInfo.java | 12 +- .../elasticsearch/monitor/jvm/JvmStats.java | 23 ++-- .../monitor/process/ProcessStats.java | 2 +- .../suggest/completion/CompletionStats.java | 2 +- .../org/elasticsearch/tasks/TaskInfo.java | 2 +- .../transport/TransportStats.java | 4 +- .../cluster/metadata/IndexGraveyardTests.java | 4 +- .../common/xcontent/BaseXContentTestCase.java | 81 +++++------ .../builder/XContentBuilderTests.java | 7 +- .../aggregations/bucket/DateHistogramIT.java | 30 ++--- .../bucket/DateHistogramOffsetIT.java | 2 +- .../aggregations/bucket/DateRangeIT.java | 8 +- .../pipeline/DateDerivativeIT.java | 6 +- 37 files changed, 253 insertions(+), 237 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java index 5adc7bee273a0..09a3fbd4d16a8 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java @@ -121,7 +121,7 @@ public void testIndex() throws Exception { builder.startObject(); { builder.field("user", "kimchy"); - builder.field("postDate", new Date()); + builder.timeField("postDate", new Date()); builder.field("message", "trying out Elasticsearch"); } builder.endObject(); @@ -331,7 +331,7 @@ public void testUpdate() throws Exception { XContentBuilder builder = XContentFactory.jsonBuilder(); builder.startObject(); { - builder.field("updated", new Date()); + builder.timeField("updated", new Date()); builder.field("reason", "daily update"); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java index e7957e0ac0818..25951f73abc53 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/SnapshotStats.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentFragment; @@ -141,8 +142,8 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par builder.startObject(Fields.STATS); builder.field(Fields.NUMBER_OF_FILES, getNumberOfFiles()); builder.field(Fields.PROCESSED_FILES, getProcessedFiles()); - builder.byteSizeField(Fields.TOTAL_SIZE_IN_BYTES, Fields.TOTAL_SIZE, getTotalSize()); - builder.byteSizeField(Fields.PROCESSED_SIZE_IN_BYTES, Fields.PROCESSED_SIZE, getProcessedSize()); + builder.humanReadableField(Fields.TOTAL_SIZE_IN_BYTES, Fields.TOTAL_SIZE, new ByteSizeValue(getTotalSize())); + builder.humanReadableField(Fields.PROCESSED_SIZE_IN_BYTES, Fields.PROCESSED_SIZE, new ByteSizeValue(getProcessedSize())); builder.field(Fields.START_TIME_IN_MILLIS, getStartTime()); builder.humanReadableField(Fields.TIME_IN_MILLIS, Fields.TIME, new TimeValue(getTime())); builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java index 2efaf2245ea04..e465256a0763b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/ClusterStatsNodes.java @@ -501,8 +501,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) } builder.endArray(); builder.startObject(Fields.MEM); - builder.byteSizeField(Fields.HEAP_USED_IN_BYTES, Fields.HEAP_USED, heapUsed); - builder.byteSizeField(Fields.HEAP_MAX_IN_BYTES, Fields.HEAP_MAX, heapMax); + builder.humanReadableField(Fields.HEAP_USED_IN_BYTES, Fields.HEAP_USED, getHeapUsed()); + builder.humanReadableField(Fields.HEAP_MAX_IN_BYTES, Fields.HEAP_MAX, getHeapMax()); builder.endObject(); builder.field(Fields.THREADS, threads); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java index 7faf24329dadd..aa693c1b9e5cc 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/segments/IndicesSegmentResponse.java @@ -132,8 +132,8 @@ protected void addCustomXContentFields(XContentBuilder builder, Params params) t builder.field(Fields.GENERATION, segment.getGeneration()); builder.field(Fields.NUM_DOCS, segment.getNumDocs()); builder.field(Fields.DELETED_DOCS, segment.getDeletedDocs()); - builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, segment.getSizeInBytes()); - builder.byteSizeField(Fields.MEMORY_IN_BYTES, Fields.MEMORY, segment.getMemoryInBytes()); + builder.humanReadableField(Fields.SIZE_IN_BYTES, Fields.SIZE, segment.getSize()); + builder.humanReadableField(Fields.MEMORY_IN_BYTES, Fields.MEMORY, new ByteSizeValue(segment.getMemoryInBytes())); builder.field(Fields.COMMITTED, segment.isCommitted()); builder.field(Fields.SEARCH, segment.isSearch()); if (segment.getVersion() != null) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/get/UpgradeStatusResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/get/UpgradeStatusResponse.java index a45b8feda89ce..76a85a2416374 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/get/UpgradeStatusResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/upgrade/get/UpgradeStatusResponse.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.support.broadcast.BroadcastResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; @@ -116,9 +117,10 @@ public long getToUpgradeBytesAncient() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, getTotalBytes()); - builder.byteSizeField(Fields.SIZE_TO_UPGRADE_IN_BYTES, Fields.SIZE_TO_UPGRADE, getToUpgradeBytes()); - builder.byteSizeField(Fields.SIZE_TO_UPGRADE_ANCIENT_IN_BYTES, Fields.SIZE_TO_UPGRADE_ANCIENT, getToUpgradeBytesAncient()); + builder.humanReadableField(Fields.SIZE_IN_BYTES, Fields.SIZE, new ByteSizeValue(getTotalBytes())); + builder.humanReadableField(Fields.SIZE_TO_UPGRADE_IN_BYTES, Fields.SIZE_TO_UPGRADE, new ByteSizeValue(getToUpgradeBytes())); + builder.humanReadableField(Fields.SIZE_TO_UPGRADE_ANCIENT_IN_BYTES, Fields.SIZE_TO_UPGRADE_ANCIENT, + new ByteSizeValue(getToUpgradeBytesAncient())); String level = params.param("level", "indices"); boolean outputShards = "shards".equals(level); @@ -128,9 +130,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws for (IndexUpgradeStatus indexUpgradeStatus : getIndices().values()) { builder.startObject(indexUpgradeStatus.getIndex()); - builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, indexUpgradeStatus.getTotalBytes()); - builder.byteSizeField(Fields.SIZE_TO_UPGRADE_IN_BYTES, Fields.SIZE_TO_UPGRADE, indexUpgradeStatus.getToUpgradeBytes()); - builder.byteSizeField(Fields.SIZE_TO_UPGRADE_ANCIENT_IN_BYTES, Fields.SIZE_TO_UPGRADE_ANCIENT, indexUpgradeStatus.getToUpgradeBytesAncient()); + builder.humanReadableField(Fields.SIZE_IN_BYTES, Fields.SIZE, new ByteSizeValue(indexUpgradeStatus.getTotalBytes())); + builder.humanReadableField(Fields.SIZE_TO_UPGRADE_IN_BYTES, Fields.SIZE_TO_UPGRADE, + new ByteSizeValue(indexUpgradeStatus.getToUpgradeBytes())); + builder.humanReadableField(Fields.SIZE_TO_UPGRADE_ANCIENT_IN_BYTES, Fields.SIZE_TO_UPGRADE_ANCIENT, + new ByteSizeValue(indexUpgradeStatus.getToUpgradeBytesAncient())); if (outputShards) { builder.startObject(Fields.SHARDS); for (IndexShardUpgradeStatus indexShardUpgradeStatus : indexUpgradeStatus) { @@ -138,9 +142,11 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws for (ShardUpgradeStatus shardUpgradeStatus : indexShardUpgradeStatus) { builder.startObject(); - builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, getTotalBytes()); - builder.byteSizeField(Fields.SIZE_TO_UPGRADE_IN_BYTES, Fields.SIZE_TO_UPGRADE, getToUpgradeBytes()); - builder.byteSizeField(Fields.SIZE_TO_UPGRADE_ANCIENT_IN_BYTES, Fields.SIZE_TO_UPGRADE_ANCIENT, getToUpgradeBytesAncient()); + builder.humanReadableField(Fields.SIZE_IN_BYTES, Fields.SIZE, new ByteSizeValue(getTotalBytes())); + builder.humanReadableField(Fields.SIZE_TO_UPGRADE_IN_BYTES, Fields.SIZE_TO_UPGRADE, + new ByteSizeValue(getToUpgradeBytes())); + builder.humanReadableField(Fields.SIZE_TO_UPGRADE_ANCIENT_IN_BYTES, Fields.SIZE_TO_UPGRADE_ANCIENT, + new ByteSizeValue(getToUpgradeBytesAncient())); builder.startObject(Fields.ROUTING); builder.field(Fields.STATE, shardUpgradeStatus.getShardRouting().state()); diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterInfo.java b/server/src/main/java/org/elasticsearch/cluster/ClusterInfo.java index ad30598201304..d004e34d06efd 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterInfo.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterInfo.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -133,7 +134,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); // end "nodes" builder.startObject("shard_sizes"); { for (ObjectObjectCursor c : this.shardSizes) { - builder.byteSizeField(c.key + "_bytes", c.key, c.value); + builder.humanReadableField(c.key + "_bytes", c.key, new ByteSizeValue(c.value)); } } builder.endObject(); // end "shard_sizes" diff --git a/server/src/main/java/org/elasticsearch/cluster/DiskUsage.java b/server/src/main/java/org/elasticsearch/cluster/DiskUsage.java index ace7be50b661e..0b0c7ac75eafb 100644 --- a/server/src/main/java/org/elasticsearch/cluster/DiskUsage.java +++ b/server/src/main/java/org/elasticsearch/cluster/DiskUsage.java @@ -75,9 +75,9 @@ private static double truncatePercent(double pct) { XContentBuilder toShortXContent(XContentBuilder builder) throws IOException { builder.field("path", this.path); - builder.byteSizeField("total_bytes", "total", this.totalBytes); - builder.byteSizeField("used_bytes", "used", this.getUsedBytes()); - builder.byteSizeField("free_bytes", "free", this.freeBytes); + builder.humanReadableField("total_bytes", "total", new ByteSizeValue(this.totalBytes)); + builder.humanReadableField("used_bytes", "used", new ByteSizeValue(this.getUsedBytes())); + builder.humanReadableField("free_bytes", "free", new ByteSizeValue(this.freeBytes)); builder.field("free_disk_percent", truncatePercent(this.getFreeDiskAsPercentage())); builder.field("used_disk_percent", truncatePercent(this.getUsedDiskAsPercentage())); return builder; diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexGraveyard.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexGraveyard.java index d07052b9b4d28..9167b28a67b86 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexGraveyard.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexGraveyard.java @@ -434,7 +434,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa builder.startObject(); builder.field(INDEX_KEY); index.toXContent(builder, params); - builder.dateField(DELETE_DATE_IN_MILLIS_KEY, DELETE_DATE_KEY, deleteDateInMillis); + builder.timeField(DELETE_DATE_IN_MILLIS_KEY, DELETE_DATE_KEY, deleteDateInMillis); return builder.endObject(); } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/NodeAllocationResult.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/NodeAllocationResult.java index ffb9351f57637..153fc2cbe3e7d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/NodeAllocationResult.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/NodeAllocationResult.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -289,7 +290,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (hasMatchingSyncId()) { builder.field("matching_sync_id", true); } else { - builder.byteSizeField("matching_size_in_bytes", "matching_size", matchingBytes); + builder.humanReadableField("matching_size_in_bytes", "matching_size", new ByteSizeValue(matchingBytes)); } } if (storeException != null) { diff --git a/server/src/main/java/org/elasticsearch/common/FieldMemoryStats.java b/server/src/main/java/org/elasticsearch/common/FieldMemoryStats.java index a09895fdbedce..7f63784d92a84 100644 --- a/server/src/main/java/org/elasticsearch/common/FieldMemoryStats.java +++ b/server/src/main/java/org/elasticsearch/common/FieldMemoryStats.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentBuilder; import java.io.IOException; @@ -85,7 +86,7 @@ public void toXContent(XContentBuilder builder, String key, String rawKey, Strin builder.startObject(key); for (ObjectLongCursor entry : stats) { builder.startObject(entry.key); - builder.byteSizeField(rawKey, readableKey, entry.value); + builder.humanReadableField(rawKey, readableKey, new ByteSizeValue(entry.value)); builder.endObject(); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java b/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java index a02733e551e2d..86b56f29e69be 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilder.java @@ -19,12 +19,7 @@ package org.elasticsearch.common.xcontent; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.CollectionUtils; -import org.joda.time.DateTimeZone; -import org.joda.time.ReadableInstant; -import org.joda.time.format.DateTimeFormatter; -import org.joda.time.format.ISODateTimeFormat; import java.io.ByteArrayOutputStream; import java.io.Closeable; @@ -38,12 +33,14 @@ import java.util.Calendar; import java.util.Collections; import java.util.Date; +import java.util.GregorianCalendar; import java.util.HashMap; import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.ServiceLoader; import java.util.Set; +import java.util.function.Function; /** * A utility to build XContent (ie json). @@ -81,16 +78,15 @@ public static XContentBuilder builder(XContent xContent, Set includes, S return new XContentBuilder(xContent, new ByteArrayOutputStream(), includes, excludes); } - public static final DateTimeFormatter DEFAULT_DATE_PRINTER = ISODateTimeFormat.dateTime().withZone(DateTimeZone.UTC); - private static final Map, Writer> WRITERS; private static final Map, HumanReadableTransformer> HUMAN_READABLE_TRANSFORMERS; + private static final Map, Function> DATE_TRANSFORMERS; static { Map, Writer> writers = new HashMap<>(); writers.put(Boolean.class, (b, v) -> b.value((Boolean) v)); writers.put(Byte.class, (b, v) -> b.value((Byte) v)); writers.put(byte[].class, (b, v) -> b.value((byte[]) v)); - writers.put(Date.class, (b, v) -> b.value((Date) v)); + writers.put(Date.class, XContentBuilder::timeValue); writers.put(Double.class, (b, v) -> b.value((Double) v)); writers.put(double[].class, (b, v) -> b.values((double[]) v)); writers.put(Float.class, (b, v) -> b.value((Float) v)); @@ -106,26 +102,37 @@ public static XContentBuilder builder(XContent xContent, Set includes, S writers.put(Locale.class, (b, v) -> b.value(v.toString())); writers.put(Class.class, (b, v) -> b.value(v.toString())); writers.put(ZonedDateTime.class, (b, v) -> b.value(v.toString())); + writers.put(Calendar.class, XContentBuilder::timeValue); + writers.put(GregorianCalendar.class, XContentBuilder::timeValue); Map, HumanReadableTransformer> humanReadableTransformer = new HashMap<>(); + Map, Function> dateTransformers = new HashMap<>(); + + // treat strings as already converted + dateTransformers.put(String.class, Function.identity()); // Load pluggable extensions for (XContentBuilderExtension service : ServiceLoader.load(XContentBuilderExtension.class)) { Map, Writer> addlWriters = service.getXContentWriters(); Map, HumanReadableTransformer> addlTransformers = service.getXContentHumanReadableTransformers(); + Map, Function> addlDateTransformers = service.getDateTransformers(); addlWriters.forEach((key, value) -> Objects.requireNonNull(value, "invalid null xcontent writer for class " + key)); addlTransformers.forEach((key, value) -> Objects.requireNonNull(value, "invalid null xcontent transformer for human readable class " + key)); + dateTransformers.forEach((key, value) -> Objects.requireNonNull(value, + "invalid null xcontent date transformer for class " + key)); writers.putAll(addlWriters); humanReadableTransformer.putAll(addlTransformers); + dateTransformers.putAll(addlDateTransformers); } WRITERS = Collections.unmodifiableMap(writers); HUMAN_READABLE_TRANSFORMERS = Collections.unmodifiableMap(humanReadableTransformer); + DATE_TRANSFORMERS = Collections.unmodifiableMap(dateTransformers); } @FunctionalInterface @@ -610,15 +617,6 @@ public XContentBuilder value(byte[] value, int offset, int length) throws IOExce return this; } - /** - * Writes the binary content of the given byte array as UTF-8 bytes. - * - * Use {@link XContentParser#charBuffer()} to read the value back - */ - public XContentBuilder utf8Field(String name, byte[] bytes, int offset, int length) throws IOException { - return field(name).utf8Value(bytes, offset, length); - } - /** * Writes the binary content of the given byte array as UTF-8 bytes. * @@ -634,63 +632,49 @@ public XContentBuilder utf8Value(byte[] bytes, int offset, int length) throws IO // Date ////////////////////////////////// - public XContentBuilder field(String name, ReadableInstant value) throws IOException { - return field(name).value(value); - } - - public XContentBuilder field(String name, ReadableInstant value, DateTimeFormatter formatter) throws IOException { - return field(name).value(value, formatter); - } - - public XContentBuilder value(ReadableInstant value) throws IOException { - return value(value, DEFAULT_DATE_PRINTER); - } - - public XContentBuilder value(ReadableInstant value, DateTimeFormatter formatter) throws IOException { - if (value == null) { - return nullValue(); - } - ensureFormatterNotNull(formatter); - return value(formatter.print(value)); - } - - public XContentBuilder field(String name, Date value) throws IOException { - return field(name).value(value); - } - - public XContentBuilder field(String name, Date value, DateTimeFormatter formatter) throws IOException { - return field(name).value(value, formatter); - } - - public XContentBuilder value(Date value) throws IOException { - return value(value, DEFAULT_DATE_PRINTER); - } + /** + * Write a time-based field and value, if the passed timeValue is null a + * null value is written, otherwise a date transformers lookup is performed. - public XContentBuilder value(Date value, DateTimeFormatter formatter) throws IOException { - if (value == null) { - return nullValue(); - } - return value(formatter, value.getTime()); + * @throws IllegalArgumentException if there is no transformers for the type of object + */ + public XContentBuilder timeField(String name, Object timeValue) throws IOException { + return field(name).timeValue(timeValue); } - public XContentBuilder dateField(String name, String readableName, long value) throws IOException { + /** + * If the {@code humanReadable} flag is set, writes both a formatted and + * unformatted version of the time value using the date transformer for the + * {@link Long} class. + */ + public XContentBuilder timeField(String name, String readableName, long value) throws IOException { if (humanReadable) { - field(readableName).value(DEFAULT_DATE_PRINTER, value); + Function longTransformer = DATE_TRANSFORMERS.get(Long.class); + if (longTransformer == null) { + throw new IllegalArgumentException("cannot write time value xcontent for unknown value of type Long"); + } + field(readableName).value(longTransformer.apply(value)); } field(name, value); return this; } - XContentBuilder value(Calendar value) throws IOException { - if (value == null) { + /** + * Write a time-based value, if the value is null a null value is written, + * otherwise a date transformers lookup is performed. + + * @throws IllegalArgumentException if there is no transformers for the type of object + */ + public XContentBuilder timeValue(Object timeValue) throws IOException { + if (timeValue == null) { return nullValue(); + } else { + Function transformer = DATE_TRANSFORMERS.get(timeValue.getClass()); + if (transformer == null) { + throw new IllegalArgumentException("cannot write time value xcontent for unknown value of type " + timeValue.getClass()); + } + return value(transformer.apply(timeValue)); } - return value(DEFAULT_DATE_PRINTER, value.getTimeInMillis()); - } - - XContentBuilder value(DateTimeFormatter formatter, long value) throws IOException { - ensureFormatterNotNull(formatter); - return value(formatter.print(value)); } //////////////////////////////////////////////////////////////////////////// @@ -761,10 +745,6 @@ private void unknownValue(Object value, boolean ensureNoSelfReferences) throws I value((Iterable) value, ensureNoSelfReferences); } else if (value instanceof Object[]) { values((Object[]) value, ensureNoSelfReferences); - } else if (value instanceof Calendar) { - value((Calendar) value); - } else if (value instanceof ReadableInstant) { - value((ReadableInstant) value); } else if (value instanceof ToXContent) { value((ToXContent) value); } else if (value instanceof Enum) { @@ -895,14 +875,6 @@ public XContentBuilder percentageField(String rawFieldName, String readableField return this; } - public XContentBuilder byteSizeField(String rawFieldName, String readableFieldName, long rawSize) throws IOException { - if (humanReadable) { - field(readableFieldName, new ByteSizeValue(rawSize).toString()); - } - field(rawFieldName, rawSize); - return this; - } - //////////////////////////////////////////////////////////////////////////// // Raw fields ////////////////////////////////// @@ -960,10 +932,6 @@ static void ensureNameNotNull(String name) { ensureNotNull(name, "Field name cannot be null"); } - static void ensureFormatterNotNull(DateTimeFormatter formatter) { - ensureNotNull(formatter, "DateTimeFormatter cannot be null"); - } - static void ensureNotNull(Object value, String message) { if (value == null) { throw new IllegalArgumentException(message); diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java b/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java index 610be4585eb9c..efa7a49571000 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/XContentBuilderExtension.java @@ -20,6 +20,7 @@ package org.elasticsearch.common.xcontent; import java.util.Map; +import java.util.function.Function; /** * This interface provides a way for non-JDK classes to plug in a way to serialize to xcontent. @@ -61,4 +62,20 @@ public interface XContentBuilderExtension { * @return a map of class name to transformer used to retrieve raw value */ Map, XContentBuilder.HumanReadableTransformer> getXContentHumanReadableTransformers(); + + /** + * Used for plugging a transformer for a date or time type object into a String (or other + * encodable object). + * + * For example: + * + *
+     * {@code
+     *     final DateTimeFormatter datePrinter = ISODateTimeFormat.dateTime().withZone(DateTimeZone.UTC);
+     *     Map, Function> transformers = new HashMap<>();
+     *     transformers.put(Date.class, d -> datePrinter.print(((Date) d).getTime()));
+     * }
+     * 
+ */ + Map, Function> getDateTransformers(); } diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/XContentElasticsearchExtension.java b/server/src/main/java/org/elasticsearch/common/xcontent/XContentElasticsearchExtension.java index 1c852c68960a7..42089d2392395 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/XContentElasticsearchExtension.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/XContentElasticsearchExtension.java @@ -23,13 +23,23 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.joda.time.DateTime; import org.joda.time.DateTimeZone; +import org.joda.time.Instant; +import org.joda.time.MutableDateTime; +import org.joda.time.ReadableInstant; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; import org.joda.time.tz.CachedDateTimeZone; import org.joda.time.tz.FixedDateTimeZone; +import java.util.Calendar; +import java.util.Date; +import java.util.GregorianCalendar; import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.function.Function; /** * SPI extensions for Elasticsearch-specific classes (like the Lucene or Joda @@ -38,6 +48,8 @@ */ public class XContentElasticsearchExtension implements XContentBuilderExtension { + public static final DateTimeFormatter DEFAULT_DATE_PRINTER = ISODateTimeFormat.dateTime().withZone(DateTimeZone.UTC); + @Override public Map, XContentBuilder.Writer> getXContentWriters() { Map, XContentBuilder.Writer> writers = new HashMap<>(); @@ -47,6 +59,8 @@ public Map, XContentBuilder.Writer> getXContentWriters() { writers.put(DateTimeZone.class, (b, v) -> b.value(Objects.toString(v))); writers.put(CachedDateTimeZone.class, (b, v) -> b.value(Objects.toString(v))); writers.put(FixedDateTimeZone.class, (b, v) -> b.value(Objects.toString(v))); + writers.put(MutableDateTime.class, XContentBuilder::timeValue); + writers.put(DateTime.class, XContentBuilder::timeValue); writers.put(BytesReference.class, (b, v) -> { if (v == null) { @@ -75,4 +89,18 @@ public Map, XContentBuilder.HumanReadableTransformer> getXContentHumanR transformers.put(ByteSizeValue.class, v -> ((ByteSizeValue) v).getBytes()); return transformers; } + + @Override + public Map, Function> getDateTransformers() { + Map, Function> transformers = new HashMap<>(); + transformers.put(Date.class, d -> DEFAULT_DATE_PRINTER.print(((Date) d).getTime())); + transformers.put(DateTime.class, d -> DEFAULT_DATE_PRINTER.print((DateTime) d)); + transformers.put(MutableDateTime.class, d -> DEFAULT_DATE_PRINTER.print((MutableDateTime) d)); + transformers.put(ReadableInstant.class, d -> DEFAULT_DATE_PRINTER.print((ReadableInstant) d)); + transformers.put(Long.class, d -> DEFAULT_DATE_PRINTER.print((long) d)); + transformers.put(Calendar.class, d -> DEFAULT_DATE_PRINTER.print(((Calendar) d).getTimeInMillis())); + transformers.put(GregorianCalendar.class, d -> DEFAULT_DATE_PRINTER.print(((Calendar) d).getTimeInMillis())); + transformers.put(Instant.class, d -> DEFAULT_DATE_PRINTER.print((Instant) d)); + return transformers; + } } diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java b/server/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java index 06cc10713bec5..4935b83c45bd2 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/XContentParser.java @@ -235,7 +235,7 @@ enum NumberType { * as well as via their String variants of the separated value methods. * Note: Do not use this method to read values written with: *
    - *
  • {@link XContentBuilder#utf8Field(String, byte[], int, int)}
  • + *
  • {@link XContentBuilder#utf8Value(byte[], int, int)}
  • *
* * these methods write UTF-8 encoded strings and must be read through: diff --git a/server/src/main/java/org/elasticsearch/http/HttpInfo.java b/server/src/main/java/org/elasticsearch/http/HttpInfo.java index 706211af6e484..4e944a0f7fac8 100644 --- a/server/src/main/java/org/elasticsearch/http/HttpInfo.java +++ b/server/src/main/java/org/elasticsearch/http/HttpInfo.java @@ -63,7 +63,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(Fields.HTTP); builder.array(Fields.BOUND_ADDRESS, (Object[]) address.boundAddresses()); builder.field(Fields.PUBLISH_ADDRESS, address.publishAddress().toString()); - builder.byteSizeField(Fields.MAX_CONTENT_LENGTH_IN_BYTES, Fields.MAX_CONTENT_LENGTH, maxContentLength); + builder.humanReadableField(Fields.MAX_CONTENT_LENGTH_IN_BYTES, Fields.MAX_CONTENT_LENGTH, maxContentLength()); builder.endObject(); return builder; } diff --git a/server/src/main/java/org/elasticsearch/index/cache/query/QueryCacheStats.java b/server/src/main/java/org/elasticsearch/index/cache/query/QueryCacheStats.java index 8af6f74e5b8fc..73cc3774055a1 100644 --- a/server/src/main/java/org/elasticsearch/index/cache/query/QueryCacheStats.java +++ b/server/src/main/java/org/elasticsearch/index/cache/query/QueryCacheStats.java @@ -128,7 +128,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { builder.startObject(Fields.QUERY_CACHE); - builder.byteSizeField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, ramBytesUsed); + builder.humanReadableField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, getMemorySize()); builder.field(Fields.TOTAL_COUNT, getTotalCount()); builder.field(Fields.HIT_COUNT, getHitCount()); builder.field(Fields.MISS_COUNT, getMissCount()); diff --git a/server/src/main/java/org/elasticsearch/index/cache/request/RequestCacheStats.java b/server/src/main/java/org/elasticsearch/index/cache/request/RequestCacheStats.java index 725f82b705ba7..9605073eeb316 100644 --- a/server/src/main/java/org/elasticsearch/index/cache/request/RequestCacheStats.java +++ b/server/src/main/java/org/elasticsearch/index/cache/request/RequestCacheStats.java @@ -92,7 +92,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(Fields.REQUEST_CACHE_STATS); - builder.byteSizeField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, memorySize); + builder.humanReadableField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, getMemorySize()); builder.field(Fields.EVICTIONS, getEvictions()); builder.field(Fields.HIT_COUNT, getHitCount()); builder.field(Fields.MISS_COUNT, getMissCount()); diff --git a/server/src/main/java/org/elasticsearch/index/engine/SegmentsStats.java b/server/src/main/java/org/elasticsearch/index/engine/SegmentsStats.java index c99b9dacd31b7..72a938e725eee 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/SegmentsStats.java +++ b/server/src/main/java/org/elasticsearch/index/engine/SegmentsStats.java @@ -291,22 +291,22 @@ public long getMaxUnsafeAutoIdTimestamp() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(Fields.SEGMENTS); builder.field(Fields.COUNT, count); - builder.byteSizeField(Fields.MEMORY_IN_BYTES, Fields.MEMORY, memoryInBytes); - builder.byteSizeField(Fields.TERMS_MEMORY_IN_BYTES, Fields.TERMS_MEMORY, termsMemoryInBytes); - builder.byteSizeField(Fields.STORED_FIELDS_MEMORY_IN_BYTES, Fields.STORED_FIELDS_MEMORY, storedFieldsMemoryInBytes); - builder.byteSizeField(Fields.TERM_VECTORS_MEMORY_IN_BYTES, Fields.TERM_VECTORS_MEMORY, termVectorsMemoryInBytes); - builder.byteSizeField(Fields.NORMS_MEMORY_IN_BYTES, Fields.NORMS_MEMORY, normsMemoryInBytes); - builder.byteSizeField(Fields.POINTS_MEMORY_IN_BYTES, Fields.POINTS_MEMORY, pointsMemoryInBytes); - builder.byteSizeField(Fields.DOC_VALUES_MEMORY_IN_BYTES, Fields.DOC_VALUES_MEMORY, docValuesMemoryInBytes); - builder.byteSizeField(Fields.INDEX_WRITER_MEMORY_IN_BYTES, Fields.INDEX_WRITER_MEMORY, indexWriterMemoryInBytes); - builder.byteSizeField(Fields.VERSION_MAP_MEMORY_IN_BYTES, Fields.VERSION_MAP_MEMORY, versionMapMemoryInBytes); - builder.byteSizeField(Fields.FIXED_BIT_SET_MEMORY_IN_BYTES, Fields.FIXED_BIT_SET, bitsetMemoryInBytes); + builder.humanReadableField(Fields.MEMORY_IN_BYTES, Fields.MEMORY, getMemory()); + builder.humanReadableField(Fields.TERMS_MEMORY_IN_BYTES, Fields.TERMS_MEMORY, getTermsMemory()); + builder.humanReadableField(Fields.STORED_FIELDS_MEMORY_IN_BYTES, Fields.STORED_FIELDS_MEMORY, getStoredFieldsMemory()); + builder.humanReadableField(Fields.TERM_VECTORS_MEMORY_IN_BYTES, Fields.TERM_VECTORS_MEMORY, getTermVectorsMemory()); + builder.humanReadableField(Fields.NORMS_MEMORY_IN_BYTES, Fields.NORMS_MEMORY, getNormsMemory()); + builder.humanReadableField(Fields.POINTS_MEMORY_IN_BYTES, Fields.POINTS_MEMORY, getPointsMemory()); + builder.humanReadableField(Fields.DOC_VALUES_MEMORY_IN_BYTES, Fields.DOC_VALUES_MEMORY, getDocValuesMemory()); + builder.humanReadableField(Fields.INDEX_WRITER_MEMORY_IN_BYTES, Fields.INDEX_WRITER_MEMORY, getIndexWriterMemory()); + builder.humanReadableField(Fields.VERSION_MAP_MEMORY_IN_BYTES, Fields.VERSION_MAP_MEMORY, getVersionMapMemory()); + builder.humanReadableField(Fields.FIXED_BIT_SET_MEMORY_IN_BYTES, Fields.FIXED_BIT_SET, getBitsetMemory()); builder.field(Fields.MAX_UNSAFE_AUTO_ID_TIMESTAMP, maxUnsafeAutoIdTimestamp); builder.startObject(Fields.FILE_SIZES); for (Iterator> it = fileSizes.iterator(); it.hasNext();) { ObjectObjectCursor entry = it.next(); builder.startObject(entry.key); - builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, entry.value); + builder.humanReadableField(Fields.SIZE_IN_BYTES, Fields.SIZE, new ByteSizeValue(entry.value)); builder.field(Fields.DESCRIPTION, fileDescriptions.getOrDefault(entry.key, "Others")); builder.endObject(); } diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/FieldDataStats.java b/server/src/main/java/org/elasticsearch/index/fielddata/FieldDataStats.java index 729184c221d0b..363313ba3df44 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/FieldDataStats.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/FieldDataStats.java @@ -99,7 +99,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(FIELDDATA); - builder.byteSizeField(MEMORY_SIZE_IN_BYTES, MEMORY_SIZE, memorySize); + builder.humanReadableField(MEMORY_SIZE_IN_BYTES, MEMORY_SIZE, getMemorySize()); builder.field(EVICTIONS, getEvictions()); if (fields != null) { fields.toXContent(builder, FIELDS, MEMORY_SIZE_IN_BYTES, MEMORY_SIZE); diff --git a/server/src/main/java/org/elasticsearch/index/merge/MergeStats.java b/server/src/main/java/org/elasticsearch/index/merge/MergeStats.java index 20329cac98ba0..603d5c304b634 100644 --- a/server/src/main/java/org/elasticsearch/index/merge/MergeStats.java +++ b/server/src/main/java/org/elasticsearch/index/merge/MergeStats.java @@ -187,14 +187,17 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(Fields.MERGES); builder.field(Fields.CURRENT, current); builder.field(Fields.CURRENT_DOCS, currentNumDocs); - builder.byteSizeField(Fields.CURRENT_SIZE_IN_BYTES, Fields.CURRENT_SIZE, currentSizeInBytes); + builder.humanReadableField(Fields.CURRENT_SIZE_IN_BYTES, Fields.CURRENT_SIZE, getCurrentSize()); builder.field(Fields.TOTAL, total); builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, getTotalTime()); builder.field(Fields.TOTAL_DOCS, totalNumDocs); - builder.byteSizeField(Fields.TOTAL_SIZE_IN_BYTES, Fields.TOTAL_SIZE, totalSizeInBytes); + builder.humanReadableField(Fields.TOTAL_SIZE_IN_BYTES, Fields.TOTAL_SIZE, getTotalSize()); builder.humanReadableField(Fields.TOTAL_STOPPED_TIME_IN_MILLIS, Fields.TOTAL_STOPPED_TIME, getTotalStoppedTime()); builder.humanReadableField(Fields.TOTAL_THROTTLED_TIME_IN_MILLIS, Fields.TOTAL_THROTTLED_TIME, getTotalThrottledTime()); - builder.byteSizeField(Fields.TOTAL_THROTTLE_BYTES_PER_SEC_IN_BYTES, Fields.TOTAL_THROTTLE_BYTES_PER_SEC, totalBytesPerSecAutoThrottle); + if (builder.humanReadable() && totalBytesPerSecAutoThrottle != -1) { + builder.field(Fields.TOTAL_THROTTLE_BYTES_PER_SEC).value(new ByteSizeValue(totalBytesPerSecAutoThrottle).toString()); + } + builder.field(Fields.TOTAL_THROTTLE_BYTES_PER_SEC_IN_BYTES, totalBytesPerSecAutoThrottle); builder.endObject(); return builder; } diff --git a/server/src/main/java/org/elasticsearch/index/store/StoreStats.java b/server/src/main/java/org/elasticsearch/index/store/StoreStats.java index b3f9f32905bbe..feabf8d19b02f 100644 --- a/server/src/main/java/org/elasticsearch/index/store/StoreStats.java +++ b/server/src/main/java/org/elasticsearch/index/store/StoreStats.java @@ -85,7 +85,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(Fields.STORE); - builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, sizeInBytes); + builder.humanReadableField(Fields.SIZE_IN_BYTES, Fields.SIZE, size()); builder.endObject(); return builder; } diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogStats.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogStats.java index c90e79eeba371..8dd5ddcee3be3 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogStats.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogStats.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -100,9 +101,9 @@ public int getUncommittedOperations() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject("translog"); builder.field("operations", numberOfOperations); - builder.byteSizeField("size_in_bytes", "size", translogSizeInBytes); + builder.humanReadableField("size_in_bytes", "size", new ByteSizeValue(translogSizeInBytes)); builder.field("uncommitted_operations", uncommittedOperations); - builder.byteSizeField("uncommitted_size_in_bytes", "uncommitted_size", uncommittedSizeInBytes); + builder.humanReadableField("uncommitted_size_in_bytes", "uncommitted_size", new ByteSizeValue(uncommittedSizeInBytes)); builder.field("earliest_last_modified_age", earliestLastModifiedAge); builder.endObject(); return builder; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index 3eb45318d7a19..0d57d9506628f 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.ToXContentObject; @@ -262,9 +263,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(Fields.TYPE, recoverySource.getType()); builder.field(Fields.STAGE, stage.toString()); builder.field(Fields.PRIMARY, primary); - builder.dateField(Fields.START_TIME_IN_MILLIS, Fields.START_TIME, timer.startTime); + builder.timeField(Fields.START_TIME_IN_MILLIS, Fields.START_TIME, timer.startTime); if (timer.stopTime > 0) { - builder.dateField(Fields.STOP_TIME_IN_MILLIS, Fields.STOP_TIME, timer.stopTime); + builder.timeField(Fields.STOP_TIME_IN_MILLIS, Fields.STOP_TIME, timer.stopTime); } builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, new TimeValue(timer.time())); @@ -634,9 +635,9 @@ public void writeTo(StreamOutput out) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); builder.field(Fields.NAME, name); - builder.byteSizeField(Fields.LENGTH_IN_BYTES, Fields.LENGTH, length); + builder.humanReadableField(Fields.LENGTH_IN_BYTES, Fields.LENGTH, new ByteSizeValue(length)); builder.field(Fields.REUSED, reused); - builder.byteSizeField(Fields.RECOVERED_IN_BYTES, Fields.RECOVERED, recovered); + builder.humanReadableField(Fields.RECOVERED_IN_BYTES, Fields.RECOVERED, new ByteSizeValue(recovered)); builder.endObject(); return builder; } @@ -905,9 +906,9 @@ public synchronized void writeTo(StreamOutput out) throws IOException { public synchronized XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { // stream size first, as it matters more and the files section can be long builder.startObject(Fields.SIZE); - builder.byteSizeField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, totalBytes()); - builder.byteSizeField(Fields.REUSED_IN_BYTES, Fields.REUSED, reusedBytes()); - builder.byteSizeField(Fields.RECOVERED_IN_BYTES, Fields.RECOVERED, recoveredBytes()); + builder.humanReadableField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, new ByteSizeValue(totalBytes())); + builder.humanReadableField(Fields.REUSED_IN_BYTES, Fields.REUSED, new ByteSizeValue(reusedBytes())); + builder.humanReadableField(Fields.RECOVERED_IN_BYTES, Fields.RECOVERED, new ByteSizeValue(recoveredBytes())); builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", recoveredBytesPercent())); builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/monitor/fs/FsInfo.java b/server/src/main/java/org/elasticsearch/monitor/fs/FsInfo.java index c8bdaad3f1f6c..fe0c8a9683832 100644 --- a/server/src/main/java/org/elasticsearch/monitor/fs/FsInfo.java +++ b/server/src/main/java/org/elasticsearch/monitor/fs/FsInfo.java @@ -165,13 +165,13 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } if (total != -1) { - builder.byteSizeField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, total); + builder.humanReadableField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, getTotal()); } if (free != -1) { - builder.byteSizeField(Fields.FREE_IN_BYTES, Fields.FREE, free); + builder.humanReadableField(Fields.FREE_IN_BYTES, Fields.FREE, getFree()); } if (available != -1) { - builder.byteSizeField(Fields.AVAILABLE_IN_BYTES, Fields.AVAILABLE, available); + builder.humanReadableField(Fields.AVAILABLE_IN_BYTES, Fields.AVAILABLE, getAvailable()); } builder.endObject(); @@ -530,8 +530,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(Fields.LEAST_ESTIMATE); { builder.field(Fields.PATH, leastDiskEstimate.getPath()); - builder.byteSizeField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, leastDiskEstimate.getTotalBytes()); - builder.byteSizeField(Fields.AVAILABLE_IN_BYTES, Fields.AVAILABLE, leastDiskEstimate.getFreeBytes()); + builder.humanReadableField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, new ByteSizeValue(leastDiskEstimate.getTotalBytes())); + builder.humanReadableField(Fields.AVAILABLE_IN_BYTES, Fields.AVAILABLE, + new ByteSizeValue(leastDiskEstimate.getFreeBytes())); builder.field(Fields.USAGE_PERCENTAGE, leastDiskEstimate.getUsedDiskAsPercentage()); } builder.endObject(); @@ -541,8 +542,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(Fields.MOST_ESTIMATE); { builder.field(Fields.PATH, mostDiskEstimate.getPath()); - builder.byteSizeField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, mostDiskEstimate.getTotalBytes()); - builder.byteSizeField(Fields.AVAILABLE_IN_BYTES, Fields.AVAILABLE, mostDiskEstimate.getFreeBytes()); + builder.humanReadableField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, new ByteSizeValue(mostDiskEstimate.getTotalBytes())); + builder.humanReadableField(Fields.AVAILABLE_IN_BYTES, Fields.AVAILABLE, new ByteSizeValue(mostDiskEstimate.getFreeBytes())); builder.field(Fields.USAGE_PERCENTAGE, mostDiskEstimate.getUsedDiskAsPercentage()); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java index 87e15b910f6ad..f24acc9c034e9 100644 --- a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java +++ b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java @@ -436,14 +436,14 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(Fields.VM_NAME, vmName); builder.field(Fields.VM_VERSION, vmVersion); builder.field(Fields.VM_VENDOR, vmVendor); - builder.dateField(Fields.START_TIME_IN_MILLIS, Fields.START_TIME, startTime); + builder.timeField(Fields.START_TIME_IN_MILLIS, Fields.START_TIME, startTime); builder.startObject(Fields.MEM); - builder.byteSizeField(Fields.HEAP_INIT_IN_BYTES, Fields.HEAP_INIT, mem.heapInit); - builder.byteSizeField(Fields.HEAP_MAX_IN_BYTES, Fields.HEAP_MAX, mem.heapMax); - builder.byteSizeField(Fields.NON_HEAP_INIT_IN_BYTES, Fields.NON_HEAP_INIT, mem.nonHeapInit); - builder.byteSizeField(Fields.NON_HEAP_MAX_IN_BYTES, Fields.NON_HEAP_MAX, mem.nonHeapMax); - builder.byteSizeField(Fields.DIRECT_MAX_IN_BYTES, Fields.DIRECT_MAX, mem.directMemoryMax); + builder.humanReadableField(Fields.HEAP_INIT_IN_BYTES, Fields.HEAP_INIT, new ByteSizeValue(mem.heapInit)); + builder.humanReadableField(Fields.HEAP_MAX_IN_BYTES, Fields.HEAP_MAX, new ByteSizeValue(mem.heapMax)); + builder.humanReadableField(Fields.NON_HEAP_INIT_IN_BYTES, Fields.NON_HEAP_INIT, new ByteSizeValue(mem.nonHeapInit)); + builder.humanReadableField(Fields.NON_HEAP_MAX_IN_BYTES, Fields.NON_HEAP_MAX, new ByteSizeValue(mem.nonHeapMax)); + builder.humanReadableField(Fields.DIRECT_MAX_IN_BYTES, Fields.DIRECT_MAX, new ByteSizeValue(mem.directMemoryMax)); builder.endObject(); builder.array(Fields.GC_COLLECTORS, gcCollectors); diff --git a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java index b548afadd3d88..e9d3adba68255 100644 --- a/server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java +++ b/server/src/main/java/org/elasticsearch/monitor/jvm/JvmStats.java @@ -194,23 +194,23 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(Fields.MEM); - builder.byteSizeField(Fields.HEAP_USED_IN_BYTES, Fields.HEAP_USED, mem.heapUsed); + builder.humanReadableField(Fields.HEAP_USED_IN_BYTES, Fields.HEAP_USED, new ByteSizeValue(mem.heapUsed)); if (mem.getHeapUsedPercent() >= 0) { builder.field(Fields.HEAP_USED_PERCENT, mem.getHeapUsedPercent()); } - builder.byteSizeField(Fields.HEAP_COMMITTED_IN_BYTES, Fields.HEAP_COMMITTED, mem.heapCommitted); - builder.byteSizeField(Fields.HEAP_MAX_IN_BYTES, Fields.HEAP_MAX, mem.heapMax); - builder.byteSizeField(Fields.NON_HEAP_USED_IN_BYTES, Fields.NON_HEAP_USED, mem.nonHeapUsed); - builder.byteSizeField(Fields.NON_HEAP_COMMITTED_IN_BYTES, Fields.NON_HEAP_COMMITTED, mem.nonHeapCommitted); + builder.humanReadableField(Fields.HEAP_COMMITTED_IN_BYTES, Fields.HEAP_COMMITTED, new ByteSizeValue(mem.heapCommitted)); + builder.humanReadableField(Fields.HEAP_MAX_IN_BYTES, Fields.HEAP_MAX, new ByteSizeValue(mem.heapMax)); + builder.humanReadableField(Fields.NON_HEAP_USED_IN_BYTES, Fields.NON_HEAP_USED, new ByteSizeValue(mem.nonHeapUsed)); + builder.humanReadableField(Fields.NON_HEAP_COMMITTED_IN_BYTES, Fields.NON_HEAP_COMMITTED, new ByteSizeValue(mem.nonHeapCommitted)); builder.startObject(Fields.POOLS); for (MemoryPool pool : mem) { builder.startObject(pool.getName()); - builder.byteSizeField(Fields.USED_IN_BYTES, Fields.USED, pool.used); - builder.byteSizeField(Fields.MAX_IN_BYTES, Fields.MAX, pool.max); + builder.humanReadableField(Fields.USED_IN_BYTES, Fields.USED, new ByteSizeValue(pool.used)); + builder.humanReadableField(Fields.MAX_IN_BYTES, Fields.MAX, new ByteSizeValue(pool.max)); - builder.byteSizeField(Fields.PEAK_USED_IN_BYTES, Fields.PEAK_USED, pool.peakUsed); - builder.byteSizeField(Fields.PEAK_MAX_IN_BYTES, Fields.PEAK_MAX, pool.peakMax); + builder.humanReadableField(Fields.PEAK_USED_IN_BYTES, Fields.PEAK_USED, new ByteSizeValue(pool.peakUsed)); + builder.humanReadableField(Fields.PEAK_MAX_IN_BYTES, Fields.PEAK_MAX, new ByteSizeValue(pool.peakMax)); builder.endObject(); } @@ -241,8 +241,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws for (BufferPool bufferPool : bufferPools) { builder.startObject(bufferPool.getName()); builder.field(Fields.COUNT, bufferPool.getCount()); - builder.byteSizeField(Fields.USED_IN_BYTES, Fields.USED, bufferPool.used); - builder.byteSizeField(Fields.TOTAL_CAPACITY_IN_BYTES, Fields.TOTAL_CAPACITY, bufferPool.totalCapacity); + builder.humanReadableField(Fields.USED_IN_BYTES, Fields.USED, new ByteSizeValue(bufferPool.used)); + builder.humanReadableField(Fields.TOTAL_CAPACITY_IN_BYTES, Fields.TOTAL_CAPACITY, + new ByteSizeValue(bufferPool.totalCapacity)); builder.endObject(); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java b/server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java index 1d051aac7b0c8..f1cc9d1f3f3fd 100644 --- a/server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java +++ b/server/src/main/java/org/elasticsearch/monitor/process/ProcessStats.java @@ -113,7 +113,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } if (mem != null) { builder.startObject(Fields.MEM); - builder.byteSizeField(Fields.TOTAL_VIRTUAL_IN_BYTES, Fields.TOTAL_VIRTUAL, mem.totalVirtual); + builder.humanReadableField(Fields.TOTAL_VIRTUAL_IN_BYTES, Fields.TOTAL_VIRTUAL, new ByteSizeValue(mem.totalVirtual)); builder.endObject(); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionStats.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionStats.java index ae71c0826ecd3..e06f2a0bce4f7 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionStats.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionStats.java @@ -75,7 +75,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(COMPLETION); - builder.byteSizeField(SIZE_IN_BYTES, SIZE, sizeInBytes); + builder.humanReadableField(SIZE_IN_BYTES, SIZE, getSize()); if (fields != null) { fields.toXContent(builder, FIELDS, SIZE_IN_BYTES, SIZE); } diff --git a/server/src/main/java/org/elasticsearch/tasks/TaskInfo.java b/server/src/main/java/org/elasticsearch/tasks/TaskInfo.java index bc40df2b8f0c4..9027f961ae75b 100644 --- a/server/src/main/java/org/elasticsearch/tasks/TaskInfo.java +++ b/server/src/main/java/org/elasticsearch/tasks/TaskInfo.java @@ -196,7 +196,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (description != null) { builder.field("description", description); } - builder.dateField("start_time_in_millis", "start_time", startTime); + builder.timeField("start_time_in_millis", "start_time", startTime); if (builder.humanReadable()) { builder.field("running_time", new TimeValue(runningTimeNanos, TimeUnit.NANOSECONDS).toString()); } diff --git a/server/src/main/java/org/elasticsearch/transport/TransportStats.java b/server/src/main/java/org/elasticsearch/transport/TransportStats.java index e911d2e7aa771..8f899aab60c3a 100644 --- a/server/src/main/java/org/elasticsearch/transport/TransportStats.java +++ b/server/src/main/java/org/elasticsearch/transport/TransportStats.java @@ -107,9 +107,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(Fields.TRANSPORT); builder.field(Fields.SERVER_OPEN, serverOpen); builder.field(Fields.RX_COUNT, rxCount); - builder.byteSizeField(Fields.RX_SIZE_IN_BYTES, Fields.RX_SIZE, rxSize); + builder.humanReadableField(Fields.RX_SIZE_IN_BYTES, Fields.RX_SIZE, new ByteSizeValue(rxSize)); builder.field(Fields.TX_COUNT, txCount); - builder.byteSizeField(Fields.TX_SIZE_IN_BYTES, Fields.TX_SIZE, txSize); + builder.humanReadableField(Fields.TX_SIZE_IN_BYTES, Fields.TX_SIZE, new ByteSizeValue(txSize)); builder.endObject(); return builder; } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexGraveyardTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexGraveyardTests.java index 344b6dc42caed..1b854d17a619e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexGraveyardTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexGraveyardTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentElasticsearchExtension; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.Index; @@ -72,7 +73,8 @@ public void testXContent() throws IOException { if (graveyard.getTombstones().size() > 0) { // check that date properly printed assertThat(Strings.toString(graveyard, false, true), - containsString(XContentBuilder.DEFAULT_DATE_PRINTER.print(graveyard.getTombstones().get(0).getDeleteDateInMillis()))); + containsString(XContentElasticsearchExtension.DEFAULT_DATE_PRINTER + .print(graveyard.getTombstones().get(0).getDeleteDateInMillis()))); } XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder)); parser.nextToken(); // the beginning of the parser diff --git a/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java b/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java index 8f7a177fae720..4efd5f480f810 100644 --- a/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java +++ b/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java @@ -330,11 +330,7 @@ public void testBinaryUTF8() throws Exception { final BytesRef randomBytesRef = new BytesRef(randomBytes()); XContentBuilder builder = builder().startObject(); - if (randomBoolean()) { - builder.utf8Field("utf8", randomBytesRef.bytes, randomBytesRef.offset, randomBytesRef.length); - } else { - builder.field("utf8").utf8Value(randomBytesRef.bytes, randomBytesRef.offset, randomBytesRef.length); - } + builder.field("utf8").utf8Value(randomBytesRef.bytes, randomBytesRef.offset, randomBytesRef.length); builder.endObject(); XContentParser parser = createParser(xcontentType().xContent(), BytesReference.bytes(builder)); @@ -366,81 +362,73 @@ public void testText() throws Exception { } public void testReadableInstant() throws Exception { - assertResult("{'instant':null}", () -> builder().startObject().field("instant", (ReadableInstant) null).endObject()); - assertResult("{'instant':null}", () -> builder().startObject().field("instant").value((ReadableInstant) null).endObject()); + assertResult("{'instant':null}", () -> builder().startObject().timeField("instant", (ReadableInstant) null).endObject()); + assertResult("{'instant':null}", () -> builder().startObject().field("instant").timeValue((ReadableInstant) null).endObject()); final DateTime t1 = new DateTime(2016, 1, 1, 0, 0, DateTimeZone.UTC); String expected = "{'t1':'2016-01-01T00:00:00.000Z'}"; - assertResult(expected, () -> builder().startObject().field("t1", t1).endObject()); - assertResult(expected, () -> builder().startObject().field("t1").value(t1).endObject()); + assertResult(expected, () -> builder().startObject().timeField("t1", t1).endObject()); + assertResult(expected, () -> builder().startObject().field("t1").timeValue(t1).endObject()); final DateTime t2 = new DateTime(2016, 12, 25, 7, 59, 42, 213, DateTimeZone.UTC); expected = "{'t2':'2016-12-25T07:59:42.213Z'}"; - assertResult(expected, () -> builder().startObject().field("t2", t2).endObject()); - assertResult(expected, () -> builder().startObject().field("t2").value(t2).endObject()); + assertResult(expected, () -> builder().startObject().timeField("t2", t2).endObject()); + assertResult(expected, () -> builder().startObject().field("t2").timeValue(t2).endObject()); final DateTimeFormatter formatter = randomFrom(ISODateTimeFormat.basicDate(), ISODateTimeFormat.dateTimeNoMillis()); final DateTime t3 = DateTime.now(); expected = "{'t3':'" + formatter.print(t3) + "'}"; - assertResult(expected, () -> builder().startObject().field("t3", t3, formatter).endObject()); - assertResult(expected, () -> builder().startObject().field("t3").value(t3, formatter).endObject()); + assertResult(expected, () -> builder().startObject().timeField("t3", formatter.print(t3)).endObject()); + assertResult(expected, () -> builder().startObject().field("t3").value(formatter.print(t3)).endObject()); final DateTime t4 = new DateTime(randomDateTimeZone()); expected = "{'t4':'" + formatter.print(t4) + "'}"; - assertResult(expected, () -> builder().startObject().field("t4", t4, formatter).endObject()); - assertResult(expected, () -> builder().startObject().field("t4").value(t4, formatter).endObject()); + assertResult(expected, () -> builder().startObject().timeField("t4", formatter.print(t4)).endObject()); + assertResult(expected, () -> builder().startObject().field("t4").value(formatter.print(t4)).endObject()); long date = Math.abs(randomLong() % (2 * (long) 10e11)); // 1970-01-01T00:00:00Z - 2033-05-18T05:33:20.000+02:00 final DateTime t5 = new DateTime(date, randomDateTimeZone()); - expected = "{'t5':'" + XContentBuilder.DEFAULT_DATE_PRINTER.print(t5) + "'}"; - assertResult(expected, () -> builder().startObject().field("t5", t5).endObject()); - assertResult(expected, () -> builder().startObject().field("t5").value(t5).endObject()); + expected = "{'t5':'" + XContentElasticsearchExtension.DEFAULT_DATE_PRINTER.print(t5) + "'}"; + assertResult(expected, () -> builder().startObject().timeField("t5", t5).endObject()); + assertResult(expected, () -> builder().startObject().field("t5").timeValue(t5).endObject()); expected = "{'t5':'" + formatter.print(t5) + "'}"; - assertResult(expected, () -> builder().startObject().field("t5", t5, formatter).endObject()); - assertResult(expected, () -> builder().startObject().field("t5").value(t5, formatter).endObject()); + assertResult(expected, () -> builder().startObject().timeField("t5", formatter.print(t5)).endObject()); + assertResult(expected, () -> builder().startObject().field("t5").value(formatter.print(t5)).endObject()); Instant i1 = new Instant(1451606400000L); // 2016-01-01T00:00:00.000Z expected = "{'i1':'2016-01-01T00:00:00.000Z'}"; - assertResult(expected, () -> builder().startObject().field("i1", i1).endObject()); - assertResult(expected, () -> builder().startObject().field("i1").value(i1).endObject()); + assertResult(expected, () -> builder().startObject().timeField("i1", i1).endObject()); + assertResult(expected, () -> builder().startObject().field("i1").timeValue(i1).endObject()); Instant i2 = new Instant(1482652782213L); // 2016-12-25T07:59:42.213Z expected = "{'i2':'" + formatter.print(i2) + "'}"; - assertResult(expected, () -> builder().startObject().field("i2", i2, formatter).endObject()); - assertResult(expected, () -> builder().startObject().field("i2").value(i2, formatter).endObject()); - - expectNonNullFormatterException(() -> builder().startObject().field("t3", t3, null).endObject()); - expectNonNullFormatterException(() -> builder().startObject().field("t3").value(t3, null).endObject()); + assertResult(expected, () -> builder().startObject().timeField("i2", formatter.print(i2)).endObject()); + assertResult(expected, () -> builder().startObject().field("i2").value(formatter.print(i2)).endObject()); } public void testDate() throws Exception { - assertResult("{'date':null}", () -> builder().startObject().field("date", (Date) null).endObject()); - assertResult("{'date':null}", () -> builder().startObject().field("date").value((Date) null).endObject()); + assertResult("{'date':null}", () -> builder().startObject().timeField("date", (Date) null).endObject()); + assertResult("{'date':null}", () -> builder().startObject().field("date").timeValue((Date) null).endObject()); final Date d1 = new DateTime(2016, 1, 1, 0, 0, DateTimeZone.UTC).toDate(); - assertResult("{'d1':'2016-01-01T00:00:00.000Z'}", () -> builder().startObject().field("d1", d1).endObject()); - assertResult("{'d1':'2016-01-01T00:00:00.000Z'}", () -> builder().startObject().field("d1").value(d1).endObject()); + assertResult("{'d1':'2016-01-01T00:00:00.000Z'}", () -> builder().startObject().timeField("d1", d1).endObject()); + assertResult("{'d1':'2016-01-01T00:00:00.000Z'}", () -> builder().startObject().field("d1").timeValue(d1).endObject()); final Date d2 = new DateTime(2016, 12, 25, 7, 59, 42, 213, DateTimeZone.UTC).toDate(); - assertResult("{'d2':'2016-12-25T07:59:42.213Z'}", () -> builder().startObject().field("d2", d2).endObject()); - assertResult("{'d2':'2016-12-25T07:59:42.213Z'}", () -> builder().startObject().field("d2").value(d2).endObject()); + assertResult("{'d2':'2016-12-25T07:59:42.213Z'}", () -> builder().startObject().timeField("d2", d2).endObject()); + assertResult("{'d2':'2016-12-25T07:59:42.213Z'}", () -> builder().startObject().field("d2").timeValue(d2).endObject()); final DateTimeFormatter formatter = randomFrom(ISODateTimeFormat.basicDate(), ISODateTimeFormat.dateTimeNoMillis()); final Date d3 = DateTime.now().toDate(); String expected = "{'d3':'" + formatter.print(d3.getTime()) + "'}"; - assertResult(expected, () -> builder().startObject().field("d3", d3, formatter).endObject()); - assertResult(expected, () -> builder().startObject().field("d3").value(d3, formatter).endObject()); - - expectNonNullFormatterException(() -> builder().startObject().field("d3", d3, null).endObject()); - expectNonNullFormatterException(() -> builder().startObject().field("d3").value(d3, null).endObject()); - expectNonNullFormatterException(() -> builder().value(null, 1L)); + assertResult(expected, () -> builder().startObject().field("d3").value(formatter.print(d3.getTime())).endObject()); } public void testDateField() throws Exception { @@ -448,12 +436,12 @@ public void testDateField() throws Exception { assertResult("{'date_in_millis':1451606400000}", () -> builder() .startObject() - .dateField("date_in_millis", "date", d.getTime()) + .timeField("date_in_millis", "date", d.getTime()) .endObject()); assertResult("{'date':'2016-01-01T00:00:00.000Z','date_in_millis':1451606400000}", () -> builder() .humanReadable(true) .startObject - ().dateField("date_in_millis", "date", d.getTime()) + ().timeField("date_in_millis", "date", d.getTime()) .endObject()); } @@ -462,7 +450,7 @@ public void testCalendar() throws Exception { assertResult("{'calendar':'2016-01-01T00:00:00.000Z'}", () -> builder() .startObject() .field("calendar") - .value(calendar) + .timeValue(calendar) .endObject()); } @@ -514,7 +502,7 @@ public void testObjects() throws Exception { final String paths = Constants.WINDOWS ? "{'objects':['a\\\\b\\\\c','d\\\\e']}" : "{'objects':['a/b/c','d/e']}"; objects.put(paths, new Object[]{PathUtils.get("a", "b", "c"), PathUtils.get("d", "e")}); - final DateTimeFormatter formatter = XContentBuilder.DEFAULT_DATE_PRINTER; + final DateTimeFormatter formatter = XContentElasticsearchExtension.DEFAULT_DATE_PRINTER; final Date d1 = new DateTime(2016, 1, 1, 0, 0, DateTimeZone.UTC).toDate(); final Date d2 = new DateTime(2015, 1, 1, 0, 0, DateTimeZone.UTC).toDate(); objects.put("{'objects':['" + formatter.print(d1.getTime()) + "','" + formatter.print(d2.getTime()) + "']}", new Object[]{d1, d2}); @@ -562,7 +550,7 @@ public void testObject() throws Exception { final String path = Constants.WINDOWS ? "{'object':'a\\\\b\\\\c'}" : "{'object':'a/b/c'}"; object.put(path, PathUtils.get("a", "b", "c")); - final DateTimeFormatter formatter = XContentBuilder.DEFAULT_DATE_PRINTER; + final DateTimeFormatter formatter = XContentElasticsearchExtension.DEFAULT_DATE_PRINTER; final Date d1 = new DateTime(2016, 1, 1, 0, 0, DateTimeZone.UTC).toDate(); object.put("{'object':'" + formatter.print(d1.getTime()) + "'}", d1); @@ -846,11 +834,6 @@ public void testEnsureNameNotNull() { assertThat(e.getMessage(), containsString("Field name cannot be null")); } - public void testFormatterNameNotNull() { - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> XContentBuilder.ensureFormatterNotNull(null)); - assertThat(e.getMessage(), containsString("DateTimeFormatter cannot be null")); - } - public void testEnsureNotNull() { IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> XContentBuilder.ensureNotNull(null, "message")); assertThat(e.getMessage(), containsString("message")); diff --git a/server/src/test/java/org/elasticsearch/common/xcontent/builder/XContentBuilderTests.java b/server/src/test/java/org/elasticsearch/common/xcontent/builder/XContentBuilderTests.java index 038d8f73c8ab2..cb666418b6cac 100644 --- a/server/src/test/java/org/elasticsearch/common/xcontent/builder/XContentBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/common/xcontent/builder/XContentBuilderTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentElasticsearchExtension; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentGenerator; import org.elasticsearch.common.xcontent.XContentParser; @@ -176,11 +177,11 @@ public void testByteConversion() throws Exception { public void testDateTypesConversion() throws Exception { Date date = new Date(); - String expectedDate = XContentBuilder.DEFAULT_DATE_PRINTER.print(date.getTime()); + String expectedDate = XContentElasticsearchExtension.DEFAULT_DATE_PRINTER.print(date.getTime()); Calendar calendar = new GregorianCalendar(TimeZone.getTimeZone("UTC"), Locale.ROOT); - String expectedCalendar = XContentBuilder.DEFAULT_DATE_PRINTER.print(calendar.getTimeInMillis()); + String expectedCalendar = XContentElasticsearchExtension.DEFAULT_DATE_PRINTER.print(calendar.getTimeInMillis()); XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); - builder.startObject().field("date", date).endObject(); + builder.startObject().timeField("date", date).endObject(); assertThat(Strings.toString(builder), equalTo("{\"date\":\"" + expectedDate + "\"}")); builder = XContentFactory.contentBuilder(XContentType.JSON); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java index 8b00c42311add..a4a561cfee35f 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramIT.java @@ -97,9 +97,9 @@ private static String format(DateTime date, String pattern) { private IndexRequestBuilder indexDoc(String idx, DateTime date, int value) throws Exception { return client().prepareIndex(idx, "type").setSource(jsonBuilder() .startObject() - .field("date", date) + .timeField("date", date) .field("value", value) - .startArray("dates").value(date).value(date.plusMonths(1).plusDays(1)).endArray() + .startArray("dates").timeValue(date).timeValue(date.plusMonths(1).plusDays(1)).endArray() .endObject()); } @@ -108,8 +108,8 @@ private IndexRequestBuilder indexDoc(int month, int day, int value) throws Excep .startObject() .field("value", value) .field("constant", 1) - .field("date", date(month, day)) - .startArray("dates").value(date(month, day)).value(date(month + 1, day + 1)).endArray() + .timeField("date", date(month, day)) + .startArray("dates").timeValue(date(month, day)).timeValue(date(month + 1, day + 1)).endArray() .endObject()); } @@ -161,26 +161,26 @@ private void getMultiSortDocs(List builders) throws IOExcep .addMapping("type", "date", "type=date").get()); for (int i = 1; i <= 3; i++) { builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 1)).field("l", 1).field("d", i).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 1)).field("l", 1).field("d", i).endObject())); builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 2)).field("l", 2).field("d", i).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 2)).field("l", 2).field("d", i).endObject())); } builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 3)).field("l", 3).field("d", 1).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 3)).field("l", 3).field("d", 1).endObject())); builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 3).plusHours(1)).field("l", 3).field("d", 2).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 3).plusHours(1)).field("l", 3).field("d", 2).endObject())); builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 4)).field("l", 3).field("d", 1).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 4)).field("l", 3).field("d", 1).endObject())); builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 4).plusHours(2)).field("l", 3).field("d", 3).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 4).plusHours(2)).field("l", 3).field("d", 3).endObject())); builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 5)).field("l", 5).field("d", 1).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 5)).field("l", 5).field("d", 1).endObject())); builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 5).plusHours(12)).field("l", 5).field("d", 2).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 5).plusHours(12)).field("l", 5).field("d", 2).endObject())); builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 6)).field("l", 5).field("d", 1).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 6)).field("l", 5).field("d", 1).endObject())); builders.add(client().prepareIndex("sort_idx", "type").setSource( - jsonBuilder().startObject().field("date", date(1, 7)).field("l", 5).field("d", 1).endObject())); + jsonBuilder().startObject().timeField("date", date(1, 7)).field("l", 5).field("d", 1).endObject())); } @Override @@ -968,7 +968,7 @@ public void testSingleValueWithTimeZone() throws Exception { IndexRequestBuilder[] reqs = new IndexRequestBuilder[5]; DateTime date = date("2014-03-11T00:00:00+00:00"); for (int i = 0; i < reqs.length; i++) { - reqs[i] = client().prepareIndex("idx2", "type", "" + i).setSource(jsonBuilder().startObject().field("date", date).endObject()); + reqs[i] = client().prepareIndex("idx2", "type", "" + i).setSource(jsonBuilder().startObject().timeField("date", date).endObject()); date = date.plusHours(1); } indexRandom(true, reqs); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java index 5e56f753274b7..f6ad9b17a4514 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateHistogramOffsetIT.java @@ -67,7 +67,7 @@ public void afterEachTest() throws IOException { private void prepareIndex(DateTime date, int numHours, int stepSizeHours, int idxIdStart) throws IOException, InterruptedException, ExecutionException { IndexRequestBuilder[] reqs = new IndexRequestBuilder[numHours]; for (int i = idxIdStart; i < idxIdStart + reqs.length; i++) { - reqs[i - idxIdStart] = client().prepareIndex("idx2", "type", "" + i).setSource(jsonBuilder().startObject().field("date", date).endObject()); + reqs[i - idxIdStart] = client().prepareIndex("idx2", "type", "" + i).setSource(jsonBuilder().startObject().timeField("date", date).endObject()); date = date.plusHours(stepSizeHours); } indexRandom(true, reqs); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateRangeIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateRangeIT.java index f47e59640073d..98f73b34b5677 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateRangeIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/DateRangeIT.java @@ -65,8 +65,8 @@ private static IndexRequestBuilder indexDoc(int month, int day, int value) throw return client().prepareIndex("idx", "type").setSource(jsonBuilder() .startObject() .field("value", value) - .field("date", date(month, day)) - .startArray("dates").value(date(month, day)).value(date(month + 1, day + 1)).endArray() + .timeField("date", date(month, day)) + .startArray("dates").timeValue(date(month, day)).timeValue(date(month + 1, day + 1)).endArray() .endObject()); } @@ -889,9 +889,9 @@ public void testDontCacheScripts() throws Exception { .get()); indexRandom(true, client().prepareIndex("cache_test_idx", "type", "1") - .setSource(jsonBuilder().startObject().field("date", date(1, 1)).endObject()), + .setSource(jsonBuilder().startObject().timeField("date", date(1, 1)).endObject()), client().prepareIndex("cache_test_idx", "type", "2") - .setSource(jsonBuilder().startObject().field("date", date(2, 1)).endObject())); + .setSource(jsonBuilder().startObject().timeField("date", date(2, 1)).endObject())); // Make sure we are starting with a clear cache assertThat(client().admin().indices().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache() diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/DateDerivativeIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/DateDerivativeIT.java index 34bf83b122b1c..b0f5eece900b1 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/DateDerivativeIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/DateDerivativeIT.java @@ -75,13 +75,13 @@ private static String format(DateTime date, String pattern) { private static IndexRequestBuilder indexDoc(String idx, DateTime date, int value) throws Exception { return client().prepareIndex(idx, "type").setSource( - jsonBuilder().startObject().field("date", date).field("value", value).endObject()); + jsonBuilder().startObject().timeField("date", date).field("value", value).endObject()); } private IndexRequestBuilder indexDoc(int month, int day, int value) throws Exception { return client().prepareIndex("idx", "type").setSource( - jsonBuilder().startObject().field("value", value).field("date", date(month, day)).startArray("dates") - .value(date(month, day)).value(date(month + 1, day + 1)).endArray().endObject()); + jsonBuilder().startObject().field("value", value).timeField("date", date(month, day)).startArray("dates") + .timeValue(date(month, day)).timeValue(date(month + 1, day + 1)).endArray().endObject()); } @Override From 7df66abaf5b483aa44f180d10db8984690cde7dc Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 27 Mar 2018 14:20:06 -0600 Subject: [PATCH 18/22] [TEST] Fix issue with HttpInfo passed invalid parameter HttpInfo is passed the maxContentLength as a parameter, but this value should never be negative. This fixes the test to only pass a positive random value. --- .../org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java b/server/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java index 107ac38400e0d..291a6b59c2980 100644 --- a/server/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java +++ b/server/src/test/java/org/elasticsearch/nodesinfo/NodeInfoStreamingTests.java @@ -137,7 +137,7 @@ private static NodeInfo createNodeInfo() { new TransportAddress[]{buildNewFakeTransportAddress()}, buildNewFakeTransportAddress()); profileAddresses.put("test_address", dummyBoundTransportAddress); TransportInfo transport = randomBoolean() ? null : new TransportInfo(dummyBoundTransportAddress, profileAddresses); - HttpInfo httpInfo = randomBoolean() ? null : new HttpInfo(dummyBoundTransportAddress, randomLong()); + HttpInfo httpInfo = randomBoolean() ? null : new HttpInfo(dummyBoundTransportAddress, randomNonNegativeLong()); PluginsAndModules pluginsAndModules = null; if (randomBoolean()) { From 67a6a76aaddc027f00e6cb520aebb450ab5c344f Mon Sep 17 00:00:00 2001 From: Bart van Oort Date: Tue, 27 Mar 2018 23:29:19 +0200 Subject: [PATCH 19/22] Docs: Update generating test coverage reports (#29255) Old docs said to use maven. That doesn't work. We can't generate the reports right now. --- TESTING.asciidoc | 31 ++++++++++--------------------- 1 file changed, 10 insertions(+), 21 deletions(-) diff --git a/TESTING.asciidoc b/TESTING.asciidoc index 4a432d48181ee..9b719826de4ff 100644 --- a/TESTING.asciidoc +++ b/TESTING.asciidoc @@ -499,32 +499,21 @@ will contain your change. . Push both branches to your remote repository. . Run the tests with `./gradlew check -Dtests.bwc.remote=${remote} -Dtests.bwc.refspec=index_req_bwc_5.x`. -== Coverage analysis +== Test coverage analysis -Tests can be run instrumented with jacoco to produce a coverage report in -`target/site/jacoco/`. +Generating test coverage reports for Elasticsearch is currently not possible through Gradle. +However, it _is_ possible to gain insight in code coverage using IntelliJ's built-in coverage +analysis tool that can measure coverage upon executing specific tests. Eclipse may also be able +to do the same using the EclEmma plugin. -Unit test coverage: - ---------------------------------------------------------------------------- -mvn -Dtests.coverage test jacoco:report ---------------------------------------------------------------------------- - -Integration test coverage: - ---------------------------------------------------------------------------- -mvn -Dtests.coverage -Dskip.unit.tests verify jacoco:report ---------------------------------------------------------------------------- - -Combined (Unit+Integration) coverage: - ---------------------------------------------------------------------------- -mvn -Dtests.coverage verify jacoco:report ---------------------------------------------------------------------------- +Test coverage reporting used to be possible with JaCoCo when Elasticsearch was using Maven +as its build system. Since the switch to Gradle though, this is no longer possible, seeing as +the code currently used to build Elasticsearch does not allow JaCoCo to recognize its tests. +For more information on this, see the discussion in https://github.com/elastic/elasticsearch/issues/28867[issue #28867]. == Launching and debugging from an IDE -If you want to run elasticsearch from your IDE, the `./gradlew run` task +If you want to run Elasticsearch from your IDE, the `./gradlew run` task supports a remote debugging option: --------------------------------------------------------------------------- From eebda6974dea3445d5a4043a843d206046861470 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 27 Mar 2018 16:51:31 -0600 Subject: [PATCH 20/22] Decouple NamedXContentRegistry from ElasticsearchException (#29253) * Decouple NamedXContentRegistry from ElasticsearchException This commit decouples `NamedXContentRegistry` from using either `ElasticsearchException`, `ParsingException`, or `UnknownNamedObjectException`. This will allow us to move NamedXContentRegistry to its own lib as part of the xcontent extraction work. Relates to #28504 --- .../cluster/metadata/MetaData.java | 4 +- .../NamedObjectNotFoundException.java | 35 +++++++++++++ .../xcontent/NamedXContentRegistry.java | 19 +++---- .../xcontent/XContentParseException.java | 52 +++++++++++++++++++ .../index/query/AbstractQueryBuilder.java | 6 +-- .../common/xcontent/BaseXContentTestCase.java | 15 +++--- .../xcontent/XContentParserUtilsTests.java | 7 ++- .../rescore/QueryRescorerBuilderTests.java | 10 ++-- .../search/suggest/SuggestionTests.java | 6 +-- 9 files changed, 118 insertions(+), 36 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/common/xcontent/NamedObjectNotFoundException.java create mode 100644 server/src/main/java/org/elasticsearch/common/xcontent/XContentParseException.java diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index 9fff294daea19..a569bb9a36e29 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -43,7 +43,7 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.xcontent.UnknownNamedObjectException; +import org.elasticsearch.common.xcontent.NamedObjectNotFoundException; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -1173,7 +1173,7 @@ public static MetaData fromXContent(XContentParser parser) throws IOException { try { Custom custom = parser.namedObject(Custom.class, currentFieldName, null); builder.putCustom(custom.getWriteableName(), custom); - } catch (UnknownNamedObjectException ex) { + } catch (NamedObjectNotFoundException ex) { logger.warn("Skipping unknown custom object with type {}", currentFieldName); parser.skipChildren(); } diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/NamedObjectNotFoundException.java b/server/src/main/java/org/elasticsearch/common/xcontent/NamedObjectNotFoundException.java new file mode 100644 index 0000000000000..ecc322b60d8fc --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/xcontent/NamedObjectNotFoundException.java @@ -0,0 +1,35 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.xcontent; + +/** + * Thrown when {@link NamedXContentRegistry} cannot locate a named object to + * parse for a particular name + */ +public class NamedObjectNotFoundException extends XContentParseException { + + public NamedObjectNotFoundException(String message) { + this(null, message); + } + + public NamedObjectNotFoundException(XContentLocation location, String message) { + super(location, message); + } +} diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/NamedXContentRegistry.java b/server/src/main/java/org/elasticsearch/common/xcontent/NamedXContentRegistry.java index c19a667776f2e..9135bf648a19e 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/NamedXContentRegistry.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/NamedXContentRegistry.java @@ -19,10 +19,8 @@ package org.elasticsearch.common.xcontent; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.ParseField; -import org.elasticsearch.common.ParsingException; import java.io.IOException; import java.util.ArrayList; @@ -114,28 +112,31 @@ public NamedXContentRegistry(List entries) { } /** - * Parse a named object, throwing an exception if the parser isn't found. Throws an {@link ElasticsearchException} if the - * {@code categoryClass} isn't registered because this is almost always a bug. Throws a {@link UnknownNamedObjectException} if the + * Parse a named object, throwing an exception if the parser isn't found. Throws an {@link NamedObjectNotFoundException} if the + * {@code categoryClass} isn't registered because this is almost always a bug. Throws an {@link NamedObjectNotFoundException} if the * {@code categoryClass} is registered but the {@code name} isn't. + * + * @throws NamedObjectNotFoundException if the categoryClass or name is not registered */ public T parseNamedObject(Class categoryClass, String name, XContentParser parser, C context) throws IOException { Map parsers = registry.get(categoryClass); if (parsers == null) { if (registry.isEmpty()) { // The "empty" registry will never work so we throw a better exception as a hint. - throw new ElasticsearchException("namedObject is not supported for this parser"); + throw new NamedObjectNotFoundException("named objects are not supported for this parser"); } - throw new ElasticsearchException("Unknown namedObject category [" + categoryClass.getName() + "]"); + throw new NamedObjectNotFoundException("unknown named object category [" + categoryClass.getName() + "]"); } Entry entry = parsers.get(name); if (entry == null) { - throw new UnknownNamedObjectException(parser.getTokenLocation(), categoryClass, name); + throw new NamedObjectNotFoundException(parser.getTokenLocation(), "unable to parse " + categoryClass.getSimpleName() + + " with name [" + name + "]: parser not found"); } if (false == entry.name.match(name, parser.getDeprecationHandler())) { /* Note that this shouldn't happen because we already looked up the entry using the names but we need to call `match` anyway * because it is responsible for logging deprecation warnings. */ - throw new ParsingException(parser.getTokenLocation(), - "Unknown " + categoryClass.getSimpleName() + " [" + name + "]: Parser didn't match"); + throw new NamedObjectNotFoundException(parser.getTokenLocation(), + "unable to parse " + categoryClass.getSimpleName() + " with name [" + name + "]: parser didn't match"); } return categoryClass.cast(entry.parser.parse(parser, context)); } diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/XContentParseException.java b/server/src/main/java/org/elasticsearch/common/xcontent/XContentParseException.java new file mode 100644 index 0000000000000..cd2e3dbb59baa --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/xcontent/XContentParseException.java @@ -0,0 +1,52 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.xcontent; + +import java.util.Optional; + +/** + * Thrown when one of the XContent parsers cannot parse something. + */ +public class XContentParseException extends IllegalArgumentException { + + private final Optional location; + + public XContentParseException(String message) { + this(null, message); + } + + public XContentParseException(XContentLocation location, String message) { + super(message); + this.location = Optional.ofNullable(location); + } + + public int getLineNumber() { + return location.map(l -> l.lineNumber).orElse(-1); + } + + public int getColumnNumber() { + return location.map(l -> l.columnNumber).orElse(-1); + } + + @Override + public String getMessage() { + return location.map(l -> "[" + l.toString() + "] ").orElse("") + super.getMessage(); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java index d272bb29fbfa6..942c72f22935b 100644 --- a/server/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/AbstractQueryBuilder.java @@ -31,7 +31,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.xcontent.AbstractObjectParser; -import org.elasticsearch.common.xcontent.UnknownNamedObjectException; +import org.elasticsearch.common.xcontent.NamedObjectNotFoundException; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentLocation; import org.elasticsearch.common.xcontent.XContentParser; @@ -316,11 +316,11 @@ public static QueryBuilder parseInnerQueryBuilder(XContentParser parser) throws QueryBuilder result; try { result = parser.namedObject(QueryBuilder.class, queryName, null); - } catch (UnknownNamedObjectException e) { + } catch (NamedObjectNotFoundException e) { // Preserve the error message from 5.0 until we have a compellingly better message so we don't break BWC. // This intentionally doesn't include the causing exception because that'd change the "root_cause" of any unknown query errors throw new ParsingException(new XContentLocation(e.getLineNumber(), e.getColumnNumber()), - "no [query] registered for [" + e.getName() + "]"); + "no [query] registered for [" + queryName + "]"); } //end_object of the specific query (e.g. match, multi_match etc.) element if (parser.currentToken() != XContentParser.Token.END_OBJECT) { diff --git a/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java b/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java index 4efd5f480f810..b46485952d702 100644 --- a/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java +++ b/server/src/test/java/org/elasticsearch/common/xcontent/BaseXContentTestCase.java @@ -67,6 +67,7 @@ import static java.util.Collections.singletonMap; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.notNullValue; @@ -1007,22 +1008,20 @@ public void testNamedObject() throws IOException { { p.nextToken(); assertEquals("test", p.namedObject(Object.class, "str", null)); - UnknownNamedObjectException e = expectThrows(UnknownNamedObjectException.class, + NamedObjectNotFoundException e = expectThrows(NamedObjectNotFoundException.class, () -> p.namedObject(Object.class, "unknown", null)); - assertEquals("Unknown Object [unknown]", e.getMessage()); - assertEquals("java.lang.Object", e.getCategoryClass()); - assertEquals("unknown", e.getName()); + assertThat(e.getMessage(), endsWith("unable to parse Object with name [unknown]: parser not found")); } { - Exception e = expectThrows(ElasticsearchException.class, () -> p.namedObject(String.class, "doesn't matter", null)); - assertEquals("Unknown namedObject category [java.lang.String]", e.getMessage()); + Exception e = expectThrows(NamedObjectNotFoundException.class, () -> p.namedObject(String.class, "doesn't matter", null)); + assertEquals("unknown named object category [java.lang.String]", e.getMessage()); } { XContentParser emptyRegistryParser = xcontentType().xContent() .createParser(NamedXContentRegistry.EMPTY, DeprecationHandler.THROW_UNSUPPORTED_OPERATION, new byte[] {}); - Exception e = expectThrows(ElasticsearchException.class, + Exception e = expectThrows(NamedObjectNotFoundException.class, () -> emptyRegistryParser.namedObject(String.class, "doesn't matter", null)); - assertEquals("namedObject is not supported for this parser", e.getMessage()); + assertEquals("named objects are not supported for this parser", e.getMessage()); } } diff --git a/server/src/test/java/org/elasticsearch/common/xcontent/XContentParserUtilsTests.java b/server/src/test/java/org/elasticsearch/common/xcontent/XContentParserUtilsTests.java index e31a1ce72025c..5b65e6af7898b 100644 --- a/server/src/test/java/org/elasticsearch/common/xcontent/XContentParserUtilsTests.java +++ b/server/src/test/java/org/elasticsearch/common/xcontent/XContentParserUtilsTests.java @@ -40,6 +40,7 @@ import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureFieldName; import static org.elasticsearch.common.xcontent.XContentParserUtils.parseTypedKeysObject; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.instanceOf; public class XContentParserUtilsTests extends ESTestCase { @@ -187,11 +188,9 @@ public void testParseTypedKeysObject() throws IOException { ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser::getTokenLocation); ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.nextToken(), parser::getTokenLocation); ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser::getTokenLocation); - UnknownNamedObjectException e = expectThrows(UnknownNamedObjectException.class, + NamedObjectNotFoundException e = expectThrows(NamedObjectNotFoundException.class, () -> parseTypedKeysObject(parser, delimiter, Boolean.class, a -> {})); - assertEquals("Unknown Boolean [type]", e.getMessage()); - assertEquals("type", e.getName()); - assertEquals("java.lang.Boolean", e.getCategoryClass()); + assertThat(e.getMessage(), endsWith("unable to parse Boolean with name [type]: parser not found")); } final long longValue = randomLong(); diff --git a/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java b/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java index 2f35a832c3021..9a9797734b65f 100644 --- a/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java @@ -19,15 +19,14 @@ package org.elasticsearch.search.rescore; -import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.NamedObjectNotFoundException; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -40,9 +39,7 @@ import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.TextFieldMapper; -import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.MatchAllQueryBuilder; -import org.elasticsearch.index.query.MatchQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.QueryShardContext; @@ -58,7 +55,6 @@ import static java.util.Collections.emptyList; import static org.elasticsearch.test.EqualsHashCodeTestUtils.checkEqualsAndHashCode; -import static org.hamcrest.Matchers.containsString; public class QueryRescorerBuilderTests extends ESTestCase { @@ -220,8 +216,8 @@ public void testUnknownFieldsExpection() throws IOException { "}\n"; { XContentParser parser = createParser(rescoreElement); - Exception e = expectThrows(ParsingException.class, () -> RescorerBuilder.parseFromXContent(parser)); - assertEquals("Unknown RescorerBuilder [bad_rescorer_name]", e.getMessage()); + Exception e = expectThrows(NamedObjectNotFoundException.class, () -> RescorerBuilder.parseFromXContent(parser)); + assertEquals("[3:27] unable to parse RescorerBuilder with name [bad_rescorer_name]: parser not found", e.getMessage()); } rescoreElement = "{\n" + diff --git a/server/src/test/java/org/elasticsearch/search/suggest/SuggestionTests.java b/server/src/test/java/org/elasticsearch/search/suggest/SuggestionTests.java index 7a57d2c3e672f..c8384a948a66c 100644 --- a/server/src/test/java/org/elasticsearch/search/suggest/SuggestionTests.java +++ b/server/src/test/java/org/elasticsearch/search/suggest/SuggestionTests.java @@ -19,10 +19,10 @@ package org.elasticsearch.search.suggest; -import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.xcontent.DeprecationHandler; +import org.elasticsearch.common.xcontent.NamedObjectNotFoundException; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContent; @@ -180,8 +180,8 @@ public void testUnknownSuggestionTypeThrows() throws IOException { ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser::getTokenLocation); ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.nextToken(), parser::getTokenLocation); ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.nextToken(), parser::getTokenLocation); - ParsingException e = expectThrows(ParsingException.class, () -> Suggestion.fromXContent(parser)); - assertEquals("Unknown Suggestion [unknownType]", e.getMessage()); + NamedObjectNotFoundException e = expectThrows(NamedObjectNotFoundException.class, () -> Suggestion.fromXContent(parser)); + assertEquals("[1:31] unable to parse Suggestion with name [unknownType]: parser not found", e.getMessage()); } } From 38fd9998e7b7a428d3bcee35516fab2d1cb58aa6 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 27 Mar 2018 19:45:13 -0400 Subject: [PATCH 21/22] Require JDK 10 to build Elasticsearch (#29174) This commit bumps the minimum compiler version required to build Elasticsearch from JDK 9 to JDK 10. --- CONTRIBUTING.md | 6 +++--- .../main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 2617baadba013..69e90473a7f61 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -92,11 +92,11 @@ Contributing to the Elasticsearch codebase **Repository:** [https://github.com/elastic/elasticsearch](https://github.com/elastic/elasticsearch) -JDK 9 is required to build Elasticsearch. You must have a JDK 9 installation +JDK 10 is required to build Elasticsearch. You must have a JDK 10 installation with the environment variable `JAVA_HOME` referencing the path to Java home for -your JDK 9 installation. By default, tests use the same runtime as `JAVA_HOME`. +your JDK 10 installation. By default, tests use the same runtime as `JAVA_HOME`. However, since Elasticsearch, supports JDK 8 the build supports compiling with -JDK 9 and testing on a JDK 8 runtime; to do this, set `RUNTIME_JAVA_HOME` +JDK 10 and testing on a JDK 8 runtime; to do this, set `RUNTIME_JAVA_HOME` pointing to the Java home of a JDK 8 installation. Note that this mechanism can be used to test against other JDKs as well, this is not only limited to JDK 8. diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy index 5eb82c12616fc..d03591722a2fd 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/BuildPlugin.groovy @@ -58,7 +58,7 @@ import java.time.ZonedDateTime class BuildPlugin implements Plugin { static final JavaVersion minimumRuntimeVersion = JavaVersion.VERSION_1_8 - static final JavaVersion minimumCompilerVersion = JavaVersion.VERSION_1_9 + static final JavaVersion minimumCompilerVersion = JavaVersion.VERSION_1_10 @Override void apply(Project project) { From 1f6a3c1d80f8ba0d4d659765f563a94d911b1606 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 27 Mar 2018 22:58:44 -0400 Subject: [PATCH 22/22] Fix building Javadoc JARs on JDK for client JARs (#29274) When a module or plugin register that it has a client JAR, we copy artifacts like the Javadoc and sources JARs as the JARs for the client as well (with -client added to the name). I previously had to disable the Javadoc task on JDK 10 due to a bug in bin/javadoc. After JDK 10 went GA without a fix for this bug, I added workaround to fix the Javadoc task on JDK 10. However, I made a mistake reverting the previously skipped Javadocs tasks and missed that one that copies the Javadoc JAR for client JARs. This commit fixes that issue. --- .../gradle/plugin/PluginBuildPlugin.groovy | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy index a18472521522e..f802a2895909e 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy @@ -168,12 +168,10 @@ public class PluginBuildPlugin extends BuildPlugin { Files.copy(jarFile.resolveSibling(sourcesFileName), jarFile.resolveSibling(clientSourcesFileName), StandardCopyOption.REPLACE_EXISTING) - if (project.compilerJavaVersion < JavaVersion.VERSION_1_10) { - String javadocFileName = jarFile.fileName.toString().replace('.jar', '-javadoc.jar') - String clientJavadocFileName = clientFileName.replace('.jar', '-javadoc.jar') - Files.copy(jarFile.resolveSibling(javadocFileName), jarFile.resolveSibling(clientJavadocFileName), - StandardCopyOption.REPLACE_EXISTING) - } + String javadocFileName = jarFile.fileName.toString().replace('.jar', '-javadoc.jar') + String clientJavadocFileName = clientFileName.replace('.jar', '-javadoc.jar') + Files.copy(jarFile.resolveSibling(javadocFileName), jarFile.resolveSibling(clientJavadocFileName), + StandardCopyOption.REPLACE_EXISTING) } project.assemble.dependsOn(clientJar) }