From 146cb39143f93b6ce453229abf5be08335a75366 Mon Sep 17 00:00:00 2001 From: Tommaso Teofili Date: Thu, 28 Nov 2024 13:46:24 +0100 Subject: [PATCH 01/27] ESQL - enabling scoring with METADATA _score (#113120) * ESQL - enabling scoring with METADATA _score Co-authored-by: ChrisHegarty --- docs/changelog/113120.yaml | 5 + muted-tests.yml | 6 + .../search/sort/SortBuilder.java | 15 +- .../core/expression/MetadataAttribute.java | 5 +- .../compute/lucene/LuceneOperator.java | 5 +- .../compute/lucene/LuceneSourceOperator.java | 96 ++++-- .../lucene/LuceneTopNSourceOperator.java | 141 +++++++-- .../elasticsearch/compute/OperatorTests.java | 3 +- .../LuceneQueryExpressionEvaluatorTests.java | 33 +- .../lucene/LuceneSourceOperatorTests.java | 31 +- .../LuceneTopNSourceOperatorScoringTests.java | 151 +++++++++ .../lucene/LuceneTopNSourceOperatorTests.java | 50 ++- .../ValueSourceReaderTypeConversionTests.java | 9 +- .../ValuesSourceReaderOperatorTests.java | 9 +- .../src/main/resources/qstr-function.csv-spec | 1 - .../src/main/resources/scoring.csv-spec | 285 +++++++++++++++++ .../xpack/esql/action/EsqlActionTaskIT.java | 7 +- .../xpack/esql/action/LookupFromIndexIT.java | 3 +- .../xpack/esql/plugin/MatchFunctionIT.java | 299 ++++++++++++++++++ .../xpack/esql/plugin/MatchOperatorIT.java | 51 +++ .../xpack/esql/plugin/QueryStringIT.java | 96 ++++++ .../xpack/esql/action/EsqlCapabilities.java | 7 +- .../xpack/esql/analysis/Verifier.java | 9 + .../local/LucenePushdownPredicates.java | 5 + .../physical/local/PushTopNToSource.java | 18 +- .../local/ReplaceSourceAttributes.java | 14 +- .../xpack/esql/parser/LogicalPlanBuilder.java | 4 +- .../xpack/esql/plan/physical/EsQueryExec.java | 14 + .../planner/EsPhysicalOperationProviders.java | 14 +- .../xpack/esql/analysis/VerifierTests.java | 25 ++ .../optimizer/PhysicalPlanOptimizerTests.java | 62 ++++ .../physical/local/PushTopNToSourceTests.java | 193 ++++++++++- 32 files changed, 1570 insertions(+), 96 deletions(-) create mode 100644 docs/changelog/113120.yaml create mode 100644 x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorScoringTests.java create mode 100644 x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec create mode 100644 x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java diff --git a/docs/changelog/113120.yaml b/docs/changelog/113120.yaml new file mode 100644 index 0000000000000..801167d61c19c --- /dev/null +++ b/docs/changelog/113120.yaml @@ -0,0 +1,5 @@ +pr: 113120 +summary: ESQL - enabling scoring with METADATA `_score` +area: ES|QL +type: enhancement +issues: [] diff --git a/muted-tests.yml b/muted-tests.yml index 5cf16fdf3da0a..fdadc747289bb 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -224,6 +224,12 @@ tests: issue: https://github.com/elastic/elasticsearch/issues/117591 - class: org.elasticsearch.repositories.s3.RepositoryS3ClientYamlTestSuiteIT issue: https://github.com/elastic/elasticsearch/issues/117596 +- class: "org.elasticsearch.xpack.esql.qa.multi_node.EsqlSpecIT" + method: "test {scoring.*}" + issue: https://github.com/elastic/elasticsearch/issues/117641 +- class: "org.elasticsearch.xpack.esql.qa.single_node.EsqlSpecIT" + method: "test {scoring.*}" + issue: https://github.com/elastic/elasticsearch/issues/117641 # Examples: # diff --git a/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java index 0ac3b42dd5b10..5832b93b9462f 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/SortBuilder.java @@ -158,6 +158,11 @@ private static void parseCompoundSortField(XContentParser parser, List buildSort(List> sortBuilders, SearchExecutionContext context) throws IOException { + return buildSort(sortBuilders, context, true); + } + + public static Optional buildSort(List> sortBuilders, SearchExecutionContext context, boolean optimize) + throws IOException { List sortFields = new ArrayList<>(sortBuilders.size()); List sortFormats = new ArrayList<>(sortBuilders.size()); for (SortBuilder builder : sortBuilders) { @@ -172,9 +177,13 @@ public static Optional buildSort(List> sortBuilde if (sortFields.size() > 1) { sort = true; } else { - SortField sortField = sortFields.get(0); - if (sortField.getType() == SortField.Type.SCORE && sortField.getReverse() == false) { - sort = false; + if (optimize) { + SortField sortField = sortFields.get(0); + if (sortField.getType() == SortField.Type.SCORE && sortField.getReverse() == false) { + sort = false; + } else { + sort = true; + } } else { sort = true; } diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/MetadataAttribute.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/MetadataAttribute.java index 6e4e9292bfc99..0f1cfbb85039c 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/MetadataAttribute.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/MetadataAttribute.java @@ -31,6 +31,7 @@ public class MetadataAttribute extends TypedAttribute { public static final String TIMESTAMP_FIELD = "@timestamp"; public static final String TSID_FIELD = "_tsid"; + public static final String SCORE = "_score"; static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( Attribute.class, @@ -50,7 +51,9 @@ public class MetadataAttribute extends TypedAttribute { SourceFieldMapper.NAME, tuple(DataType.SOURCE, false), IndexModeFieldMapper.NAME, - tuple(DataType.KEYWORD, true) + tuple(DataType.KEYWORD, true), + SCORE, + tuple(DataType.DOUBLE, false) ); private final boolean searchable; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java index 6f75298e95dd7..bbc3ace3716ba 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java @@ -79,6 +79,7 @@ public abstract static class Factory implements SourceOperator.SourceOperatorFac protected final DataPartitioning dataPartitioning; protected final int taskConcurrency; protected final int limit; + protected final ScoreMode scoreMode; protected final LuceneSliceQueue sliceQueue; /** @@ -95,6 +96,7 @@ protected Factory( ScoreMode scoreMode ) { this.limit = limit; + this.scoreMode = scoreMode; this.dataPartitioning = dataPartitioning; var weightFunction = weightFunction(queryFunction, scoreMode); this.sliceQueue = LuceneSliceQueue.create(contexts, weightFunction, dataPartitioning, taskConcurrency); @@ -438,7 +440,8 @@ static Function weightFunction(Function 0) { - --remainingDocs; - docsBuilder.appendInt(doc); - currentPagePos++; - } else { - throw new CollectionTerminatedException(); - } + class LimitingCollector implements LeafCollector { + @Override + public void setScorer(Scorable scorer) {} + + @Override + public void collect(int doc) throws IOException { + if (remainingDocs > 0) { + --remainingDocs; + docsBuilder.appendInt(doc); + currentPagePos++; + } else { + throw new CollectionTerminatedException(); } - }; + } + } + + final class ScoringCollector extends LuceneSourceOperator.LimitingCollector { + private Scorable scorable; + + @Override + public void setScorer(Scorable scorer) { + this.scorable = scorer; + } + + @Override + public void collect(int doc) throws IOException { + super.collect(doc); + scoreBuilder.appendDouble(scorable.score()); + } } @Override @@ -139,15 +179,27 @@ public Page getCheckedOutput() throws IOException { IntBlock shard = null; IntBlock leaf = null; IntVector docs = null; + DoubleVector scores = null; + DocBlock docBlock = null; try { shard = blockFactory.newConstantIntBlockWith(scorer.shardContext().index(), currentPagePos); leaf = blockFactory.newConstantIntBlockWith(scorer.leafReaderContext().ord, currentPagePos); docs = docsBuilder.build(); docsBuilder = blockFactory.newIntVectorBuilder(Math.min(remainingDocs, maxPageSize)); - page = new Page(currentPagePos, new DocVector(shard.asVector(), leaf.asVector(), docs, true).asBlock()); + docBlock = new DocVector(shard.asVector(), leaf.asVector(), docs, true).asBlock(); + shard = null; + leaf = null; + docs = null; + if (scoreBuilder == null) { + page = new Page(currentPagePos, docBlock); + } else { + scores = scoreBuilder.build(); + scoreBuilder = blockFactory.newDoubleVectorBuilder(Math.min(remainingDocs, maxPageSize)); + page = new Page(currentPagePos, docBlock, scores.asBlock()); + } } finally { if (page == null) { - Releasables.closeExpectNoException(shard, leaf, docs); + Releasables.closeExpectNoException(shard, leaf, docs, docBlock, scores); } } currentPagePos = 0; @@ -160,7 +212,7 @@ public Page getCheckedOutput() throws IOException { @Override public void close() { - docsBuilder.close(); + Releasables.close(docsBuilder, scoreBuilder); } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java index 0f600958b93b3..8da62963ffb64 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java @@ -10,15 +10,22 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.ReaderUtil; import org.apache.lucene.search.CollectionTerminatedException; +import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.ScoreMode; -import org.apache.lucene.search.TopFieldCollector; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; +import org.apache.lucene.search.TopDocsCollector; import org.apache.lucene.search.TopFieldCollectorManager; +import org.apache.lucene.search.TopScoreDocCollectorManager; import org.elasticsearch.common.Strings; import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.DocBlock; import org.elasticsearch.compute.data.DocVector; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.DoubleVector; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; @@ -29,17 +36,21 @@ import org.elasticsearch.search.sort.SortBuilder; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.lucene.search.ScoreMode.COMPLETE; +import static org.apache.lucene.search.ScoreMode.TOP_DOCS; + /** * Source operator that builds Pages out of the output of a TopFieldCollector (aka TopN) */ public final class LuceneTopNSourceOperator extends LuceneOperator { - public static final class Factory extends LuceneOperator.Factory { + public static class Factory extends LuceneOperator.Factory { private final int maxPageSize; private final List> sorts; @@ -50,16 +61,17 @@ public Factory( int taskConcurrency, int maxPageSize, int limit, - List> sorts + List> sorts, + boolean scoring ) { - super(contexts, queryFunction, dataPartitioning, taskConcurrency, limit, ScoreMode.TOP_DOCS); + super(contexts, queryFunction, dataPartitioning, taskConcurrency, limit, scoring ? COMPLETE : TOP_DOCS); this.maxPageSize = maxPageSize; this.sorts = sorts; } @Override public SourceOperator get(DriverContext driverContext) { - return new LuceneTopNSourceOperator(driverContext.blockFactory(), maxPageSize, sorts, limit, sliceQueue); + return new LuceneTopNSourceOperator(driverContext.blockFactory(), maxPageSize, sorts, limit, sliceQueue, scoreMode); } public int maxPageSize() { @@ -75,6 +87,8 @@ public String describe() { + maxPageSize + ", limit = " + limit + + ", scoreMode = " + + scoreMode + ", sorts = [" + notPrettySorts + "]]"; @@ -93,17 +107,20 @@ public String describe() { private PerShardCollector perShardCollector; private final List> sorts; private final int limit; + private final ScoreMode scoreMode; public LuceneTopNSourceOperator( BlockFactory blockFactory, int maxPageSize, List> sorts, int limit, - LuceneSliceQueue sliceQueue + LuceneSliceQueue sliceQueue, + ScoreMode scoreMode ) { super(blockFactory, maxPageSize, sliceQueue); this.sorts = sorts; this.limit = limit; + this.scoreMode = scoreMode; } @Override @@ -145,7 +162,7 @@ private Page collect() throws IOException { try { if (perShardCollector == null || perShardCollector.shardContext.index() != scorer.shardContext().index()) { // TODO: share the bottom between shardCollectors - perShardCollector = new PerShardCollector(scorer.shardContext(), sorts, limit); + perShardCollector = newPerShardCollector(scorer.shardContext(), sorts, limit); } var leafCollector = perShardCollector.getLeafCollector(scorer.leafReaderContext()); scorer.scoreNextRange(leafCollector, scorer.leafReaderContext().reader().getLiveDocs(), maxPageSize); @@ -171,7 +188,7 @@ private Page emit(boolean startEmitting) { assert isEmitting() == false : "offset=" + offset + " score_docs=" + Arrays.toString(scoreDocs); offset = 0; if (perShardCollector != null) { - scoreDocs = perShardCollector.topFieldCollector.topDocs().scoreDocs; + scoreDocs = perShardCollector.collector.topDocs().scoreDocs; } else { scoreDocs = new ScoreDoc[0]; } @@ -183,10 +200,13 @@ private Page emit(boolean startEmitting) { IntBlock shard = null; IntVector segments = null; IntVector docs = null; + DocBlock docBlock = null; + DoubleBlock scores = null; Page page = null; try ( IntVector.Builder currentSegmentBuilder = blockFactory.newIntVectorFixedBuilder(size); - IntVector.Builder currentDocsBuilder = blockFactory.newIntVectorFixedBuilder(size) + IntVector.Builder currentDocsBuilder = blockFactory.newIntVectorFixedBuilder(size); + DoubleVector.Builder currentScoresBuilder = scoreVectorOrNull(size); ) { int start = offset; offset += size; @@ -196,53 +216,130 @@ private Page emit(boolean startEmitting) { int segment = ReaderUtil.subIndex(doc, leafContexts); currentSegmentBuilder.appendInt(segment); currentDocsBuilder.appendInt(doc - leafContexts.get(segment).docBase); // the offset inside the segment + if (currentScoresBuilder != null) { + float score = getScore(scoreDocs[i]); + currentScoresBuilder.appendDouble(score); + } } shard = blockFactory.newConstantIntBlockWith(perShardCollector.shardContext.index(), size); segments = currentSegmentBuilder.build(); docs = currentDocsBuilder.build(); - page = new Page(size, new DocVector(shard.asVector(), segments, docs, null).asBlock()); + docBlock = new DocVector(shard.asVector(), segments, docs, null).asBlock(); + shard = null; + segments = null; + docs = null; + if (currentScoresBuilder == null) { + page = new Page(size, docBlock); + } else { + scores = currentScoresBuilder.build().asBlock(); + page = new Page(size, docBlock, scores); + } } finally { if (page == null) { - Releasables.closeExpectNoException(shard, segments, docs); + Releasables.closeExpectNoException(shard, segments, docs, docBlock, scores); } } pagesEmitted++; return page; } + private float getScore(ScoreDoc scoreDoc) { + if (scoreDoc instanceof FieldDoc fieldDoc) { + if (Float.isNaN(fieldDoc.score)) { + if (sorts != null) { + return (Float) fieldDoc.fields[sorts.size() + 1]; + } else { + return (Float) fieldDoc.fields[0]; + } + } else { + return fieldDoc.score; + } + } else { + return scoreDoc.score; + } + } + + private DoubleVector.Builder scoreVectorOrNull(int size) { + if (scoreMode.needsScores()) { + return blockFactory.newDoubleVectorFixedBuilder(size); + } else { + return null; + } + } + @Override protected void describe(StringBuilder sb) { sb.append(", limit = ").append(limit); + sb.append(", scoreMode = ").append(scoreMode); String notPrettySorts = sorts.stream().map(Strings::toString).collect(Collectors.joining(",")); sb.append(", sorts = [").append(notPrettySorts).append("]"); } - static final class PerShardCollector { + PerShardCollector newPerShardCollector(ShardContext shardContext, List> sorts, int limit) throws IOException { + Optional sortAndFormats = shardContext.buildSort(sorts); + if (sortAndFormats.isEmpty()) { + throw new IllegalStateException("sorts must not be disabled in TopN"); + } + if (scoreMode.needsScores() == false) { + return new NonScoringPerShardCollector(shardContext, sortAndFormats.get().sort, limit); + } else { + SortField[] sortFields = sortAndFormats.get().sort.getSort(); + if (sortFields != null && sortFields.length == 1 && sortFields[0].needsScores() && sortFields[0].getReverse() == false) { + // SORT _score DESC + return new ScoringPerShardCollector( + shardContext, + new TopScoreDocCollectorManager(limit, null, limit, false).newCollector() + ); + } else { + // SORT ..., _score, ... + var sort = new Sort(); + if (sortFields != null) { + var l = new ArrayList<>(Arrays.asList(sortFields)); + l.add(SortField.FIELD_DOC); + l.add(SortField.FIELD_SCORE); + sort = new Sort(l.toArray(SortField[]::new)); + } + return new ScoringPerShardCollector( + shardContext, + new TopFieldCollectorManager(sort, limit, null, limit, false).newCollector() + ); + } + } + } + + abstract static class PerShardCollector { private final ShardContext shardContext; - private final TopFieldCollector topFieldCollector; + private final TopDocsCollector collector; private int leafIndex; private LeafCollector leafCollector; private Thread currentThread; - PerShardCollector(ShardContext shardContext, List> sorts, int limit) throws IOException { + PerShardCollector(ShardContext shardContext, TopDocsCollector collector) { this.shardContext = shardContext; - Optional sortAndFormats = shardContext.buildSort(sorts); - if (sortAndFormats.isEmpty()) { - throw new IllegalStateException("sorts must not be disabled in TopN"); - } - - // We don't use CollectorManager here as we don't retrieve the total hits and sort by score. - this.topFieldCollector = new TopFieldCollectorManager(sortAndFormats.get().sort, limit, null, 0, false).newCollector(); + this.collector = collector; } LeafCollector getLeafCollector(LeafReaderContext leafReaderContext) throws IOException { if (currentThread != Thread.currentThread() || leafIndex != leafReaderContext.ord) { - leafCollector = topFieldCollector.getLeafCollector(leafReaderContext); + leafCollector = collector.getLeafCollector(leafReaderContext); leafIndex = leafReaderContext.ord; currentThread = Thread.currentThread(); } return leafCollector; } } + + static final class NonScoringPerShardCollector extends PerShardCollector { + NonScoringPerShardCollector(ShardContext shardContext, Sort sort, int limit) { + // We don't use CollectorManager here as we don't retrieve the total hits and sort by score. + super(shardContext, new TopFieldCollectorManager(sort, limit, null, 0, false).newCollector()); + } + } + + static final class ScoringPerShardCollector extends PerShardCollector { + ScoringPerShardCollector(ShardContext shardContext, TopDocsCollector topDocsCollector) { + super(shardContext, topDocsCollector); + } + } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java index 0d39a5bf8227e..e6ef10e53ec7c 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/OperatorTests.java @@ -394,7 +394,8 @@ static LuceneOperator.Factory luceneOperatorFactory(IndexReader reader, Query qu randomFrom(DataPartitioning.values()), randomIntBetween(1, 10), randomPageSize(), - limit + limit, + false // no scoring ); } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneQueryExpressionEvaluatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneQueryExpressionEvaluatorTests.java index beca522878358..ffaee536b443e 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneQueryExpressionEvaluatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneQueryExpressionEvaluatorTests.java @@ -27,6 +27,8 @@ import org.elasticsearch.compute.data.BooleanVector; import org.elasticsearch.compute.data.BytesRefBlock; import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.DocBlock; +import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.lucene.LuceneQueryExpressionEvaluator.DenseCollector; @@ -120,8 +122,9 @@ public void testTermQueryShuffled() throws IOException { private void assertTermQuery(String term, List results) { int matchCount = 0; for (Page page : results) { - BytesRefVector terms = page.getBlock(1).asVector(); - BooleanVector matches = page.getBlock(2).asVector(); + int initialBlockIndex = initialBlockIndex(page); + BytesRefVector terms = page.getBlock(initialBlockIndex).asVector(); + BooleanVector matches = page.getBlock(initialBlockIndex + 1).asVector(); for (int i = 0; i < page.getPositionCount(); i++) { BytesRef termAtPosition = terms.getBytesRef(i, new BytesRef()); assertThat(matches.getBoolean(i), equalTo(termAtPosition.utf8ToString().equals(term))); @@ -155,8 +158,9 @@ private void testTermsQuery(boolean shuffleDocs) throws IOException { List results = runQuery(values, new TermInSetQuery(MultiTermQuery.CONSTANT_SCORE_REWRITE, FIELD, matchingBytes), shuffleDocs); int matchCount = 0; for (Page page : results) { - BytesRefVector terms = page.getBlock(1).asVector(); - BooleanVector matches = page.getBlock(2).asVector(); + int initialBlockIndex = initialBlockIndex(page); + BytesRefVector terms = page.getBlock(initialBlockIndex).asVector(); + BooleanVector matches = page.getBlock(initialBlockIndex + 1).asVector(); for (int i = 0; i < page.getPositionCount(); i++) { BytesRef termAtPosition = terms.getBytesRef(i, new BytesRef()); assertThat(matches.getBoolean(i), equalTo(matching.contains(termAtPosition.utf8ToString()))); @@ -207,7 +211,7 @@ private List runQuery(Set values, Query query, boolean shuffleDocs List results = new ArrayList<>(); Driver driver = new Driver( driverContext, - luceneOperatorFactory(reader, new MatchAllDocsQuery(), LuceneOperator.NO_LIMIT).get(driverContext), + luceneOperatorFactory(reader, new MatchAllDocsQuery(), LuceneOperator.NO_LIMIT, scoring).get(driverContext), operators, new TestResultPageSinkOperator(results::add), () -> {} @@ -248,7 +252,21 @@ private DriverContext driverContext() { return new DriverContext(blockFactory.bigArrays(), blockFactory); } - static LuceneOperator.Factory luceneOperatorFactory(IndexReader reader, Query query, int limit) { + // Scores are not interesting to this test, but enabled conditionally and effectively ignored just for coverage. + private final boolean scoring = randomBoolean(); + + // Returns the initial block index, ignoring the score block if scoring is enabled + private int initialBlockIndex(Page page) { + assert page.getBlock(0) instanceof DocBlock : "expected doc block at index 0"; + if (scoring) { + assert page.getBlock(1) instanceof DoubleBlock : "expected double block at index 1"; + return 2; + } else { + return 1; + } + } + + static LuceneOperator.Factory luceneOperatorFactory(IndexReader reader, Query query, int limit, boolean scoring) { final ShardContext searchContext = new LuceneSourceOperatorTests.MockShardContext(reader, 0); return new LuceneSourceOperator.Factory( List.of(searchContext), @@ -256,7 +274,8 @@ static LuceneOperator.Factory luceneOperatorFactory(IndexReader reader, Query qu randomFrom(DataPartitioning.values()), randomIntBetween(1, 10), randomPageSize(), - limit + limit, + scoring ); } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java index 626190c04c501..2dcc5e20d3f98 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java @@ -17,6 +17,8 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.compute.data.DocBlock; +import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; @@ -63,10 +65,10 @@ public void closeIndex() throws IOException { @Override protected LuceneSourceOperator.Factory simple() { - return simple(randomFrom(DataPartitioning.values()), between(1, 10_000), 100); + return simple(randomFrom(DataPartitioning.values()), between(1, 10_000), 100, scoring); } - private LuceneSourceOperator.Factory simple(DataPartitioning dataPartitioning, int numDocs, int limit) { + private LuceneSourceOperator.Factory simple(DataPartitioning dataPartitioning, int numDocs, int limit, boolean scoring) { int commitEvery = Math.max(1, numDocs / 10); try ( RandomIndexWriter writer = new RandomIndexWriter( @@ -91,7 +93,7 @@ private LuceneSourceOperator.Factory simple(DataPartitioning dataPartitioning, i ShardContext ctx = new MockShardContext(reader, 0); Function queryFunction = c -> new MatchAllDocsQuery(); int maxPageSize = between(10, Math.max(10, numDocs)); - return new LuceneSourceOperator.Factory(List.of(ctx), queryFunction, dataPartitioning, 1, maxPageSize, limit); + return new LuceneSourceOperator.Factory(List.of(ctx), queryFunction, dataPartitioning, 1, maxPageSize, limit, scoring); } @Override @@ -101,7 +103,10 @@ protected Matcher expectedToStringOfSimple() { @Override protected Matcher expectedDescriptionOfSimple() { - return matchesRegex("LuceneSourceOperator\\[dataPartitioning = (DOC|SHARD|SEGMENT), maxPageSize = \\d+, limit = 100]"); + return matchesRegex( + "LuceneSourceOperator" + + "\\[dataPartitioning = (DOC|SHARD|SEGMENT), maxPageSize = \\d+, limit = 100, scoreMode = (COMPLETE|COMPLETE_NO_SCORES)]" + ); } // TODO tests for the other data partitioning configurations @@ -149,7 +154,7 @@ public void testShardDataPartitioningWithCranky() { } private void testSimple(DriverContext ctx, int size, int limit) { - LuceneSourceOperator.Factory factory = simple(DataPartitioning.SHARD, size, limit); + LuceneSourceOperator.Factory factory = simple(DataPartitioning.SHARD, size, limit, scoring); Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD, ElementType.LONG); List results = new ArrayList<>(); @@ -164,7 +169,7 @@ private void testSimple(DriverContext ctx, int size, int limit) { } for (Page page : results) { - LongBlock sBlock = page.getBlock(1); + LongBlock sBlock = page.getBlock(initialBlockIndex(page)); for (int p = 0; p < page.getPositionCount(); p++) { assertThat(sBlock.getLong(sBlock.getFirstValueIndex(p)), both(greaterThanOrEqualTo(0L)).and(lessThan((long) size))); } @@ -174,6 +179,20 @@ private void testSimple(DriverContext ctx, int size, int limit) { assertThat(results, hasSize(both(greaterThanOrEqualTo(minPages)).and(lessThanOrEqualTo(maxPages)))); } + // Scores are not interesting to this test, but enabled conditionally and effectively ignored just for coverage. + private final boolean scoring = randomBoolean(); + + // Returns the initial block index, ignoring the score block if scoring is enabled + private int initialBlockIndex(Page page) { + assert page.getBlock(0) instanceof DocBlock : "expected doc block at index 0"; + if (scoring) { + assert page.getBlock(1) instanceof DoubleBlock : "expected double block at index 1"; + return 2; + } else { + return 1; + } + } + /** * Creates a mock search context with the given index reader. * The returned mock search context can be used to test with {@link LuceneOperator}. diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorScoringTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorScoringTests.java new file mode 100644 index 0000000000000..a0fa1c2c01c0a --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorScoringTests.java @@ -0,0 +1,151 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.lucene; + +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.NoMergePolicy; +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.SortedNumericSelector; +import org.apache.lucene.search.SortedNumericSortField; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.elasticsearch.compute.data.DoubleBlock; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.Driver; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.compute.operator.OperatorTestCase; +import org.elasticsearch.compute.operator.TestResultPageSinkOperator; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.NumberFieldMapper; +import org.elasticsearch.search.DocValueFormat; +import org.elasticsearch.search.sort.FieldSortBuilder; +import org.elasticsearch.search.sort.SortAndFormats; +import org.elasticsearch.search.sort.SortBuilder; +import org.hamcrest.Matcher; +import org.junit.After; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.function.Function; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.matchesRegex; + +public class LuceneTopNSourceOperatorScoringTests extends LuceneTopNSourceOperatorTests { + private static final MappedFieldType S_FIELD = new NumberFieldMapper.NumberFieldType("s", NumberFieldMapper.NumberType.LONG); + private Directory directory = newDirectory(); + private IndexReader reader; + + @After + private void closeIndex() throws IOException { + IOUtils.close(reader, directory); + } + + @Override + protected LuceneTopNSourceOperator.Factory simple() { + return simple(DataPartitioning.SHARD, 10_000, 100); + } + + private LuceneTopNSourceOperator.Factory simple(DataPartitioning dataPartitioning, int size, int limit) { + int commitEvery = Math.max(1, size / 10); + try ( + RandomIndexWriter writer = new RandomIndexWriter( + random(), + directory, + newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE) + ) + ) { + for (int d = 0; d < size; d++) { + List doc = new ArrayList<>(); + doc.add(new SortedNumericDocValuesField("s", d)); + writer.addDocument(doc); + if (d % commitEvery == 0) { + writer.commit(); + } + } + reader = writer.getReader(); + } catch (IOException e) { + throw new RuntimeException(e); + } + + ShardContext ctx = new LuceneSourceOperatorTests.MockShardContext(reader, 0) { + @Override + public Optional buildSort(List> sorts) { + SortField field = new SortedNumericSortField("s", SortField.Type.LONG, false, SortedNumericSelector.Type.MIN); + return Optional.of(new SortAndFormats(new Sort(field), new DocValueFormat[] { null })); + } + }; + Function queryFunction = c -> new MatchAllDocsQuery(); + int taskConcurrency = 0; + int maxPageSize = between(10, Math.max(10, size)); + List> sorts = List.of(new FieldSortBuilder("s")); + return new LuceneTopNSourceOperator.Factory( + List.of(ctx), + queryFunction, + dataPartitioning, + taskConcurrency, + maxPageSize, + limit, + sorts, + true // scoring + ); + } + + @Override + protected Matcher expectedToStringOfSimple() { + return matchesRegex("LuceneTopNSourceOperator\\[maxPageSize = \\d+, limit = 100, scoreMode = COMPLETE, sorts = \\[\\{.+}]]"); + } + + @Override + protected Matcher expectedDescriptionOfSimple() { + return matchesRegex( + "LuceneTopNSourceOperator" + + "\\[dataPartitioning = (DOC|SHARD|SEGMENT), maxPageSize = \\d+, limit = 100, scoreMode = COMPLETE, sorts = \\[\\{.+}]]" + ); + } + + @Override + protected void testSimple(DriverContext ctx, int size, int limit) { + LuceneTopNSourceOperator.Factory factory = simple(DataPartitioning.SHARD, size, limit); + Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD, ElementType.LONG); + + List results = new ArrayList<>(); + OperatorTestCase.runDriver( + new Driver(ctx, factory.get(ctx), List.of(readS.get(ctx)), new TestResultPageSinkOperator(results::add), () -> {}) + ); + OperatorTestCase.assertDriverContext(ctx); + + long expectedS = 0; + int maxPageSize = factory.maxPageSize(); + for (Page page : results) { + if (limit - expectedS < maxPageSize) { + assertThat(page.getPositionCount(), equalTo((int) (limit - expectedS))); + } else { + assertThat(page.getPositionCount(), equalTo(maxPageSize)); + } + DoubleBlock sBlock = page.getBlock(1); + for (int p = 0; p < page.getPositionCount(); p++) { + assertThat(sBlock.getDouble(sBlock.getFirstValueIndex(p)), equalTo(1.0d)); + expectedS++; + } + } + int pages = (int) Math.ceil((float) Math.min(size, limit) / maxPageSize); + assertThat(results, hasSize(pages)); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java index 938c4ce5c9f7d..d9a0b70b7931e 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperatorTests.java @@ -20,6 +20,8 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.compute.data.DocBlock; +import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; @@ -56,7 +58,7 @@ public class LuceneTopNSourceOperatorTests extends AnyOperatorTestCase { private IndexReader reader; @After - public void closeIndex() throws IOException { + private void closeIndex() throws IOException { IOUtils.close(reader, directory); } @@ -105,19 +107,25 @@ public Optional buildSort(List> sorts) { taskConcurrency, maxPageSize, limit, - sorts + sorts, + scoring ); } @Override protected Matcher expectedToStringOfSimple() { - return matchesRegex("LuceneTopNSourceOperator\\[maxPageSize = \\d+, limit = 100, sorts = \\[\\{.+}]]"); + var s = scoring ? "COMPLETE" : "TOP_DOCS"; + return matchesRegex("LuceneTopNSourceOperator\\[maxPageSize = \\d+, limit = 100, scoreMode = " + s + ", sorts = \\[\\{.+}]]"); } @Override protected Matcher expectedDescriptionOfSimple() { + var s = scoring ? "COMPLETE" : "TOP_DOCS"; return matchesRegex( - "LuceneTopNSourceOperator\\[dataPartitioning = (DOC|SHARD|SEGMENT), maxPageSize = \\d+, limit = 100, sorts = \\[\\{.+}]]" + "LuceneTopNSourceOperator" + + "\\[dataPartitioning = (DOC|SHARD|SEGMENT), maxPageSize = \\d+, limit = 100, scoreMode = " + + s + + ", sorts = \\[\\{.+}]]" ); } @@ -137,12 +145,24 @@ public void testShardDataPartitioningWithCranky() { } } - private void testShardDataPartitioning(DriverContext context) { + void testShardDataPartitioning(DriverContext context) { int size = between(1_000, 20_000); int limit = between(10, size); testSimple(context, size, limit); } + public void testWithCranky() { + try { + int size = between(1_000, 20_000); + int limit = between(10, size); + testSimple(crankyDriverContext(), size, limit); + logger.info("cranky didn't break"); + } catch (CircuitBreakingException e) { + logger.info("broken", e); + assertThat(e.getMessage(), equalTo(CrankyCircuitBreakerService.ERROR_MESSAGE)); + } + } + public void testEmpty() { testEmpty(driverContext()); } @@ -157,11 +177,11 @@ public void testEmptyWithCranky() { } } - private void testEmpty(DriverContext context) { + void testEmpty(DriverContext context) { testSimple(context, 0, between(10, 10_000)); } - private void testSimple(DriverContext ctx, int size, int limit) { + protected void testSimple(DriverContext ctx, int size, int limit) { LuceneTopNSourceOperator.Factory factory = simple(DataPartitioning.SHARD, size, limit); Operator.OperatorFactory readS = ValuesSourceReaderOperatorTests.factory(reader, S_FIELD, ElementType.LONG); @@ -178,7 +198,7 @@ private void testSimple(DriverContext ctx, int size, int limit) { } else { assertThat(page.getPositionCount(), equalTo(factory.maxPageSize())); } - LongBlock sBlock = page.getBlock(1); + LongBlock sBlock = page.getBlock(initialBlockIndex(page)); for (int p = 0; p < page.getPositionCount(); p++) { assertThat(sBlock.getLong(sBlock.getFirstValueIndex(p)), equalTo(expectedS++)); } @@ -186,4 +206,18 @@ private void testSimple(DriverContext ctx, int size, int limit) { int pages = (int) Math.ceil((float) Math.min(size, limit) / factory.maxPageSize()); assertThat(results, hasSize(pages)); } + + // Scores are not interesting to this test, but enabled conditionally and effectively ignored just for coverage. + private final boolean scoring = randomBoolean(); + + // Returns the initial block index, ignoring the score block if scoring is enabled + private int initialBlockIndex(Page page) { + assert page.getBlock(0) instanceof DocBlock : "expected doc block at index 0"; + if (scoring) { + assert page.getBlock(1) instanceof DoubleBlock : "expected double block at index 1"; + return 2; + } else { + return 1; + } + } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValueSourceReaderTypeConversionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValueSourceReaderTypeConversionTests.java index f6d81af7c14e5..f31573f121a71 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValueSourceReaderTypeConversionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValueSourceReaderTypeConversionTests.java @@ -265,7 +265,8 @@ private SourceOperator simpleInput(DriverContext context, int size, int commitEv DataPartitioning.SHARD, 1,// randomIntBetween(1, 10), pageSize, - LuceneOperator.NO_LIMIT + LuceneOperator.NO_LIMIT, + false // no scoring ); return luceneFactory.get(context); } @@ -1292,7 +1293,8 @@ public void testWithNulls() throws IOException { randomFrom(DataPartitioning.values()), randomIntBetween(1, 10), randomPageSize(), - LuceneOperator.NO_LIMIT + LuceneOperator.NO_LIMIT, + false // no scoring ); var vsShardContext = new ValuesSourceReaderOperator.ShardContext(reader(indexKey), () -> SourceLoader.FROM_STORED_SOURCE); try ( @@ -1450,7 +1452,8 @@ public void testManyShards() throws IOException { DataPartitioning.SHARD, randomIntBetween(1, 10), 1000, - LuceneOperator.NO_LIMIT + LuceneOperator.NO_LIMIT, + false // no scoring ); // TODO add index2 MappedFieldType ft = mapperService(indexKey).fieldType("key"); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java index c8dd6f87be5fc..95b313b0b5412 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperatorTests.java @@ -170,7 +170,8 @@ private SourceOperator simpleInput(DriverContext context, int size, int commitEv DataPartitioning.SHARD, randomIntBetween(1, 10), pageSize, - LuceneOperator.NO_LIMIT + LuceneOperator.NO_LIMIT, + false // no scoring ); return luceneFactory.get(context); } @@ -1301,7 +1302,8 @@ public void testWithNulls() throws IOException { randomFrom(DataPartitioning.values()), randomIntBetween(1, 10), randomPageSize(), - LuceneOperator.NO_LIMIT + LuceneOperator.NO_LIMIT, + false // no scoring ); try ( Driver driver = new Driver( @@ -1524,7 +1526,8 @@ public void testManyShards() throws IOException { DataPartitioning.SHARD, randomIntBetween(1, 10), 1000, - LuceneOperator.NO_LIMIT + LuceneOperator.NO_LIMIT, + false // no scoring ); MappedFieldType ft = mapperService.fieldType("key"); var readerFactory = new ValuesSourceReaderOperator.Factory( diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/qstr-function.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/qstr-function.csv-spec index 6039dc05b6c44..2c84bdae6b32e 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/qstr-function.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/qstr-function.csv-spec @@ -100,7 +100,6 @@ book_no:keyword | title:text 7140 | The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) ; - qstrWithMultivaluedTextField required_capability: qstr_function diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec new file mode 100644 index 0000000000000..d4c7b8c59fdbc --- /dev/null +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/scoring.csv-spec @@ -0,0 +1,285 @@ +############################################### +# Tests for scoring support +# + +singleQstrBoostScoringSorted +required_capability: metadata_score +required_capability: qstr_function + +from books metadata _score +| where qstr("author:Lord Rings^2") +| eval c_score = ceil(_score) +| keep book_no, title, c_score +| sort c_score desc, book_no asc +| LIMIT 2; + +book_no:keyword | title:text | c_score:double +2675 | The Lord of the Rings - Boxed Set | 6.0 +4023 | A Tolkien Compass: Including J. R. R. Tolkien's Guide to the Names in The Lord of the Rings | 6.0 +; + +singleMatchWithKeywordFieldScoring +required_capability: metadata_score +required_capability: match_operator_colon + +from books metadata _score +| where author.keyword:"William Faulkner" +| keep book_no, author, _score +| sort book_no; + +book_no:keyword | author:text | _score:double +2713 | William Faulkner | 2.3142893314361572 +2883 | William Faulkner | 2.3142893314361572 +4724 | William Faulkner | 2.3142893314361572 +4977 | William Faulkner | 2.3142893314361572 +5119 | William Faulkner | 2.3142893314361572 +5404 | William Faulkner | 2.3142893314361572 +5578 | William Faulkner | 2.3142893314361572 +8077 | William Faulkner | 2.3142893314361572 +9896 | William Faulkner | 2.3142893314361572 +; + +qstrWithFieldAndScoringSortedEval +required_capability: qstr_function +required_capability: metadata_score + +from books metadata _score +| where qstr("title:rings") +| sort _score desc +| eval _score::long +| keep book_no, title, _score +| limit 3; + +book_no:keyword | title:text | _score:double +2675 | The Lord of the Rings - Boxed Set | 2.7583377361297607 +7140 | The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) | 1.9239964485168457 +2714 | Return of the King Being the Third Part of The Lord of the Rings | 1.9239964485168457 +; + +qstrWithFieldAndScoringSorted +required_capability: qstr_function +required_capability: metadata_score + +from books metadata _score +| where qstr("title:rings") +| sort _score desc, book_no desc +| keep book_no, title, _score +| limit 3; + +book_no:keyword | title:text | _score:double +2675 | The Lord of the Rings - Boxed Set | 2.7583377361297607 +7140 | The Lord of the Rings Poster Collection: Six Paintings by Alan Lee (No. 1) | 1.9239964485168457 +2714 | Return of the King Being the Third Part of The Lord of the Rings | 1.9239964485168457 +; + +singleQstrScoringManipulated +required_capability: metadata_score +required_capability: qstr_function + +from books metadata _score +| where qstr("author:William Faulkner") +| eval add_score = ceil(_score) + 1 +| keep book_no, author, add_score +| sort book_no +| LIMIT 2; + +book_no:keyword | author:text | add_score:double +2378 | [Carol Faulkner, Holly Byers Ochoa, Lucretia Mott] | 2.0 +2713 | William Faulkner | 7.0 +; + +testMultiValuedFieldWithConjunctionWithScore +required_capability: match_function +required_capability: metadata_score + +from employees metadata _score +| where match(job_positions, "Data Scientist") and match(job_positions, "Support Engineer") +| keep emp_no, first_name, last_name, job_positions, _score; + +emp_no:integer | first_name:keyword | last_name:keyword | job_positions:keyword | _score:double +10043 | Yishay | Tzvieli | [Data Scientist, Python Developer, Support Engineer] | 5.233309745788574 +; + +testMatchAndQueryStringFunctionsWithScore +required_capability: match_function +required_capability: metadata_score + +from employees metadata _score +| where match(job_positions, "Data Scientist") and qstr("job_positions: (Support Engineer) and gender: F") +| keep emp_no, first_name, last_name, job_positions, _score; +ignoreOrder:true + +emp_no:integer | first_name:keyword | last_name:keyword | job_positions:keyword | _score:double +10041 | Uri | Lenart | [Data Scientist, Head Human Resources, Internship, Senior Team Lead] | 3.509873867034912 +10043 | Yishay | Tzvieli | [Data Scientist, Python Developer, Support Engineer] | 5.233309745788574 +; + +multipleWhereWithMatchScoringNoSort +required_capability: metadata_score +required_capability: match_operator_colon + +from books metadata _score +| where title:"short stories" +| where author:"Ursula K. Le Guin" +| keep book_no, title, author, _score; + +ignoreOrder:true +book_no:keyword | title:text | author:text | _score:double +8480 | The wind's twelve quarters: Short stories | Ursula K. Le Guin | 14.489097595214844 +; + +multipleWhereWithMatchScoring +required_capability: metadata_score +required_capability: match_operator_colon + +from books metadata _score +| where title:"short stories" +| where author:"Ursula K. Le Guin" +| keep book_no, title, author, _score +| sort book_no; + +book_no:keyword | title:text | author:text | _score:double +8480 | The wind's twelve quarters: Short stories | Ursula K. Le Guin | 14.489097595214844 +; + +combinedMatchWithFunctionsScoring +required_capability: metadata_score +required_capability: match_operator_colon + +from books metadata _score +| where title:"Tolkien" AND author:"Tolkien" AND year > 2000 +| where mv_count(author) == 1 +| keep book_no, title, author, year, _score +| sort book_no; + +book_no:keyword | title:text | author:text | year:integer | _score:double +5335 | Letters of J R R Tolkien | J.R.R. Tolkien | 2014 | 5.448054313659668 +; + +singleQstrScoring +required_capability: metadata_score +required_capability: qstr_function + +from books metadata _score +| where qstr("author:William Faulkner") +| keep book_no, author, _score +| sort book_no +| LIMIT 2; + +book_no:keyword | author:text | _score:double +2378 | [Carol Faulkner, Holly Byers Ochoa, Lucretia Mott] | 0.9976131916046143 +2713 | William Faulkner | 5.9556169509887695 +; + +singleQstrScoringGrok +required_capability: metadata_score +required_capability: qstr_function + +from books metadata _score +| where qstr("author:Lord Rings") +| GROK title "%{WORD:title} %{WORD}" +| sort _score desc +| keep book_no, title, _score +| LIMIT 3; + +book_no:keyword | title:keyword | _score:double +8875 | The | 2.9505908489227295 +4023 | A | 2.8327860832214355 +2675 | The | 2.7583377361297607 +; + +combinedMatchWithScoringEvalNoSort +required_capability: metadata_score +required_capability: match_operator_colon + +from books metadata _score +| where title:"Tolkien" AND author:"Tolkien" AND year > 2000 +| where mv_count(author) == 1 +| eval c_score = ceil(_score) +| keep book_no, title, author, year, c_score; + +ignoreOrder:true +book_no:keyword | title:text | author:text | year:integer | c_score:double +5335 | Letters of J R R Tolkien | J.R.R. Tolkien | 2014 | 6 +; + +singleQstrScoringRename +required_capability: metadata_score +required_capability: qstr_function + +from books metadata _score +| where qstr("author:Lord Rings") +| rename _score as rank +| sort rank desc +| keep book_no, rank +| LIMIT 3; + +book_no:keyword | rank:double +8875 | 2.9505908489227295 +4023 | 2.8327860832214355 +2675 | 2.7583377361297607 +; + +singleMatchWithTextFieldScoring +required_capability: metadata_score +required_capability: match_operator_colon + +from books metadata _score +| where author:"William Faulkner" +| sort book_no +| keep book_no, author, _score +| limit 5; + +book_no:keyword | author:text | _score:double +2378 | [Carol Faulkner, Holly Byers Ochoa, Lucretia Mott] | 0.9976131916046143 +2713 | William Faulkner | 4.272439002990723 +2847 | Colleen Faulkner | 1.7401835918426514 +2883 | William Faulkner | 4.272439002990723 +3293 | Danny Faulkner | 1.7401835918426514 +; + +combinedMatchWithFunctionsScoringNoSort +required_capability: metadata_score +required_capability: match_operator_colon + +from books metadata _score +| where title:"Tolkien" AND author:"Tolkien" AND year > 2000 +| where mv_count(author) == 1 +| keep book_no, title, author, year, _score; + +ignoreOrder:true +book_no:keyword | title:text | author:text | year:integer | _score:double +5335 | Letters of J R R Tolkien | J.R.R. Tolkien | 2014 | 5.448054313659668 +; + +combinedMatchWithScoringEval +required_capability: metadata_score +required_capability: match_operator_colon + +from books metadata _score +| where title:"Tolkien" AND author:"Tolkien" AND year > 2000 +| where mv_count(author) == 1 +| eval c_score = ceil(_score) +| keep book_no, title, author, year, c_score +| sort book_no; + +book_no:keyword | title:text | author:text | year:integer | c_score:double +5335 | Letters of J R R Tolkien | J.R.R. Tolkien | 2014 | 6 +; + +singleQstrScoringEval +required_capability: metadata_score +required_capability: qstr_function + +from books metadata _score +| where qstr("author:Lord Rings") +| eval c_score = ceil(_score) +| keep book_no, c_score +| sort book_no desc +| LIMIT 3; + +book_no:keyword | c_score:double +8875 | 3.0 +7350 | 2.0 +7140 | 3.0 +; diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java index 56453a291ea81..1939f81353c0e 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java @@ -89,7 +89,7 @@ public void setup() { assumeTrue("requires query pragmas", canUseQueryPragmas()); nodeLevelReduction = randomBoolean(); READ_DESCRIPTION = """ - \\_LuceneSourceOperator[dataPartitioning = SHARD, maxPageSize = pageSize(), limit = 2147483647] + \\_LuceneSourceOperator[dataPartitioning = SHARD, maxPageSize = pageSize(), limit = 2147483647, scoreMode = COMPLETE_NO_SCORES] \\_ValuesSourceReaderOperator[fields = [pause_me]] \\_AggregationOperator[mode = INITIAL, aggs = sum of longs] \\_ExchangeSinkOperator""".replace("pageSize()", Integer.toString(pageSize())); @@ -448,6 +448,7 @@ protected void doRun() throws Exception { public void testTaskContentsForTopNQuery() throws Exception { READ_DESCRIPTION = ("\\_LuceneTopNSourceOperator[dataPartitioning = SHARD, maxPageSize = pageSize(), limit = 1000, " + + "scoreMode = TOP_DOCS, " + "sorts = [{\"pause_me\":{\"order\":\"asc\",\"missing\":\"_last\",\"unmapped_type\":\"long\"}}]]\n" + "\\_ValuesSourceReaderOperator[fields = [pause_me]]\n" + "\\_ProjectOperator[projection = [1]]\n" @@ -482,7 +483,7 @@ public void testTaskContentsForTopNQuery() throws Exception { public void testTaskContentsForLimitQuery() throws Exception { String limit = Integer.toString(randomIntBetween(pageSize() + 1, 2 * numberOfDocs())); READ_DESCRIPTION = """ - \\_LuceneSourceOperator[dataPartitioning = SHARD, maxPageSize = pageSize(), limit = limit()] + \\_LuceneSourceOperator[dataPartitioning = SHARD, maxPageSize = pageSize(), limit = limit(), scoreMode = COMPLETE_NO_SCORES] \\_ValuesSourceReaderOperator[fields = [pause_me]] \\_ProjectOperator[projection = [1]] \\_ExchangeSinkOperator""".replace("pageSize()", Integer.toString(pageSize())).replace("limit()", limit); @@ -511,7 +512,7 @@ public void testTaskContentsForLimitQuery() throws Exception { public void testTaskContentsForGroupingStatsQuery() throws Exception { READ_DESCRIPTION = """ - \\_LuceneSourceOperator[dataPartitioning = SHARD, maxPageSize = pageSize(), limit = 2147483647] + \\_LuceneSourceOperator[dataPartitioning = SHARD, maxPageSize = pageSize(), limit = 2147483647, scoreMode = COMPLETE_NO_SCORES] \\_ValuesSourceReaderOperator[fields = [foo]] \\_OrdinalsGroupingOperator(aggs = max of longs) \\_ExchangeSinkOperator""".replace("pageSize()", Integer.toString(pageSize())); diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java index 5c0c13b48df3b..3b9359fe66d40 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java @@ -148,7 +148,8 @@ public void testLookupIndex() throws IOException { DataPartitioning.SEGMENT, 1, 10000, - DocIdSetIterator.NO_MORE_DOCS + DocIdSetIterator.NO_MORE_DOCS, + false // no scoring ); ValuesSourceReaderOperator.Factory reader = new ValuesSourceReaderOperator.Factory( List.of( diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java new file mode 100644 index 0000000000000..99f7d48a0d636 --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchFunctionIT.java @@ -0,0 +1,299 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.esql.plugin; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.xpack.esql.VerificationException; +import org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; +import org.elasticsearch.xpack.esql.action.EsqlQueryRequest; +import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; +import org.junit.Before; + +import java.util.List; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.CoreMatchers.containsString; + +//@TestLogging(value = "org.elasticsearch.xpack.esql:TRACE,org.elasticsearch.compute:TRACE", reason = "debug") +public class MatchFunctionIT extends AbstractEsqlIntegTestCase { + + @Before + public void setupIndex() { + createAndPopulateIndex(); + } + + @Override + protected EsqlQueryResponse run(EsqlQueryRequest request) { + assumeTrue("match function capability not available", EsqlCapabilities.Cap.MATCH_FUNCTION.isEnabled()); + return super.run(request); + } + + public void testSimpleWhereMatch() { + var query = """ + FROM test + | WHERE match(content, "fox") + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(1), List.of(6))); + } + } + + public void testCombinedWhereMatch() { + var query = """ + FROM test + | WHERE match(content, "fox") AND id > 5 + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(6))); + } + } + + public void testMultipleMatch() { + var query = """ + FROM test + | WHERE match(content, "fox") AND match(content, "brown") + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(1), List.of(6))); + } + } + + public void testMultipleWhereMatch() { + var query = """ + FROM test + | WHERE match(content, "fox") AND match(content, "brown") + | EVAL summary = CONCAT("document with id: ", to_str(id), "and content: ", content) + | SORT summary + | LIMIT 4 + | WHERE match(content, "brown fox") + | KEEP id + """; + + var error = expectThrows(ElasticsearchException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("[MATCH] function cannot be used after LIMIT")); + } + + public void testNotWhereMatch() { + var query = """ + FROM test + | WHERE NOT match(content, "brown fox") + | KEEP id + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id")); + assertColumnTypes(resp.columns(), List.of("integer")); + assertValues(resp.values(), List.of(List.of(5))); + } + } + + public void testWhereMatchWithScoring() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE match(content, "fox") + | KEEP id, _score + | SORT id ASC + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + + public void testWhereMatchWithScoringDifferentSort() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE match(content, "fox") + | KEEP id, _score + | SORT id DESC + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues(resp.values(), List.of(List.of(6, 0.9114001989364624), List.of(1, 1.156558871269226))); + } + } + + public void testWhereMatchWithScoringSortScore() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE match(content, "fox") + | KEEP id, _score + | SORT _score DESC + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + + public void testWhereMatchWithScoringNoSort() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE content:"fox" + | KEEP id, _score + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValuesInAnyOrder(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + + public void testNonExistingColumn() { + var query = """ + FROM test + | WHERE something:"fox" + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("Unknown column [something]")); + } + + public void testWhereMatchEvalColumn() { + var query = """ + FROM test + | EVAL upper_content = to_upper(content) + | WHERE upper_content:"FOX" + | KEEP id + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat( + error.getMessage(), + containsString("[:] operator cannot operate on [upper_content], which is not a field from an index mapping") + ); + } + + public void testWhereMatchOverWrittenColumn() { + var query = """ + FROM test + | DROP content + | EVAL content = CONCAT("document with ID ", to_str(id)) + | WHERE content:"document" + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat( + error.getMessage(), + containsString("[:] operator cannot operate on [content], which is not a field from an index mapping") + ); + } + + public void testWhereMatchAfterStats() { + var query = """ + FROM test + | STATS count(*) + | WHERE content:"fox" + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("Unknown column [content]")); + } + + public void testWhereMatchWithFunctions() { + var query = """ + FROM test + | WHERE content:"fox" OR to_upper(content) == "FOX" + """; + var error = expectThrows(ElasticsearchException.class, () -> run(query)); + assertThat( + error.getMessage(), + containsString( + "Invalid condition [content:\"fox\" OR to_upper(content) == \"FOX\"]. " + + "[:] operator can't be used as part of an or condition" + ) + ); + } + + public void testWhereMatchWithRow() { + var query = """ + ROW content = "a brown fox" + | WHERE content:"fox" + """; + + var error = expectThrows(ElasticsearchException.class, () -> run(query)); + assertThat( + error.getMessage(), + containsString("[:] operator cannot operate on [\"a brown fox\"], which is not a field from an index mapping") + ); + } + + public void testMatchWithinEval() { + var query = """ + FROM test + | EVAL matches_query = content:"fox" + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("[:] operator is only supported in WHERE commands")); + } + + public void testMatchWithNonTextField() { + var query = """ + FROM test + | WHERE id:"fox" + """; + + var error = expectThrows(VerificationException.class, () -> run(query)); + assertThat(error.getMessage(), containsString("first argument of [id:\"fox\"] must be [string], found value [id] type [integer]")); + } + + private void createAndPopulateIndex() { + var indexName = "test"; + var client = client().admin().indices(); + var CreateRequest = client.prepareCreate(indexName) + .setSettings(Settings.builder().put("index.number_of_shards", 1)) + .setMapping("id", "type=integer", "content", "type=text"); + assertAcked(CreateRequest); + client().prepareBulk() + .add(new IndexRequest(indexName).id("1").source("id", 1, "content", "This is a brown fox")) + .add(new IndexRequest(indexName).id("2").source("id", 2, "content", "This is a brown dog")) + .add(new IndexRequest(indexName).id("3").source("id", 3, "content", "This dog is really brown")) + .add(new IndexRequest(indexName).id("4").source("id", 4, "content", "The dog is brown but this document is very very long")) + .add(new IndexRequest(indexName).id("5").source("id", 5, "content", "There is also a white cat")) + .add(new IndexRequest(indexName).id("6").source("id", 6, "content", "The quick brown fox jumps over the lazy dog")) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + ensureYellow(indexName); + } +} diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java index 3b647583f1129..6a360eb319abb 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/MatchOperatorIT.java @@ -14,6 +14,7 @@ import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; import org.junit.Before; import java.util.List; @@ -105,6 +106,56 @@ public void testNotWhereMatch() { } } + public void testWhereMatchWithScoring() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE content:"fox" + | KEEP id, _score + | SORT id ASC + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + + public void testWhereMatchWithScoringDifferentSort() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE content:"fox" + | KEEP id, _score + | SORT id + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + + public void testWhereMatchWithScoringNoSort() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE content:"fox" + | KEEP id, _score + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValuesInAnyOrder(resp.values(), List.of(List.of(1, 1.156558871269226), List.of(6, 0.9114001989364624))); + } + } + public void testNonExistingColumn() { var query = """ FROM test diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/QueryStringIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/QueryStringIT.java index 03af16d29e9b4..a3d1ac931528c 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/QueryStringIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/plugin/QueryStringIT.java @@ -13,6 +13,7 @@ import org.elasticsearch.index.query.QueryShardException; import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.action.AbstractEsqlIntegTestCase; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; import org.junit.Before; import java.util.List; @@ -137,4 +138,99 @@ private void createAndPopulateIndex() { .get(); ensureYellow(indexName); } + + public void testWhereQstrWithScoring() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE qstr("content: fox") + | KEEP id, _score + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValuesInAnyOrder( + resp.values(), + List.of( + List.of(2, 0.3028995096683502), + List.of(3, 0.3028995096683502), + List.of(4, 0.2547692656517029), + List.of(5, 0.28161853551864624) + ) + ); + + } + } + + public void testWhereQstrWithScoringSorted() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE qstr("content:fox fox") + | KEEP id, _score + | SORT _score DESC + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValues( + resp.values(), + List.of( + List.of(3, 1.5605685710906982), + List.of(2, 0.6057990193367004), + List.of(5, 0.5632370710372925), + List.of(4, 0.5095385313034058) + ) + ); + + } + } + + public void testWhereQstrWithScoringNoSort() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE qstr("content: fox") + | KEEP id, _score + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValuesInAnyOrder( + resp.values(), + List.of( + List.of(2, 0.3028995096683502), + List.of(3, 0.3028995096683502), + List.of(4, 0.2547692656517029), + List.of(5, 0.28161853551864624) + ) + ); + } + } + + public void testWhereQstrWithNonPushableAndScoring() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var query = """ + FROM test + METADATA _score + | WHERE qstr("content: fox") + AND abs(id) > 0 + | EVAL c_score = ceil(_score) + | KEEP id, c_score + | SORT id DESC + | LIMIT 2 + """; + + try (var resp = run(query)) { + assertColumnNames(resp.columns(), List.of("id", "c_score")); + assertColumnTypes(resp.columns(), List.of("integer", "double")); + assertValuesInAnyOrder(resp.values(), List.of(List.of(5, 1.0), List.of(4, 1.0))); + } + } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index d8004f73f613f..9bd4211855699 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -539,7 +539,12 @@ public enum Cap { /** * Fix for https://github.com/elastic/elasticsearch/issues/114714, again */ - FIX_STATS_BY_FOLDABLE_EXPRESSION_2,; + FIX_STATS_BY_FOLDABLE_EXPRESSION_2, + + /** + * Support the "METADATA _score" directive to enable _score column. + */ + METADATA_SCORE(Build.current().isSnapshot()); private final boolean enabled; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java index 2be13398dab2f..5f8c011cff53a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Verifier.java @@ -19,6 +19,7 @@ import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.Expressions; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; +import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute; import org.elasticsearch.xpack.esql.core.expression.NameId; import org.elasticsearch.xpack.esql.core.expression.NamedExpression; import org.elasticsearch.xpack.esql.core.expression.TypeResolutions; @@ -221,6 +222,7 @@ else if (p instanceof Lookup lookup) { checkFullTextQueryFunctions(p, failures); }); checkRemoteEnrich(plan, failures); + checkMetadataScoreNameReserved(plan, failures); if (failures.isEmpty()) { checkLicense(plan, licenseState, failures); @@ -234,6 +236,13 @@ else if (p instanceof Lookup lookup) { return failures; } + private static void checkMetadataScoreNameReserved(LogicalPlan p, Set failures) { + // _score can only be set as metadata attribute + if (p.inputSet().stream().anyMatch(a -> MetadataAttribute.SCORE.equals(a.name()) && (a instanceof MetadataAttribute) == false)) { + failures.add(fail(p, "`" + MetadataAttribute.SCORE + "` is a reserved METADATA attribute")); + } + } + private void checkSort(LogicalPlan p, Set failures) { if (p instanceof OrderBy ob) { ob.order().forEach(o -> { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/LucenePushdownPredicates.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/LucenePushdownPredicates.java index feb8717f007b7..8046d6bc56607 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/LucenePushdownPredicates.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/LucenePushdownPredicates.java @@ -9,6 +9,7 @@ import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; +import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.stats.SearchStats; @@ -59,6 +60,10 @@ default boolean isPushableFieldAttribute(Expression exp) { return false; } + default boolean isPushableMetadataAttribute(Expression exp) { + return exp instanceof MetadataAttribute ma && ma.name().equals(MetadataAttribute.SCORE); + } + /** * The default implementation of this has no access to SearchStats, so it can only make decisions based on the FieldAttribute itself. * In particular, it assumes TEXT fields have no exact subfields (underlying keyword field), diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java index 925e144b69fcc..2b531257e594a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java @@ -14,6 +14,7 @@ import org.elasticsearch.xpack.esql.core.expression.AttributeMap; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; +import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute; import org.elasticsearch.xpack.esql.core.expression.NameId; import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute; import org.elasticsearch.xpack.esql.expression.Order; @@ -57,6 +58,7 @@ * */ public class PushTopNToSource extends PhysicalOptimizerRules.ParameterizedOptimizerRule { + @Override protected PhysicalPlan rule(TopNExec topNExec, LocalPhysicalOptimizerContext ctx) { Pushable pushable = evaluatePushable(topNExec, LucenePushdownPredicates.from(ctx.searchStats())); @@ -155,6 +157,8 @@ && canPushDownOrders(topNExec.order(), lucenePushdownPredicates)) { order.nullsPosition() ) ); + } else if (lucenePushdownPredicates.isPushableMetadataAttribute(order.child())) { + pushableSorts.add(new EsQueryExec.ScoreSort(order.direction())); } else if (order.child() instanceof ReferenceAttribute referenceAttribute) { Attribute resolvedAttribute = aliasReplacedBy.resolve(referenceAttribute, referenceAttribute); if (distances.containsKey(resolvedAttribute.id())) { @@ -192,13 +196,23 @@ && canPushDownOrders(topNExec.order(), lucenePushdownPredicates)) { private static boolean canPushDownOrders(List orders, LucenePushdownPredicates lucenePushdownPredicates) { // allow only exact FieldAttributes (no expressions) for sorting - return orders.stream().allMatch(o -> lucenePushdownPredicates.isPushableFieldAttribute(o.child())); + return orders.stream() + .allMatch( + o -> lucenePushdownPredicates.isPushableFieldAttribute(o.child()) + || lucenePushdownPredicates.isPushableMetadataAttribute(o.child()) + ); } private static List buildFieldSorts(List orders) { List sorts = new ArrayList<>(orders.size()); for (Order o : orders) { - sorts.add(new EsQueryExec.FieldSort(((FieldAttribute) o.child()).exactAttribute(), o.direction(), o.nullsPosition())); + if (o.child() instanceof FieldAttribute fa) { + sorts.add(new EsQueryExec.FieldSort(fa.exactAttribute(), o.direction(), o.nullsPosition())); + } else if (o.child() instanceof MetadataAttribute ma && MetadataAttribute.SCORE.equals(ma.name())) { + sorts.add(new EsQueryExec.ScoreSort(o.direction())); + } else { + assert false : "unexpected ordering on expression type " + o.child().getClass(); + } } return sorts; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java index 74ea6f99e5e59..11e386ddd046c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/ReplaceSourceAttributes.java @@ -16,6 +16,7 @@ import org.elasticsearch.xpack.esql.plan.physical.EsSourceExec; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; +import java.util.ArrayList; import java.util.List; import static org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection.UP; @@ -29,6 +30,8 @@ public ReplaceSourceAttributes() { @Override protected PhysicalPlan rule(EsSourceExec plan) { var docId = new FieldAttribute(plan.source(), EsQueryExec.DOC_ID_FIELD.getName(), EsQueryExec.DOC_ID_FIELD); + final List attributes = new ArrayList<>(); + attributes.add(docId); if (plan.indexMode() == IndexMode.TIME_SERIES) { Attribute tsid = null, timestamp = null; for (Attribute attr : plan.output()) { @@ -42,9 +45,14 @@ protected PhysicalPlan rule(EsSourceExec plan) { if (tsid == null || timestamp == null) { throw new IllegalStateException("_tsid or @timestamp are missing from the time-series source"); } - return new EsQueryExec(plan.source(), plan.index(), plan.indexMode(), List.of(docId, tsid, timestamp), plan.query()); - } else { - return new EsQueryExec(plan.source(), plan.index(), plan.indexMode(), List.of(docId), plan.query()); + attributes.add(tsid); + attributes.add(timestamp); } + plan.output().forEach(attr -> { + if (attr instanceof MetadataAttribute ma && ma.name().equals(MetadataAttribute.SCORE)) { + attributes.add(ma); + } + }); + return new EsQueryExec(plan.source(), plan.index(), plan.indexMode(), attributes, plan.query()); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java index 99e03b3653f79..24398afa18010 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/LogicalPlanBuilder.java @@ -16,6 +16,7 @@ import org.elasticsearch.dissect.DissectParser; import org.elasticsearch.index.IndexMode; import org.elasticsearch.xpack.esql.VerificationException; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; import org.elasticsearch.xpack.esql.common.Failure; import org.elasticsearch.xpack.esql.core.expression.Alias; import org.elasticsearch.xpack.esql.core.expression.Attribute; @@ -276,7 +277,8 @@ public LogicalPlan visitFromCommand(EsqlBaseParser.FromCommandContext ctx) { for (var c : metadataOptionContext.UNQUOTED_SOURCE()) { String id = c.getText(); Source src = source(c); - if (MetadataAttribute.isSupported(id) == false) { + if (MetadataAttribute.isSupported(id) == false // TODO: drop check below once METADATA_SCORE is no longer snapshot-only + || (EsqlCapabilities.Cap.METADATA_SCORE.isEnabled() == false && MetadataAttribute.SCORE.equals(id))) { throw new ParsingException(src, "unsupported metadata field [" + id + "]"); } Attribute a = metadataMap.put(id, MetadataAttribute.create(src, id)); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsQueryExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsQueryExec.java index 82848fb2f1062..267b9e613abef 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsQueryExec.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/EsQueryExec.java @@ -15,6 +15,7 @@ import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.GeoDistanceSortBuilder; +import org.elasticsearch.search.sort.ScoreSortBuilder; import org.elasticsearch.search.sort.SortBuilder; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.xpack.esql.core.expression.Attribute; @@ -94,6 +95,19 @@ public SortBuilder sortBuilder() { } } + public record ScoreSort(Order.OrderDirection direction) implements Sort { + @Override + public SortBuilder sortBuilder() { + return new ScoreSortBuilder(); + } + + @Override + public FieldAttribute field() { + // TODO: refactor this: not all Sorts are backed by FieldAttributes + return null; + } + } + public EsQueryExec(Source source, EsIndex index, IndexMode indexMode, List attributes, QueryBuilder query) { this(source, index, indexMode, attributes, query, null, null, null); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java index ab0d68b152262..15f5b6579098d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java @@ -51,6 +51,7 @@ import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; +import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.MultiTypeEsField; import org.elasticsearch.xpack.esql.expression.function.scalar.convert.AbstractConvertFunction; @@ -165,7 +166,10 @@ public final PhysicalOperation sourcePhysicalOperation(EsQueryExec esQueryExec, assert esQueryExec.estimatedRowSize() != null : "estimated row size not initialized"; int rowEstimatedSize = esQueryExec.estimatedRowSize(); int limit = esQueryExec.limit() != null ? (Integer) esQueryExec.limit().fold() : NO_LIMIT; - if (sorts != null && sorts.isEmpty() == false) { + boolean scoring = esQueryExec.attrs() + .stream() + .anyMatch(a -> a instanceof MetadataAttribute && a.name().equals(MetadataAttribute.SCORE)); + if ((sorts != null && sorts.isEmpty() == false)) { List> sortBuilders = new ArrayList<>(sorts.size()); for (Sort sort : sorts) { sortBuilders.add(sort.sortBuilder()); @@ -177,7 +181,8 @@ public final PhysicalOperation sourcePhysicalOperation(EsQueryExec esQueryExec, context.queryPragmas().taskConcurrency(), context.pageSize(rowEstimatedSize), limit, - sortBuilders + sortBuilders, + scoring ); } else { if (esQueryExec.indexMode() == IndexMode.TIME_SERIES) { @@ -195,7 +200,8 @@ public final PhysicalOperation sourcePhysicalOperation(EsQueryExec esQueryExec, context.queryPragmas().dataPartitioning(), context.queryPragmas().taskConcurrency(), context.pageSize(rowEstimatedSize), - limit + limit, + scoring ); } } @@ -273,7 +279,7 @@ public IndexSearcher searcher() { @Override public Optional buildSort(List> sorts) throws IOException { - return SortBuilder.buildSort(sorts, ctx); + return SortBuilder.buildSort(sorts, ctx, false); } @Override diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 355073fcc873f..6074601535477 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -12,6 +12,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.esql.VerificationException; import org.elasticsearch.xpack.esql.action.EsqlCapabilities; +import org.elasticsearch.xpack.esql.core.expression.Attribute; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.core.type.InvalidMappedField; @@ -21,6 +22,7 @@ import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.parser.QueryParam; import org.elasticsearch.xpack.esql.parser.QueryParams; +import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import java.util.ArrayList; import java.util.LinkedHashMap; @@ -1754,6 +1756,29 @@ public void testToDatePeriodToTimeDurationWithInvalidType() { ); } + public void testNonMetadataScore() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + assertEquals("1:12: `_score` is a reserved METADATA attribute", error("from foo | eval _score = 10")); + + assertEquals( + "1:48: `_score` is a reserved METADATA attribute", + error("from foo metadata _score | where qstr(\"bar\") | eval _score = _score + 1") + ); + } + + public void testScoreRenaming() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + assertEquals("1:33: `_score` is a reserved METADATA attribute", error("from foo METADATA _id, _score | rename _id as _score")); + + assertTrue(passes("from foo metadata _score | rename _score as foo").stream().anyMatch(a -> a.name().equals("foo"))); + } + + private List passes(String query) { + LogicalPlan logicalPlan = defaultAnalyzer.analyze(parser.createStatement(query)); + assertTrue(logicalPlan.resolved()); + return logicalPlan.output(); + } + public void testIntervalAsString() { // DateTrunc for (String interval : List.of("1 minu", "1 dy", "1.5 minutes", "0.5 days", "minutes 1", "day 5")) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index f3ba11457a715..1f131f79c3d0e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -36,6 +36,7 @@ import org.elasticsearch.xpack.esql.EsqlTestUtils.TestConfigurableSearchStats; import org.elasticsearch.xpack.esql.EsqlTestUtils.TestConfigurableSearchStats.Config; import org.elasticsearch.xpack.esql.VerificationException; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; import org.elasticsearch.xpack.esql.analysis.Analyzer; import org.elasticsearch.xpack.esql.analysis.AnalyzerContext; import org.elasticsearch.xpack.esql.analysis.EnrichResolution; @@ -63,6 +64,7 @@ import org.elasticsearch.xpack.esql.expression.function.aggregate.SpatialAggregateFunction; import org.elasticsearch.xpack.esql.expression.function.aggregate.SpatialCentroid; import org.elasticsearch.xpack.esql.expression.function.aggregate.Sum; +import org.elasticsearch.xpack.esql.expression.function.fulltext.Match; import org.elasticsearch.xpack.esql.expression.function.scalar.math.Round; import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialContains; import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialDisjoint; @@ -6581,6 +6583,66 @@ public void testLookupThenTopN() { ); } + public void testScore() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var plan = physicalPlan(""" + from test metadata _score + | where match(first_name, "john") + | keep _score + """); + + ProjectExec outerProject = as(plan, ProjectExec.class); + LimitExec limitExec = as(outerProject.child(), LimitExec.class); + ExchangeExec exchange = as(limitExec.child(), ExchangeExec.class); + FragmentExec frag = as(exchange.child(), FragmentExec.class); + + LogicalPlan opt = logicalOptimizer.optimize(frag.fragment()); + Limit limit = as(opt, Limit.class); + Filter filter = as(limit.child(), Filter.class); + + Match match = as(filter.condition(), Match.class); + assertTrue(match.field() instanceof FieldAttribute); + assertEquals("first_name", ((FieldAttribute) match.field()).field().getName()); + + EsRelation esRelation = as(filter.child(), EsRelation.class); + assertTrue(esRelation.optimized()); + assertTrue(esRelation.resolved()); + assertTrue(esRelation.output().stream().anyMatch(a -> a.name().equals(MetadataAttribute.SCORE) && a instanceof MetadataAttribute)); + } + + public void testScoreTopN() { + assumeTrue("'METADATA _score' is disabled", EsqlCapabilities.Cap.METADATA_SCORE.isEnabled()); + var plan = physicalPlan(""" + from test metadata _score + | where match(first_name, "john") + | keep _score + | sort _score desc + """); + + ProjectExec projectExec = as(plan, ProjectExec.class); + TopNExec topNExec = as(projectExec.child(), TopNExec.class); + ExchangeExec exchange = as(topNExec.child(), ExchangeExec.class); + FragmentExec frag = as(exchange.child(), FragmentExec.class); + + LogicalPlan opt = logicalOptimizer.optimize(frag.fragment()); + TopN topN = as(opt, TopN.class); + List order = topN.order(); + Order scoreOrer = order.getFirst(); + assertEquals(Order.OrderDirection.DESC, scoreOrer.direction()); + Expression child = scoreOrer.child(); + assertTrue(child instanceof MetadataAttribute ma && ma.name().equals(MetadataAttribute.SCORE)); + Filter filter = as(topN.child(), Filter.class); + + Match match = as(filter.condition(), Match.class); + assertTrue(match.field() instanceof FieldAttribute); + assertEquals("first_name", ((FieldAttribute) match.field()).field().getName()); + + EsRelation esRelation = as(filter.child(), EsRelation.class); + assertTrue(esRelation.optimized()); + assertTrue(esRelation.resolved()); + assertTrue(esRelation.output().stream().anyMatch(a -> a.name().equals(MetadataAttribute.SCORE) && a instanceof MetadataAttribute)); + } + @SuppressWarnings("SameParameterValue") private static void assertFilterCondition( Filter filter, diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSourceTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSourceTests.java index 98f0af8e4b8e6..2429bcb1a1b04 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSourceTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSourceTests.java @@ -20,6 +20,7 @@ import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; import org.elasticsearch.xpack.esql.core.expression.Literal; +import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute; import org.elasticsearch.xpack.esql.core.expression.Nullability; import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute; import org.elasticsearch.xpack.esql.core.tree.Source; @@ -64,6 +65,13 @@ public void testSimpleSortField() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSimpleScoreSortField() { + // FROM index METADATA _score | SORT _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false).scoreSort().limit(10); + assertPushdownSort(query); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSimpleSortMultipleFields() { // FROM index | SORT field, integer, double | LIMIT 10 var query = from("index").sort("field").sort("integer").sort("double").limit(10); @@ -71,6 +79,13 @@ public void testSimpleSortMultipleFields() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSimpleSortMultipleFieldsAndScore() { + // FROM index | SORT field, integer, double, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false).sort("field").sort("integer").sort("double").scoreSort().limit(10); + assertPushdownSort(query); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSimpleSortFieldAndEvalLiteral() { // FROM index | EVAL x = 1 | SORT field | LIMIT 10 var query = from("index").eval("x", e -> e.i(1)).sort("field").limit(10); @@ -78,6 +93,13 @@ public void testSimpleSortFieldAndEvalLiteral() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSimpleSortFieldScoreAndEvalLiteral() { + // FROM index METADATA _score | EVAL x = 1 | SORT field, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false).eval("x", e -> e.i(1)).sort("field").scoreSort().limit(10); + assertPushdownSort(query, List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSimpleSortFieldWithAlias() { // FROM index | EVAL x = field | SORT field | LIMIT 10 var query = from("index").eval("x", b -> b.field("field")).sort("field").limit(10); @@ -98,6 +120,21 @@ public void testSimpleSortMultipleFieldsWithAliases() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSimpleSortMultipleFieldsWithAliasesAndScore() { + // FROM index | EVAL x = field, y = integer, z = double | SORT field, integer, double, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("x", b -> b.field("field")) + .eval("y", b -> b.field("integer")) + .eval("z", b -> b.field("double")) + .sort("field") + .sort("integer") + .sort("double") + .scoreSort() + .limit(10); + assertPushdownSort(query, List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSimpleSortFieldAsAlias() { // FROM index | EVAL x = field | SORT x | LIMIT 10 var query = from("index").eval("x", b -> b.field("field")).sort("x").limit(10); @@ -105,6 +142,13 @@ public void testSimpleSortFieldAsAlias() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSimpleSortFieldAsAliasAndScore() { + // FROM index METADATA _score | EVAL x = field | SORT x, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false).eval("x", b -> b.field("field")).sort("x").scoreSort().limit(10); + assertPushdownSort(query, Map.of("x", "field"), List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSimpleSortFieldAndEvalSumLiterals() { // FROM index | EVAL sum = 1 + 2 | SORT field | LIMIT 10 var query = from("index").eval("sum", b -> b.add(b.i(1), b.i(2))).sort("field").limit(10); @@ -112,6 +156,17 @@ public void testSimpleSortFieldAndEvalSumLiterals() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSimpleSortFieldAndEvalSumLiteralsAndScore() { + // FROM index METADATA _score | EVAL sum = 1 + 2 | SORT field, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("sum", b -> b.add(b.i(1), b.i(2))) + .sort("field") + .scoreSort() + .limit(10); + assertPushdownSort(query, List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSimpleSortFieldAndEvalSumLiteralAndField() { // FROM index | EVAL sum = 1 + integer | SORT integer | LIMIT 10 var query = from("index").eval("sum", b -> b.add(b.i(1), b.field("integer"))).sort("integer").limit(10); @@ -119,6 +174,17 @@ public void testSimpleSortFieldAndEvalSumLiteralAndField() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSimpleSortFieldAndEvalSumLiteralAndFieldAndScore() { + // FROM index METADATA _score | EVAL sum = 1 + integer | SORT integer, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("sum", b -> b.add(b.i(1), b.field("integer"))) + .sort("integer") + .scoreSort() + .limit(10); + assertPushdownSort(query, List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSimpleSortEvalSumLiteralAndField() { // FROM index | EVAL sum = 1 + integer | SORT sum | LIMIT 10 var query = from("index").eval("sum", b -> b.add(b.i(1), b.field("integer"))).sort("sum").limit(10); @@ -144,6 +210,14 @@ public void testSortGeoPointField() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSortGeoPointFieldAnsScore() { + // FROM index METADATA _score | SORT location, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false).sort("location", Order.OrderDirection.ASC).scoreSort().limit(10); + // NOTE: while geo_point is not sortable, this is checked during logical planning and the physical planner does not know or care + assertPushdownSort(query); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSortGeoDistanceFunction() { // FROM index | EVAL distance = ST_DISTANCE(location, POINT(1 2)) | SORT distance | LIMIT 10 var query = from("index").eval("distance", b -> b.distance("location", "POINT(1 2)")) @@ -154,6 +228,18 @@ public void testSortGeoDistanceFunction() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSortGeoDistanceFunctionAndScore() { + // FROM index METADATA _score | EVAL distance = ST_DISTANCE(location, POINT(1 2)) | SORT distance, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("distance", b -> b.distance("location", "POINT(1 2)")) + .sort("distance", Order.OrderDirection.ASC) + .scoreSort() + .limit(10); + // The pushed-down sort will use the underlying field 'location', not the sorted reference field 'distance' + assertPushdownSort(query, Map.of("distance", "location"), List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSortGeoDistanceFunctionInverted() { // FROM index | EVAL distance = ST_DISTANCE(POINT(1 2), location) | SORT distance | LIMIT 10 var query = from("index").eval("distance", b -> b.distance("POINT(1 2)", "location")) @@ -164,6 +250,18 @@ public void testSortGeoDistanceFunctionInverted() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSortGeoDistanceFunctionInvertedAndScore() { + // FROM index METADATA _score | EVAL distance = ST_DISTANCE(POINT(1 2), location) | SORT distance, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("distance", b -> b.distance("POINT(1 2)", "location")) + .sort("distance", Order.OrderDirection.ASC) + .scoreSort() + .limit(10); + // The pushed-down sort will use the underlying field 'location', not the sorted reference field 'distance' + assertPushdownSort(query, Map.of("distance", "location"), List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSortGeoDistanceFunctionLiterals() { // FROM index | EVAL distance = ST_DISTANCE(POINT(2 1), POINT(1 2)) | SORT distance | LIMIT 10 var query = from("index").eval("distance", b -> b.distance("POINT(2 1)", "POINT(1 2)")) @@ -174,6 +272,18 @@ public void testSortGeoDistanceFunctionLiterals() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSortGeoDistanceFunctionLiteralsAndScore() { + // FROM index METADATA _score | EVAL distance = ST_DISTANCE(POINT(2 1), POINT(1 2)) | SORT distance, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("distance", b -> b.distance("POINT(2 1)", "POINT(1 2)")) + .sort("distance", Order.OrderDirection.ASC) + .scoreSort() + .limit(10); + // The pushed-down sort will use the underlying field 'location', not the sorted reference field 'distance' + assertNoPushdownSort(query, "sort on foldable distance function"); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSortGeoDistanceFunctionAndFieldsWithAliases() { // FROM index | EVAL distance = ST_DISTANCE(location, POINT(1 2)), x = field | SORT distance, field, integer | LIMIT 10 var query = from("index").eval("distance", b -> b.distance("location", "POINT(1 2)")) @@ -187,6 +297,21 @@ public void testSortGeoDistanceFunctionAndFieldsWithAliases() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSortGeoDistanceFunctionAndFieldsWithAliasesAndScore() { + // FROM index | EVAL distance = ST_DISTANCE(location, POINT(1 2)), x = field | SORT distance, field, integer, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("distance", b -> b.distance("location", "POINT(1 2)")) + .eval("x", b -> b.field("field")) + .sort("distance", Order.OrderDirection.ASC) + .sort("field", Order.OrderDirection.DESC) + .sort("integer", Order.OrderDirection.DESC) + .scoreSort() + .limit(10); + // The pushed-down sort will use the underlying field 'location', not the sorted reference field 'distance' + assertPushdownSort(query, query.orders, Map.of("distance", "location"), List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSortGeoDistanceFunctionAndFieldsAndAliases() { // FROM index | EVAL distance = ST_DISTANCE(location, POINT(1 2)), x = field | SORT distance, x, integer | LIMIT 10 var query = from("index").eval("distance", b -> b.distance("location", "POINT(1 2)")) @@ -200,6 +325,21 @@ public void testSortGeoDistanceFunctionAndFieldsAndAliases() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSortGeoDistanceFunctionAndFieldsAndAliasesAndScore() { + // FROM index | EVAL distance = ST_DISTANCE(location, POINT(1 2)), x = field | SORT distance, x, integer, _score | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("distance", b -> b.distance("location", "POINT(1 2)")) + .eval("x", b -> b.field("field")) + .sort("distance", Order.OrderDirection.ASC) + .sort("x", Order.OrderDirection.DESC) + .sort("integer", Order.OrderDirection.DESC) + .scoreSort() + .limit(10); + // The pushed-down sort will use the underlying field 'location', not the sorted reference field 'distance' + assertPushdownSort(query, query.orders, Map.of("distance", "location", "x", "field"), List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + public void testSortGeoDistanceFunctionAndFieldsAndManyAliases() { // FROM index // | EVAL loc = location, loc2 = loc, loc3 = loc2, distance = ST_DISTANCE(loc3, POINT(1 2)), x = field @@ -219,6 +359,27 @@ public void testSortGeoDistanceFunctionAndFieldsAndManyAliases() { assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); } + public void testSortGeoDistanceFunctionAndFieldsAndManyAliasesAndScore() { + // FROM index METADATA _score + // | EVAL loc = location, loc2 = loc, loc3 = loc2, distance = ST_DISTANCE(loc3, POINT(1 2)), x = field + // | SORT distance, x, integer, _score + // | LIMIT 10 + var query = from("index").metadata("_score", DOUBLE, false) + .eval("loc", b -> b.field("location")) + .eval("loc2", b -> b.ref("loc")) + .eval("loc3", b -> b.ref("loc2")) + .eval("distance", b -> b.distance("loc3", "POINT(1 2)")) + .eval("x", b -> b.field("field")) + .sort("distance", Order.OrderDirection.ASC) + .sort("x", Order.OrderDirection.DESC) + .sort("integer", Order.OrderDirection.DESC) + .scoreSort() + .limit(10); + // The pushed-down sort will use the underlying field 'location', not the sorted reference field 'distance' + assertPushdownSort(query, Map.of("distance", "location", "x", "field"), List.of(EvalExec.class, EsQueryExec.class)); + assertNoPushdownSort(query.asTimeSeries(), "for time series index mode"); + } + private static void assertPushdownSort(TestPhysicalPlanBuilder builder) { assertPushdownSort(builder, null, List.of(EsQueryExec.class)); } @@ -289,9 +450,12 @@ private static void assertPushdownSort( assertThat("Expect sorts count to match", sorts.size(), is(expectedSorts.size())); for (int i = 0; i < expectedSorts.size(); i++) { String name = ((Attribute) expectedSorts.get(i).child()).name(); - String fieldName = sorts.get(i).field().fieldName(); - assertThat("Expect sort[" + i + "] name to match", fieldName, is(sortName(name, fieldMap))); - assertThat("Expect sort[" + i + "] direction to match", sorts.get(i).direction(), is(expectedSorts.get(i).direction())); + EsQueryExec.Sort sort = sorts.get(i); + if (sort.field() != null) { + String fieldName = sort.field().fieldName(); + assertThat("Expect sort[" + i + "] name to match", fieldName, is(sortName(name, fieldMap))); + } + assertThat("Expect sort[" + i + "] direction to match", sort.direction(), is(expectedSorts.get(i).direction())); } } @@ -317,6 +481,7 @@ static class TestPhysicalPlanBuilder { private final String index; private final LinkedHashMap fields; private final LinkedHashMap refs; + private final LinkedHashMap metadata; private IndexMode indexMode; private final List aliases = new ArrayList<>(); private final List orders = new ArrayList<>(); @@ -327,6 +492,7 @@ private TestPhysicalPlanBuilder(String index, IndexMode indexMode) { this.indexMode = indexMode; this.fields = new LinkedHashMap<>(); this.refs = new LinkedHashMap<>(); + this.metadata = new LinkedHashMap<>(); addSortableFieldAttributes(this.fields); } @@ -346,6 +512,11 @@ static TestPhysicalPlanBuilder from(String index) { return new TestPhysicalPlanBuilder(index, IndexMode.STANDARD); } + TestPhysicalPlanBuilder metadata(String metadataAttribute, DataType dataType, boolean searchable) { + metadata.put(metadataAttribute, new MetadataAttribute(Source.EMPTY, metadataAttribute, dataType, searchable)); + return this; + } + public TestPhysicalPlanBuilder eval(Alias... aliases) { if (orders.isEmpty() == false) { throw new IllegalArgumentException("Eval must be before sort"); @@ -376,6 +547,22 @@ public TestPhysicalPlanBuilder sort(String field) { return sort(field, Order.OrderDirection.ASC); } + public TestPhysicalPlanBuilder scoreSort(Order.OrderDirection direction) { + orders.add( + new Order( + Source.EMPTY, + MetadataAttribute.create(Source.EMPTY, MetadataAttribute.SCORE), + direction, + Order.NullsPosition.LAST + ) + ); + return this; + } + + public TestPhysicalPlanBuilder scoreSort() { + return scoreSort(Order.OrderDirection.DESC); + } + public TestPhysicalPlanBuilder sort(String field, Order.OrderDirection direction) { Attribute attr = refs.get(field); if (attr == null) { From 6b94a91633fc846fe02ac8cf3173d613af27bc01 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Iv=C3=A1n=20Cea=20Fontenla?= Date: Thu, 28 Nov 2024 16:07:07 +0100 Subject: [PATCH 02/27] ESQL: Add nulls support to Categorize (#117655) Handle nulls and empty strings (Which resolve to null) on Categorize grouping function. Also, implement `seenGroupIds()`, which would fail some queries with nulls otherwise. --- docs/changelog/117655.yaml | 5 + .../AbstractCategorizeBlockHash.java | 37 +++++- .../blockhash/CategorizeRawBlockHash.java | 12 +- .../CategorizedIntermediateBlockHash.java | 19 ++- .../blockhash/CategorizeBlockHashTests.java | 72 +++++++---- .../src/main/resources/categorize.csv-spec | 122 ++++++++++-------- .../xpack/esql/action/EsqlCapabilities.java | 5 +- .../xpack/esql/analysis/VerifierTests.java | 6 +- .../optimizer/LogicalPlanOptimizerTests.java | 4 +- .../categorization/TokenListCategorizer.java | 2 + 10 files changed, 186 insertions(+), 98 deletions(-) create mode 100644 docs/changelog/117655.yaml diff --git a/docs/changelog/117655.yaml b/docs/changelog/117655.yaml new file mode 100644 index 0000000000000..f2afd3570f104 --- /dev/null +++ b/docs/changelog/117655.yaml @@ -0,0 +1,5 @@ +pr: 117655 +summary: Add nulls support to Categorize +area: ES|QL +type: enhancement +issues: [] diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractCategorizeBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractCategorizeBlockHash.java index 22d3a10facb06..0e89d77820883 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractCategorizeBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractCategorizeBlockHash.java @@ -13,8 +13,10 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BitArray; import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.compute.aggregation.SeenGroupIds; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.BytesRefBlock; import org.elasticsearch.compute.data.BytesRefVector; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; @@ -31,11 +33,21 @@ * Base BlockHash implementation for {@code Categorize} grouping function. */ public abstract class AbstractCategorizeBlockHash extends BlockHash { + protected static final int NULL_ORD = 0; + // TODO: this should probably also take an emitBatchSize private final int channel; private final boolean outputPartial; protected final TokenListCategorizer.CloseableTokenListCategorizer categorizer; + /** + * Store whether we've seen any {@code null} values. + *

+ * Null gets the {@link #NULL_ORD} ord. + *

+ */ + protected boolean seenNull = false; + AbstractCategorizeBlockHash(BlockFactory blockFactory, int channel, boolean outputPartial) { super(blockFactory); this.channel = channel; @@ -58,12 +70,12 @@ public Block[] getKeys() { @Override public IntVector nonEmpty() { - return IntVector.range(0, categorizer.getCategoryCount(), blockFactory); + return IntVector.range(seenNull ? 0 : 1, categorizer.getCategoryCount() + 1, blockFactory); } @Override public BitArray seenGroupIds(BigArrays bigArrays) { - throw new UnsupportedOperationException(); + return new SeenGroupIds.Range(seenNull ? 0 : 1, Math.toIntExact(categorizer.getCategoryCount() + 1)).seenGroupIds(bigArrays); } @Override @@ -76,24 +88,39 @@ public final ReleasableIterator lookup(Page page, ByteSizeValue target */ private Block buildIntermediateBlock() { if (categorizer.getCategoryCount() == 0) { - return blockFactory.newConstantNullBlock(0); + return blockFactory.newConstantNullBlock(seenNull ? 1 : 0); } try (BytesStreamOutput out = new BytesStreamOutput()) { // TODO be more careful here. + out.writeBoolean(seenNull); out.writeVInt(categorizer.getCategoryCount()); for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { category.writeTo(out); } // We're returning a block with N positions just because the Page must have all blocks with the same position count! - return blockFactory.newConstantBytesRefBlockWith(out.bytes().toBytesRef(), categorizer.getCategoryCount()); + int positionCount = categorizer.getCategoryCount() + (seenNull ? 1 : 0); + return blockFactory.newConstantBytesRefBlockWith(out.bytes().toBytesRef(), positionCount); } catch (IOException e) { throw new RuntimeException(e); } } private Block buildFinalBlock() { + BytesRefBuilder scratch = new BytesRefBuilder(); + + if (seenNull) { + try (BytesRefBlock.Builder result = blockFactory.newBytesRefBlockBuilder(categorizer.getCategoryCount())) { + result.appendNull(); + for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { + scratch.copyChars(category.getRegex()); + result.appendBytesRef(scratch.get()); + scratch.clear(); + } + return result.build(); + } + } + try (BytesRefVector.Builder result = blockFactory.newBytesRefVectorBuilder(categorizer.getCategoryCount())) { - BytesRefBuilder scratch = new BytesRefBuilder(); for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { scratch.copyChars(category.getRegex()); result.appendBytesRef(scratch.get()); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java index bf633e0454384..0d0a2fef2f82b 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java @@ -64,7 +64,7 @@ public void close() { /** * Similar implementation to an Evaluator. */ - public static final class CategorizeEvaluator implements Releasable { + public final class CategorizeEvaluator implements Releasable { private final CategorizationAnalyzer analyzer; private final TokenListCategorizer.CloseableTokenListCategorizer categorizer; @@ -95,7 +95,8 @@ public IntBlock eval(int positionCount, BytesRefBlock vBlock) { BytesRef vScratch = new BytesRef(); for (int p = 0; p < positionCount; p++) { if (vBlock.isNull(p)) { - result.appendNull(); + seenNull = true; + result.appendInt(NULL_ORD); continue; } int first = vBlock.getFirstValueIndex(p); @@ -126,7 +127,12 @@ public IntVector eval(int positionCount, BytesRefVector vVector) { } private int process(BytesRef v) { - return categorizer.computeCategory(v.utf8ToString(), analyzer).getId(); + var category = categorizer.computeCategory(v.utf8ToString(), analyzer); + if (category == null) { + seenNull = true; + return NULL_ORD; + } + return category.getId() + 1; } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizedIntermediateBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizedIntermediateBlockHash.java index 1bca34a70e5fa..c774d3b26049d 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizedIntermediateBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizedIntermediateBlockHash.java @@ -40,9 +40,19 @@ public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { return; } BytesRefBlock categorizerState = page.getBlock(channel()); + if (categorizerState.areAllValuesNull()) { + seenNull = true; + try (var newIds = blockFactory.newConstantIntVector(NULL_ORD, 1)) { + addInput.add(0, newIds); + } + return; + } + Map idMap = readIntermediate(categorizerState.getBytesRef(0, new BytesRef())); try (IntBlock.Builder newIdsBuilder = blockFactory.newIntBlockBuilder(idMap.size())) { - for (int i = 0; i < idMap.size(); i++) { + int fromId = idMap.containsKey(0) ? 0 : 1; + int toId = fromId + idMap.size(); + for (int i = fromId; i < toId; i++) { newIdsBuilder.appendInt(idMap.get(i)); } try (IntBlock newIds = newIdsBuilder.build()) { @@ -59,10 +69,15 @@ public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { private Map readIntermediate(BytesRef bytes) { Map idMap = new HashMap<>(); try (StreamInput in = new BytesArray(bytes).streamInput()) { + if (in.readBoolean()) { + seenNull = true; + idMap.put(NULL_ORD, NULL_ORD); + } int count = in.readVInt(); for (int oldCategoryId = 0; oldCategoryId < count; oldCategoryId++) { int newCategoryId = categorizer.mergeWireCategory(new SerializableTokenListCategory(in)).getId(); - idMap.put(oldCategoryId, newCategoryId); + // +1 because the 0 ordinal is reserved for null + idMap.put(oldCategoryId + 1, newCategoryId + 1); } return idMap; } catch (IOException e) { diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java index de8a2a44266fe..dd7a87dc4a574 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java @@ -52,7 +52,8 @@ public class CategorizeBlockHashTests extends BlockHashTestCase { public void testCategorizeRaw() { final Page page; - final int positions = 7; + boolean withNull = randomBoolean(); + final int positions = 7 + (withNull ? 1 : 0); try (BytesRefBlock.Builder builder = blockFactory.newBytesRefBlockBuilder(positions)) { builder.appendBytesRef(new BytesRef("Connected to 10.1.0.1")); builder.appendBytesRef(new BytesRef("Connection error")); @@ -61,6 +62,13 @@ public void testCategorizeRaw() { builder.appendBytesRef(new BytesRef("Disconnected")); builder.appendBytesRef(new BytesRef("Connected to 10.1.0.2")); builder.appendBytesRef(new BytesRef("Connected to 10.1.0.3")); + if (withNull) { + if (randomBoolean()) { + builder.appendNull(); + } else { + builder.appendBytesRef(new BytesRef("")); + } + } page = new Page(builder.build()); } @@ -70,13 +78,16 @@ public void testCategorizeRaw() { public void add(int positionOffset, IntBlock groupIds) { assertEquals(groupIds.getPositionCount(), positions); - assertEquals(0, groupIds.getInt(0)); - assertEquals(1, groupIds.getInt(1)); - assertEquals(1, groupIds.getInt(2)); - assertEquals(1, groupIds.getInt(3)); - assertEquals(2, groupIds.getInt(4)); - assertEquals(0, groupIds.getInt(5)); - assertEquals(0, groupIds.getInt(6)); + assertEquals(1, groupIds.getInt(0)); + assertEquals(2, groupIds.getInt(1)); + assertEquals(2, groupIds.getInt(2)); + assertEquals(2, groupIds.getInt(3)); + assertEquals(3, groupIds.getInt(4)); + assertEquals(1, groupIds.getInt(5)); + assertEquals(1, groupIds.getInt(6)); + if (withNull) { + assertEquals(0, groupIds.getInt(7)); + } } @Override @@ -100,7 +111,8 @@ public void close() { public void testCategorizeIntermediate() { Page page1; - int positions1 = 7; + boolean withNull = randomBoolean(); + int positions1 = 7 + (withNull ? 1 : 0); try (BytesRefBlock.Builder builder = blockFactory.newBytesRefBlockBuilder(positions1)) { builder.appendBytesRef(new BytesRef("Connected to 10.1.0.1")); builder.appendBytesRef(new BytesRef("Connection error")); @@ -109,6 +121,13 @@ public void testCategorizeIntermediate() { builder.appendBytesRef(new BytesRef("Connection error")); builder.appendBytesRef(new BytesRef("Connected to 10.1.0.3")); builder.appendBytesRef(new BytesRef("Connected to 10.1.0.4")); + if (withNull) { + if (randomBoolean()) { + builder.appendNull(); + } else { + builder.appendBytesRef(new BytesRef("")); + } + } page1 = new Page(builder.build()); } Page page2; @@ -133,13 +152,16 @@ public void testCategorizeIntermediate() { @Override public void add(int positionOffset, IntBlock groupIds) { assertEquals(groupIds.getPositionCount(), positions1); - assertEquals(0, groupIds.getInt(0)); - assertEquals(1, groupIds.getInt(1)); - assertEquals(1, groupIds.getInt(2)); - assertEquals(0, groupIds.getInt(3)); - assertEquals(1, groupIds.getInt(4)); - assertEquals(0, groupIds.getInt(5)); - assertEquals(0, groupIds.getInt(6)); + assertEquals(1, groupIds.getInt(0)); + assertEquals(2, groupIds.getInt(1)); + assertEquals(2, groupIds.getInt(2)); + assertEquals(1, groupIds.getInt(3)); + assertEquals(2, groupIds.getInt(4)); + assertEquals(1, groupIds.getInt(5)); + assertEquals(1, groupIds.getInt(6)); + if (withNull) { + assertEquals(0, groupIds.getInt(7)); + } } @Override @@ -158,11 +180,11 @@ public void close() { @Override public void add(int positionOffset, IntBlock groupIds) { assertEquals(groupIds.getPositionCount(), positions2); - assertEquals(0, groupIds.getInt(0)); - assertEquals(1, groupIds.getInt(1)); - assertEquals(0, groupIds.getInt(2)); - assertEquals(1, groupIds.getInt(3)); - assertEquals(2, groupIds.getInt(4)); + assertEquals(1, groupIds.getInt(0)); + assertEquals(2, groupIds.getInt(1)); + assertEquals(1, groupIds.getInt(2)); + assertEquals(2, groupIds.getInt(3)); + assertEquals(3, groupIds.getInt(4)); } @Override @@ -189,7 +211,11 @@ public void add(int positionOffset, IntBlock groupIds) { .map(groupIds::getInt) .boxed() .collect(Collectors.toSet()); - assertEquals(values, Set.of(0, 1)); + if (withNull) { + assertEquals(Set.of(0, 1, 2), values); + } else { + assertEquals(Set.of(1, 2), values); + } } @Override @@ -212,7 +238,7 @@ public void add(int positionOffset, IntBlock groupIds) { .collect(Collectors.toSet()); // The category IDs {0, 1, 2} should map to groups {0, 2, 3}, because // 0 matches an existing category (Connected to ...), and the others are new. - assertEquals(values, Set.of(0, 2, 3)); + assertEquals(Set.of(1, 3, 4), values); } @Override diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/categorize.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/categorize.csv-spec index 89d9026423204..547c430ed7518 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/categorize.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/categorize.csv-spec @@ -1,5 +1,5 @@ standard aggs -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS count=COUNT(), @@ -17,7 +17,7 @@ count:long | sum:long | avg:double | count_distinct:long | category:keyw ; values aggs -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS values=MV_SORT(VALUES(message)), @@ -33,7 +33,7 @@ values:keyword | top ; mv -required_capability: categorize_v2 +required_capability: categorize_v3 FROM mv_sample_data | STATS COUNT(), SUM(event_duration) BY category=CATEGORIZE(message) @@ -48,7 +48,7 @@ COUNT():long | SUM(event_duration):long | category:keyword ; row mv -required_capability: categorize_v2 +required_capability: categorize_v3 ROW message = ["connected to a", "connected to b", "disconnected"], str = ["a", "b", "c"] | STATS COUNT(), VALUES(str) BY category=CATEGORIZE(message) @@ -61,7 +61,7 @@ COUNT():long | VALUES(str):keyword | category:keyword ; with multiple indices -required_capability: categorize_v2 +required_capability: categorize_v3 required_capability: union_types FROM sample_data* @@ -76,7 +76,7 @@ COUNT():long | category:keyword ; mv with many values -required_capability: categorize_v2 +required_capability: categorize_v3 FROM employees | STATS COUNT() BY category=CATEGORIZE(job_positions) @@ -92,24 +92,37 @@ COUNT():long | category:keyword 10 | .*?Head.+?Human.+?Resources.*? ; -# Throws when calling AbstractCategorizeBlockHash.seenGroupIds() - Requires nulls support? -mv with many values-Ignore -required_capability: categorize_v2 +mv with many values and SUM +required_capability: categorize_v3 FROM employees | STATS SUM(languages) BY category=CATEGORIZE(job_positions) - | SORT category DESC + | SORT category | LIMIT 3 ; -SUM(languages):integer | category:keyword - 43 | .*?Accountant.*? - 46 | .*?Architect.*? - 35 | .*?Business.+?Analyst.*? +SUM(languages):long | category:keyword + 43 | .*?Accountant.*? + 46 | .*?Architect.*? + 35 | .*?Business.+?Analyst.*? +; + +mv with many values and nulls and SUM +required_capability: categorize_v3 + +FROM employees + | STATS SUM(languages) BY category=CATEGORIZE(job_positions) + | SORT category DESC + | LIMIT 2 +; + +SUM(languages):long | category:keyword + 27 | null + 46 | .*?Tech.+?Lead.*? ; mv via eval -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | EVAL message = MV_APPEND(message, "Banana") @@ -125,7 +138,7 @@ COUNT():long | category:keyword ; mv via eval const -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | EVAL message = ["Banana", "Bread"] @@ -139,7 +152,7 @@ COUNT():long | category:keyword ; mv via eval const without aliases -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | EVAL message = ["Banana", "Bread"] @@ -153,7 +166,7 @@ COUNT():long | CATEGORIZE(message):keyword ; mv const in parameter -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY c = CATEGORIZE(["Banana", "Bread"]) @@ -166,7 +179,7 @@ COUNT():long | c:keyword ; agg alias shadowing -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS c = COUNT() BY c = CATEGORIZE(["Banana", "Bread"]) @@ -181,7 +194,7 @@ c:keyword ; chained aggregations using categorize -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(message) @@ -196,7 +209,7 @@ COUNT():long | category:keyword ; stats without aggs -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS BY category=CATEGORIZE(message) @@ -210,7 +223,7 @@ category:keyword ; text field -required_capability: categorize_v2 +required_capability: categorize_v3 FROM hosts | STATS COUNT() BY category=CATEGORIZE(host_group) @@ -221,10 +234,11 @@ COUNT():long | category:keyword 2 | .*?DB.+?servers.*? 2 | .*?Gateway.+?instances.*? 5 | .*?Kubernetes.+?cluster.*? + 1 | null ; on TO_UPPER -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(TO_UPPER(message)) @@ -238,7 +252,7 @@ COUNT():long | category:keyword ; on CONCAT -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(CONCAT(message, " banana")) @@ -252,7 +266,7 @@ COUNT():long | category:keyword ; on CONCAT with unicode -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(CONCAT(message, " 👍🏽😊")) @@ -266,7 +280,7 @@ COUNT():long | category:keyword ; on REVERSE(CONCAT()) -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(REVERSE(CONCAT(message, " 👍🏽😊"))) @@ -280,7 +294,7 @@ COUNT():long | category:keyword ; and then TO_LOWER -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(message) @@ -294,9 +308,8 @@ COUNT():long | category:keyword 1 | .*?disconnected.*? ; -# Throws NPE - Requires nulls support -on const empty string-Ignore -required_capability: categorize_v2 +on const empty string +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY category=CATEGORIZE("") @@ -304,12 +317,11 @@ FROM sample_data ; COUNT():long | category:keyword - 7 | .*?.*? + 7 | null ; -# Throws NPE - Requires nulls support -on const empty string from eval-Ignore -required_capability: categorize_v2 +on const empty string from eval +required_capability: categorize_v3 FROM sample_data | EVAL x = "" @@ -318,26 +330,24 @@ FROM sample_data ; COUNT():long | category:keyword - 7 | .*?.*? + 7 | null ; -# Doesn't give the correct results - Requires nulls support -on null-Ignore -required_capability: categorize_v2 +on null +required_capability: categorize_v3 FROM sample_data | EVAL x = null - | STATS COUNT() BY category=CATEGORIZE(x) + | STATS COUNT(), SUM(event_duration) BY category=CATEGORIZE(x) | SORT category ; -COUNT():long | category:keyword - 7 | null +COUNT():long | SUM(event_duration):long | category:keyword + 7 | 23231327 | null ; -# Doesn't give the correct results - Requires nulls support -on null string-Ignore -required_capability: categorize_v2 +on null string +required_capability: categorize_v3 FROM sample_data | EVAL x = null::string @@ -350,7 +360,7 @@ COUNT():long | category:keyword ; filtering out all data -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | WHERE @timestamp < "2023-10-23T00:00:00Z" @@ -362,7 +372,7 @@ COUNT():long | category:keyword ; filtering out all data with constant -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(message) @@ -373,7 +383,7 @@ COUNT():long | category:keyword ; drop output columns -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS count=COUNT() BY category=CATEGORIZE(message) @@ -388,7 +398,7 @@ x:integer ; category value processing -required_capability: categorize_v2 +required_capability: categorize_v3 ROW message = ["connected to a", "connected to b", "disconnected"] | STATS COUNT() BY category=CATEGORIZE(message) @@ -402,7 +412,7 @@ COUNT():long | category:keyword ; row aliases -required_capability: categorize_v2 +required_capability: categorize_v3 ROW message = "connected to a" | EVAL x = message @@ -416,7 +426,7 @@ COUNT():long | category:keyword | y:keyword ; from aliases -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | EVAL x = message @@ -432,7 +442,7 @@ COUNT():long | category:keyword | y:keyword ; row aliases with keep -required_capability: categorize_v2 +required_capability: categorize_v3 ROW message = "connected to a" | EVAL x = message @@ -448,7 +458,7 @@ COUNT():long | y:keyword ; from aliases with keep -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | EVAL x = message @@ -466,7 +476,7 @@ COUNT():long | y:keyword ; row rename -required_capability: categorize_v2 +required_capability: categorize_v3 ROW message = "connected to a" | RENAME message as x @@ -480,7 +490,7 @@ COUNT():long | y:keyword ; from rename -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | RENAME message as x @@ -496,7 +506,7 @@ COUNT():long | y:keyword ; row drop -required_capability: categorize_v2 +required_capability: categorize_v3 ROW message = "connected to a" | STATS c = COUNT() BY category=CATEGORIZE(message) @@ -509,7 +519,7 @@ c:long ; from drop -required_capability: categorize_v2 +required_capability: categorize_v3 FROM sample_data | STATS c = COUNT() BY category=CATEGORIZE(message) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 9bd4211855699..77a3e2840977f 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -402,11 +402,8 @@ public enum Cap { /** * Supported the text categorization function "CATEGORIZE". - *

- * This capability was initially named `CATEGORIZE`, and got renamed after the function started correctly returning keywords. - *

*/ - CATEGORIZE_V2(Build.current().isSnapshot()), + CATEGORIZE_V3(Build.current().isSnapshot()), /** * QSTR function diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 6074601535477..dd14e8dd82123 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1846,7 +1846,7 @@ public void testIntervalAsString() { } public void testCategorizeSingleGrouping() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V2.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); query("from test | STATS COUNT(*) BY CATEGORIZE(first_name)"); query("from test | STATS COUNT(*) BY cat = CATEGORIZE(first_name)"); @@ -1875,7 +1875,7 @@ public void testCategorizeSingleGrouping() { } public void testCategorizeNestedGrouping() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V2.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); query("from test | STATS COUNT(*) BY CATEGORIZE(LENGTH(first_name)::string)"); @@ -1890,7 +1890,7 @@ public void testCategorizeNestedGrouping() { } public void testCategorizeWithinAggregations() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V2.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); query("from test | STATS MV_COUNT(cat), COUNT(*) BY cat = CATEGORIZE(first_name)"); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index 8373528531902..e98f2b88b33c9 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -1212,7 +1212,7 @@ public void testCombineProjectionWithAggregationFirstAndAliasedGroupingUsedInAgg * \_EsRelation[test][_meta_field{f}#23, emp_no{f}#17, first_name{f}#18, ..] */ public void testCombineProjectionWithCategorizeGrouping() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V2.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); var plan = plan(""" from test @@ -3949,7 +3949,7 @@ public void testNestedExpressionsInGroups() { * \_EsRelation[test][_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, ge..] */ public void testNestedExpressionsInGroupsWithCategorize() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V2.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); var plan = optimizedPlan(""" from test diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/TokenListCategorizer.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/TokenListCategorizer.java index e4257270ce641..7fef6cdafa372 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/TokenListCategorizer.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/aggs/categorization/TokenListCategorizer.java @@ -115,6 +115,7 @@ public TokenListCategorizer( cacheRamUsage(0); } + @Nullable public TokenListCategory computeCategory(String s, CategorizationAnalyzer analyzer) { try (TokenStream ts = analyzer.tokenStream("text", s)) { return computeCategory(ts, s.length(), 1); @@ -123,6 +124,7 @@ public TokenListCategory computeCategory(String s, CategorizationAnalyzer analyz } } + @Nullable public TokenListCategory computeCategory(TokenStream ts, int unfilteredStringLen, long numDocs) throws IOException { assert partOfSpeechDictionary != null : "This version of computeCategory should only be used when a part-of-speech dictionary is available"; From 3c70cd081d40c36a5ac375b009932a0ce5eff1bd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mariusz=20J=C3=B3zala?= <377355+jozala@users.noreply.github.com> Date: Thu, 28 Nov 2024 16:20:05 +0100 Subject: [PATCH 03/27] Revert "[CI] Ignore error about missing UBI artifact (#117506)" (#117704) This reverts commit 219372efaaf46a3b496df2142d3091d3434e67ec. This ignore is no longer necessary since the change to release-manager has been applied. --- .buildkite/scripts/dra-workflow.sh | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/.buildkite/scripts/dra-workflow.sh b/.buildkite/scripts/dra-workflow.sh index bbfa81f51b286..f2dc40ca1927f 100755 --- a/.buildkite/scripts/dra-workflow.sh +++ b/.buildkite/scripts/dra-workflow.sh @@ -75,7 +75,6 @@ find "$WORKSPACE" -type d -path "*/build/distributions" -exec chmod a+w {} \; echo --- Running release-manager -set +e # Artifacts should be generated docker run --rm \ --name release-manager \ @@ -92,16 +91,4 @@ docker run --rm \ --version "$ES_VERSION" \ --artifact-set main \ --dependency "beats:https://artifacts-${WORKFLOW}.elastic.co/beats/${BEATS_BUILD_ID}/manifest-${ES_VERSION}${VERSION_SUFFIX}.json" \ - --dependency "ml-cpp:https://artifacts-${WORKFLOW}.elastic.co/ml-cpp/${ML_CPP_BUILD_ID}/manifest-${ES_VERSION}${VERSION_SUFFIX}.json" \ -2>&1 | tee release-manager.log -EXIT_CODE=$? -set -e - -# This failure is just generating a ton of noise right now, so let's just ignore it -# This should be removed once this issue has been fixed -if grep "elasticsearch-ubi-9.0.0-SNAPSHOT-docker-image.tar.gz" release-manager.log; then - echo "Ignoring error about missing ubi artifact" - exit 0 -fi - -exit "$EXIT_CODE" + --dependency "ml-cpp:https://artifacts-${WORKFLOW}.elastic.co/ml-cpp/${ML_CPP_BUILD_ID}/manifest-${ES_VERSION}${VERSION_SUFFIX}.json" From 54db9470207df11f07475a6e8d4837b29515a4d7 Mon Sep 17 00:00:00 2001 From: Oleksandr Kolomiiets Date: Thu, 28 Nov 2024 07:33:35 -0800 Subject: [PATCH 04/27] Fix scaled_float test (#117662) --- .../index/mapper/extras/ScaledFloatFieldMapperTests.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapperTests.java index dc9bc96f107a0..83fe07170d6e7 100644 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/extras/ScaledFloatFieldMapperTests.java @@ -527,7 +527,13 @@ protected Number randomNumber() { public void testEncodeDecodeExactScalingFactor() { double v = randomValue(); - assertThat(encodeDecode(1 / v, v), equalTo(1 / v)); + double expected = 1 / v; + // We don't produce infinities while decoding. See #testDecodeHandlingInfinity(). + if (Double.isInfinite(expected)) { + var sign = expected == Double.POSITIVE_INFINITY ? 1 : -1; + expected = sign * Double.MAX_VALUE; + } + assertThat(encodeDecode(1 / v, v), equalTo(expected)); } /** From ab604ada78d779a18b82465d51829006540ce546 Mon Sep 17 00:00:00 2001 From: Liam Thompson <32779855+leemthompo@users.noreply.github.com> Date: Thu, 28 Nov 2024 16:34:57 +0100 Subject: [PATCH 05/27] [DOCS] Update tutorial example (#117538) --- .../full-text-filtering-tutorial.asciidoc | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/docs/reference/quickstart/full-text-filtering-tutorial.asciidoc b/docs/reference/quickstart/full-text-filtering-tutorial.asciidoc index fee4b797da724..a024305588cae 100644 --- a/docs/reference/quickstart/full-text-filtering-tutorial.asciidoc +++ b/docs/reference/quickstart/full-text-filtering-tutorial.asciidoc @@ -511,8 +511,9 @@ In this tutorial scenario it's useful for when users have complex requirements f Let's create a query that addresses the following user needs: -* Must be a vegetarian main course +* Must be a vegetarian recipe * Should contain "curry" or "spicy" in the title or description +* Should be a main course * Must not be a dessert * Must have a rating of at least 4.5 * Should prefer recipes published in the last month @@ -524,16 +525,7 @@ GET /cooking_blog/_search "query": { "bool": { "must": [ - { - "term": { - "category.keyword": "Main Course" - } - }, - { - "term": { - "tags": "vegetarian" - } - }, + { "term": { "tags": "vegetarian" } }, { "range": { "rating": { @@ -543,10 +535,18 @@ GET /cooking_blog/_search } ], "should": [ + { + "term": { + "category": "Main Course" + } + }, { "multi_match": { "query": "curry spicy", - "fields": ["title^2", "description"] + "fields": [ + "title^2", + "description" + ] } }, { @@ -590,12 +590,12 @@ GET /cooking_blog/_search "value": 1, "relation": "eq" }, - "max_score": 7.9835095, + "max_score": 7.444513, "hits": [ { "_index": "cooking_blog", "_id": "2", - "_score": 7.9835095, + "_score": 7.444513, "_source": { "title": "Spicy Thai Green Curry: A Vegetarian Adventure", <1> "description": "Dive into the flavors of Thailand with this vibrant green curry. Packed with vegetables and aromatic herbs, this dish is both healthy and satisfying. Don't worry about the heat - you can easily adjust the spice level to your liking.", <2> @@ -619,8 +619,8 @@ GET /cooking_blog/_search <1> The title contains "Spicy" and "Curry", matching our should condition. With the default <> behavior, this field contributes most to the relevance score. <2> While the description also contains matching terms, only the best matching field's score is used by default. <3> The recipe was published within the last month, satisfying our recency preference. -<4> The "Main Course" category matches our `must` condition. -<5> The "vegetarian" tag satisfies another `must` condition, while "curry" and "spicy" tags align with our `should` preferences. +<4> The "Main Course" category satisfies another `should` condition. +<5> The "vegetarian" tag satisfies a `must` condition, while "curry" and "spicy" tags align with our `should` preferences. <6> The rating of 4.6 meets our minimum rating requirement of 4.5. ============== From f096c317c06052dc26c00b72448eda4743ab5965 Mon Sep 17 00:00:00 2001 From: Dimitris Rempapis Date: Thu, 28 Nov 2024 19:38:37 +0200 Subject: [PATCH 06/27] fix/SearchStatesIt_failures (#117618) Investigate and unmute automatically muted tests --- docs/changelog/117618.yaml | 5 +++++ muted-tests.yml | 6 ------ 2 files changed, 5 insertions(+), 6 deletions(-) create mode 100644 docs/changelog/117618.yaml diff --git a/docs/changelog/117618.yaml b/docs/changelog/117618.yaml new file mode 100644 index 0000000000000..5de29e2fe768c --- /dev/null +++ b/docs/changelog/117618.yaml @@ -0,0 +1,5 @@ +pr: 117618 +summary: SearchStatesIt failures reported by CI +area: Search +type: bug +issues: [116617, 116618] diff --git a/muted-tests.yml b/muted-tests.yml index fdadc747289bb..d703cfaa1b9aa 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -156,12 +156,6 @@ tests: - class: org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshotsCanMatchOnCoordinatorIntegTests method: testSearchableSnapshotShardsAreSkippedBySearchRequestWithoutQueryingAnyNodeWhenTheyAreOutsideOfTheQueryRange issue: https://github.com/elastic/elasticsearch/issues/116523 -- class: org.elasticsearch.upgrades.SearchStatesIT - method: testBWCSearchStates - issue: https://github.com/elastic/elasticsearch/issues/116617 -- class: org.elasticsearch.upgrades.SearchStatesIT - method: testCanMatch - issue: https://github.com/elastic/elasticsearch/issues/116618 - class: org.elasticsearch.reservedstate.service.RepositoriesFileSettingsIT method: testSettingsApplied issue: https://github.com/elastic/elasticsearch/issues/116694 From 8350ff29ba18c7d03d652b107532415705426da9 Mon Sep 17 00:00:00 2001 From: John Verwolf Date: Thu, 28 Nov 2024 13:25:02 -0800 Subject: [PATCH 07/27] Extensible Completion Postings Formats (#111494) Allows the Completion Postings Format to be extensible by providing an implementation of the CompletionsPostingsFormatExtension SPIs. --- docs/changelog/111494.yaml | 5 ++++ server/src/main/java/module-info.java | 6 +++- .../index/codec/PerFieldFormatSupplier.java | 24 ++++++++++++++-- .../index/mapper/CompletionFieldMapper.java | 5 ---- .../index/mapper/MappingLookup.java | 17 ----------- .../CompletionsPostingsFormatExtension.java | 28 +++++++++++++++++++ 6 files changed, 59 insertions(+), 26 deletions(-) create mode 100644 docs/changelog/111494.yaml create mode 100644 server/src/main/java/org/elasticsearch/internal/CompletionsPostingsFormatExtension.java diff --git a/docs/changelog/111494.yaml b/docs/changelog/111494.yaml new file mode 100644 index 0000000000000..6c7b84bb04798 --- /dev/null +++ b/docs/changelog/111494.yaml @@ -0,0 +1,5 @@ +pr: 111494 +summary: Extensible Completion Postings Formats +area: "Suggesters" +type: enhancement +issues: [] diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index 63dbac3a72487..d572d3b90fec8 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -7,6 +7,7 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ +import org.elasticsearch.internal.CompletionsPostingsFormatExtension; import org.elasticsearch.plugins.internal.RestExtension; /** The Elasticsearch Server Module. */ @@ -288,7 +289,8 @@ to org.elasticsearch.serverless.version, org.elasticsearch.serverless.buildinfo, - org.elasticsearch.serverless.constants; + org.elasticsearch.serverless.constants, + org.elasticsearch.serverless.codec; exports org.elasticsearch.lucene.analysis.miscellaneous; exports org.elasticsearch.lucene.grouping; exports org.elasticsearch.lucene.queries; @@ -395,6 +397,7 @@ org.elasticsearch.stateless, org.elasticsearch.settings.secure, org.elasticsearch.serverless.constants, + org.elasticsearch.serverless.codec, org.elasticsearch.serverless.apifiltering, org.elasticsearch.internal.security; @@ -414,6 +417,7 @@ uses org.elasticsearch.node.internal.TerminationHandlerProvider; uses org.elasticsearch.internal.VersionExtension; uses org.elasticsearch.internal.BuildExtension; + uses CompletionsPostingsFormatExtension; uses org.elasticsearch.features.FeatureSpecification; uses org.elasticsearch.plugins.internal.LoggingDataProvider; diff --git a/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java b/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java index 9c2a08a69002c..4d3d37ab4f3af 100644 --- a/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java +++ b/server/src/main/java/org/elasticsearch/index/codec/PerFieldFormatSupplier.java @@ -20,10 +20,15 @@ import org.elasticsearch.index.codec.bloomfilter.ES87BloomFilterPostingsFormat; import org.elasticsearch.index.codec.postings.ES812PostingsFormat; import org.elasticsearch.index.codec.tsdb.ES87TSDBDocValuesFormat; +import org.elasticsearch.index.mapper.CompletionFieldMapper; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.Mapper; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.vectors.DenseVectorFieldMapper; +import org.elasticsearch.internal.CompletionsPostingsFormatExtension; +import org.elasticsearch.plugins.ExtensionLoader; + +import java.util.ServiceLoader; /** * Class that encapsulates the logic of figuring out the most appropriate file format for a given field, across postings, doc values and @@ -53,15 +58,28 @@ public PostingsFormat getPostingsFormatForField(String field) { private PostingsFormat internalGetPostingsFormatForField(String field) { if (mapperService != null) { - final PostingsFormat format = mapperService.mappingLookup().getPostingsFormat(field); - if (format != null) { - return format; + Mapper mapper = mapperService.mappingLookup().getMapper(field); + if (mapper instanceof CompletionFieldMapper) { + return PostingsFormatHolder.POSTINGS_FORMAT; } } // return our own posting format using PFOR return es812PostingsFormat; } + private static class PostingsFormatHolder { + private static final PostingsFormat POSTINGS_FORMAT = getPostingsFormat(); + + private static PostingsFormat getPostingsFormat() { + String defaultName = "Completion912"; // Caution: changing this name will result in exceptions if a field is created during a + // rolling upgrade and the new codec (specified by the name) is not available on all nodes in the cluster. + String codecName = ExtensionLoader.loadSingleton(ServiceLoader.load(CompletionsPostingsFormatExtension.class)) + .map(CompletionsPostingsFormatExtension::getFormatName) + .orElse(defaultName); + return PostingsFormat.forName(codecName); + } + } + boolean useBloomFilter(String field) { if (mapperService == null) { return false; diff --git a/server/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper.java index 53ccccdbd4bab..bb229c795a83e 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/CompletionFieldMapper.java @@ -8,7 +8,6 @@ */ package org.elasticsearch.index.mapper; -import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.document.FieldType; import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.Term; @@ -344,10 +343,6 @@ public CompletionFieldType fieldType() { return (CompletionFieldType) super.fieldType(); } - static PostingsFormat postingsFormat() { - return PostingsFormat.forName("Completion912"); - } - @Override public boolean parsesArrayValue() { return true; diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java index 2f78e11761448..ce3f8cfb53184 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MappingLookup.java @@ -9,7 +9,6 @@ package org.elasticsearch.index.mapper; -import org.apache.lucene.codecs.PostingsFormat; import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.InferenceFieldMetadata; import org.elasticsearch.index.IndexSettings; @@ -21,7 +20,6 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -58,7 +56,6 @@ private CacheKey() {} private final Map indexAnalyzersMap; private final List indexTimeScriptMappers; private final Mapping mapping; - private final Set completionFields; private final int totalFieldsCount; /** @@ -161,7 +158,6 @@ private MappingLookup( this.nestedLookup = NestedLookup.build(nestedMappers); final Map indexAnalyzersMap = new HashMap<>(); - final Set completionFields = new HashSet<>(); final List indexTimeScriptMappers = new ArrayList<>(); for (FieldMapper mapper : mappers) { if (objects.containsKey(mapper.fullPath())) { @@ -174,9 +170,6 @@ private MappingLookup( if (mapper.hasScript()) { indexTimeScriptMappers.add(mapper); } - if (mapper instanceof CompletionFieldMapper) { - completionFields.add(mapper.fullPath()); - } } for (FieldAliasMapper aliasMapper : aliasMappers) { @@ -211,7 +204,6 @@ private MappingLookup( this.objectMappers = Map.copyOf(objects); this.runtimeFieldMappersCount = runtimeFields.size(); this.indexAnalyzersMap = Map.copyOf(indexAnalyzersMap); - this.completionFields = Set.copyOf(completionFields); this.indexTimeScriptMappers = List.copyOf(indexTimeScriptMappers); runtimeFields.stream().flatMap(RuntimeField::asMappedFieldTypes).map(MappedFieldType::name).forEach(this::validateDoesNotShadow); @@ -285,15 +277,6 @@ public Iterable fieldMappers() { return fieldMappers.values(); } - /** - * Gets the postings format for a particular field - * @param field the field to retrieve a postings format for - * @return the postings format for the field, or {@code null} if the default format should be used - */ - public PostingsFormat getPostingsFormat(String field) { - return completionFields.contains(field) ? CompletionFieldMapper.postingsFormat() : null; - } - void checkLimits(IndexSettings settings) { checkFieldLimit(settings.getMappingTotalFieldsLimit()); checkObjectDepthLimit(settings.getMappingDepthLimit()); diff --git a/server/src/main/java/org/elasticsearch/internal/CompletionsPostingsFormatExtension.java b/server/src/main/java/org/elasticsearch/internal/CompletionsPostingsFormatExtension.java new file mode 100644 index 0000000000000..bb28d4dd6c901 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/internal/CompletionsPostingsFormatExtension.java @@ -0,0 +1,28 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.internal; + +import org.apache.lucene.search.suggest.document.CompletionPostingsFormat; + +/** + * Allows plugging-in the Completions Postings Format. + */ +public interface CompletionsPostingsFormatExtension { + + /** + * Returns the name of the {@link CompletionPostingsFormat} that Elasticsearch should use. Should return null if the extension + * is not enabled. + *

+ * Note that the name must match a codec that is available on all nodes in the cluster, otherwise IndexCorruptionExceptions will occur. + * A feature can be used to protect against this scenario, or alternatively, the codec code can be rolled out prior to its usage by this + * extension. + */ + String getFormatName(); +} From 2895f1e900b2f41704fd507845102a281cff437e Mon Sep 17 00:00:00 2001 From: Ed Savage Date: Fri, 29 Nov 2024 11:37:45 +1300 Subject: [PATCH 08/27] [ML] Remove deprecated sort from reindex operation (#117606) Sort in reindex is deprecated. This PR removes its use from within the reindexing step of dataframe analytics. Testing indicates that having the destination index sorted is a "nice to have" and not necessary for the DFA functionality to succeed. --- docs/changelog/117606.yaml | 5 +++++ .../xpack/ml/dataframe/steps/ReindexingStep.java | 3 --- 2 files changed, 5 insertions(+), 3 deletions(-) create mode 100644 docs/changelog/117606.yaml diff --git a/docs/changelog/117606.yaml b/docs/changelog/117606.yaml new file mode 100644 index 0000000000000..ea61099a1a6b4 --- /dev/null +++ b/docs/changelog/117606.yaml @@ -0,0 +1,5 @@ +pr: 117606 +summary: Remove deprecated sort from reindex operation within dataframe analytics procedure +area: Machine Learning +type: enhancement +issues: [] diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/ReindexingStep.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/ReindexingStep.java index 0ccdd1eb64601..2a6d6eb329503 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/ReindexingStep.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/ReindexingStep.java @@ -27,13 +27,11 @@ import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.reindex.BulkByScrollResponse; import org.elasticsearch.index.reindex.BulkByScrollTask; import org.elasticsearch.index.reindex.ReindexAction; import org.elasticsearch.index.reindex.ReindexRequest; import org.elasticsearch.script.Script; -import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.tasks.TaskId; @@ -147,7 +145,6 @@ protected void doExecute(ActionListener listener) { reindexRequest.setSourceQuery(config.getSource().getParsedQuery()); reindexRequest.getSearchRequest().allowPartialSearchResults(false); reindexRequest.getSearchRequest().source().fetchSource(config.getSource().getSourceFiltering()); - reindexRequest.getSearchRequest().source().sort(SeqNoFieldMapper.NAME, SortOrder.ASC); reindexRequest.setDestIndex(config.getDest().getIndex()); // We explicitly set slices to 1 as we cannot parallelize in order to have the incremental id From c35777a175f10a49ae860d28aa16b40d6f66c49a Mon Sep 17 00:00:00 2001 From: Rene Groeschke Date: Fri, 29 Nov 2024 02:26:34 +0100 Subject: [PATCH 09/27] [Build] Declare mirror for eclipse p2 repository (#117732) The spotlight plugin directly resolves dependencies from p2 which causes `java.io.IOException: Failed to load eclipse jdt formatter` issues if that repo is not accessible. This is a workaround for the eclipse p2 default repository being down resulting in all our ci jobs to fail. The artifacts in question we wanna cache live in `~/.m2/repository` --- .../conventions/precommit/FormattingPrecommitPlugin.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/build-conventions/src/main/java/org/elasticsearch/gradle/internal/conventions/precommit/FormattingPrecommitPlugin.java b/build-conventions/src/main/java/org/elasticsearch/gradle/internal/conventions/precommit/FormattingPrecommitPlugin.java index ea9009172c7e2..41c0b4d67e1df 100644 --- a/build-conventions/src/main/java/org/elasticsearch/gradle/internal/conventions/precommit/FormattingPrecommitPlugin.java +++ b/build-conventions/src/main/java/org/elasticsearch/gradle/internal/conventions/precommit/FormattingPrecommitPlugin.java @@ -17,6 +17,8 @@ import org.gradle.api.Project; import java.io.File; +import java.util.Arrays; +import java.util.Map; /** * This plugin configures formatting for Java source using Spotless @@ -64,7 +66,8 @@ public void apply(Project project) { java.importOrderFile(new File(elasticsearchWorkspace, importOrderPath)); // Most formatting is done through the Eclipse formatter - java.eclipse().configFile(new File(elasticsearchWorkspace, formatterConfigPath)); + java.eclipse().withP2Mirrors(Map.of("https://download.eclipse.org/", "https://mirror.umd.edu/eclipse/")) + .configFile(new File(elasticsearchWorkspace, formatterConfigPath)); // Ensure blank lines are actually empty. Since formatters are applied in // order, apply this one last, otherwise non-empty blank lines can creep From e54c7cf5edd4ffd24725412015b5d3db1e7ce5a4 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Fri, 29 Nov 2024 02:19:48 +0000 Subject: [PATCH 10/27] [ML] Disable machine learning on macOS x86_64 (#104125) As previously advised in #104087, machine learning functionality will no longer be available on macOS x86_64. Machine learning functionality is still available on macOS by using an arm64 machine (Apple silicon). It is also possible to run Elasticsearch with machine learning functionality within a Docker container on macOS x86_64. This PR should be merged to main after the branch is split for the last minor release scheduled for before December 2024. For example, suppose 8.17.0 is scheduled for release in November 2024 and 8.18.0 is scheduled for release in January 2025. Then this PR should be merged to main after the 8.17 branch is split. One this PR is merged a followup PR should be opened against the ml-cpp repo to remove the build system for darwin-x86_64. It has been confirmed that with this change in place the Elasticsearch build system works with an ml-cpp bundle that does not contain a platform/darwin-x86_64 directory. It still produces an Elasticsearch build that will run providing xpack.ml.enabled is not explicitly set to true. After the build system for darwin-x86_64 has been removed from the ml-cpp repo, we will be able to do another PyTorch upgrade without having to worry about tweaking the build system to work on Intel macOS. --------- Co-authored-by: Ed Savage Co-authored-by: Valeriy Khakhutskyy <1292899+valeriy42@users.noreply.github.com> --- docs/changelog/104125.yaml | 18 +++++++++++++++ .../xpack/core/XPackSettings.java | 22 +++++++++++++++++-- .../xpack/ml/MachineLearning.java | 11 ---------- 3 files changed, 38 insertions(+), 13 deletions(-) create mode 100644 docs/changelog/104125.yaml diff --git a/docs/changelog/104125.yaml b/docs/changelog/104125.yaml new file mode 100644 index 0000000000000..e5c5ea6a3f1cd --- /dev/null +++ b/docs/changelog/104125.yaml @@ -0,0 +1,18 @@ +pr: 104125 +summary: Disable machine learning on macOS x86_64 +area: Machine Learning +type: breaking +issues: [] +breaking: + title: Disable machine learning on macOS x86_64 + area: Packaging + details: The machine learning plugin is permanently disabled on macOS x86_64. + For the last three years Apple has been selling hardware based on the arm64 + architecture, and support will increasingly focus on this architecture in + the future. Changes to upstream dependencies of Elastic's machine learning + functionality have made it unviable for Elastic to continue to build machine + learning on macOS x86_64. + impact: To continue to use machine learning functionality on macOS please switch to + an arm64 machine (Apple silicon). Alternatively, it will still be possible to run + Elasticsearch with machine learning enabled in a Docker container on macOS x86_64. + notable: false diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java index 72e8805e96fc4..6aef618288fd2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java @@ -7,12 +7,16 @@ package org.elasticsearch.xpack.core; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.ssl.SslClientAuthenticationMode; import org.elasticsearch.common.ssl.SslVerificationMode; import org.elasticsearch.core.Strings; +import org.elasticsearch.plugins.Platforms; import org.elasticsearch.transport.RemoteClusterPortSettings; import org.elasticsearch.xpack.core.security.SecurityField; import org.elasticsearch.xpack.core.security.authc.support.Hasher; @@ -26,6 +30,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.function.Function; import javax.crypto.SecretKeyFactory; @@ -40,6 +45,8 @@ */ public class XPackSettings { + private static final Logger logger = LogManager.getLogger(XPackSettings.class); + private XPackSettings() { throw new IllegalStateException("Utility class should not be instantiated"); } @@ -76,10 +83,21 @@ public Iterator> settings() { /** Setting for enabling or disabling graph. Defaults to true. */ public static final Setting GRAPH_ENABLED = Setting.boolSetting("xpack.graph.enabled", true, Setting.Property.NodeScope); - /** Setting for enabling or disabling machine learning. Defaults to true. */ + public static final Set ML_NATIVE_CODE_PLATFORMS = Set.of("darwin-aarch64", "linux-aarch64", "linux-x86_64", "windows-x86_64"); + + /** Setting for enabling or disabling machine learning. Defaults to true on platforms that have the ML native code available. */ public static final Setting MACHINE_LEARNING_ENABLED = Setting.boolSetting( "xpack.ml.enabled", - true, + ML_NATIVE_CODE_PLATFORMS.contains(Platforms.PLATFORM_NAME), + enabled -> { + if (enabled && ML_NATIVE_CODE_PLATFORMS.contains(Platforms.PLATFORM_NAME) == false) { + SettingsException e = new SettingsException("xpack.ml.enabled cannot be set to [true] on [{}]", Platforms.PLATFORM_NAME); + // The exception doesn't get logged nicely on the console because it's thrown during initial plugin loading, + // so log separately here to make absolutely clear what happened + logger.fatal(e.getMessage()); + throw e; + } + }, Setting.Property.NodeScope ); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index 1feb95661f33a..8363e0f5c19a1 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -32,7 +32,6 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.logging.DeprecationCategory; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; @@ -69,7 +68,6 @@ import org.elasticsearch.plugins.ExtensiblePlugin; import org.elasticsearch.plugins.IngestPlugin; import org.elasticsearch.plugins.PersistentTaskPlugin; -import org.elasticsearch.plugins.Platforms; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.plugins.ShutdownAwarePlugin; @@ -931,15 +929,6 @@ public Collection createComponents(PluginServices services) { return List.of(new JobManagerHolder(), new MachineLearningExtensionHolder()); } - if ("darwin-x86_64".equals(Platforms.PLATFORM_NAME)) { - String msg = "The machine learning plugin will be permanently disabled on macOS x86_64 in new minor versions released " - + "from December 2024 onwards. To continue to use machine learning functionality on macOS please switch to an arm64 " - + "machine (Apple silicon). Alternatively, it will still be possible to run Elasticsearch with machine learning " - + "enabled in a Docker container on macOS x86_64."; - logger.warn(msg); - deprecationLogger.warn(DeprecationCategory.PLUGINS, "ml-darwin-x86_64", msg); - } - machineLearningExtension.get().configure(environment.settings()); this.mlUpgradeModeActionFilter.set(new MlUpgradeModeActionFilter(clusterService)); From 56637285a8f2bacc88a12c7824b8b88d06752b07 Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Fri, 29 Nov 2024 13:47:40 +1100 Subject: [PATCH 11/27] Implement CAS support in Azure test fixture (#117104) Closes ES-5680 --- .../azure/AzureBlobStoreRepositoryTests.java | 8 +- .../AzureStorageCleanupThirdPartyTests.java | 4 +- .../azure/AzureBlobContainer.java | 2 +- .../repositories/azure/AzureBlobStore.java | 26 +- .../azure/AzureBlobContainerStatsTests.java | 3 +- .../RepositoryAzureClientYamlTestSuiteIT.java | 4 +- .../test/repository_azure/20_repository.yml | 14 + .../java/fixture/azure/AzureHttpFixture.java | 15 +- .../java/fixture/azure/AzureHttpHandler.java | 333 ++++++++---- .../fixture/azure/MockAzureBlobStore.java | 484 ++++++++++++++++++ .../azure/AzureRepositoriesMeteringIT.java | 4 +- .../AzureSearchableSnapshotsIT.java | 4 +- .../AzureSnapshotBasedRecoveryIT.java | 4 +- .../AzureRepositoryAnalysisRestIT.java | 12 +- 14 files changed, 800 insertions(+), 117 deletions(-) create mode 100644 test/fixtures/azure-fixture/src/main/java/fixture/azure/MockAzureBlobStore.java diff --git a/modules/repository-azure/src/internalClusterTest/java/org/elasticsearch/repositories/azure/AzureBlobStoreRepositoryTests.java b/modules/repository-azure/src/internalClusterTest/java/org/elasticsearch/repositories/azure/AzureBlobStoreRepositoryTests.java index bd21f208faac4..3fa4f7de7e717 100644 --- a/modules/repository-azure/src/internalClusterTest/java/org/elasticsearch/repositories/azure/AzureBlobStoreRepositoryTests.java +++ b/modules/repository-azure/src/internalClusterTest/java/org/elasticsearch/repositories/azure/AzureBlobStoreRepositoryTests.java @@ -9,6 +9,7 @@ package org.elasticsearch.repositories.azure; import fixture.azure.AzureHttpHandler; +import fixture.azure.MockAzureBlobStore; import com.azure.storage.common.policy.RequestRetryOptions; import com.azure.storage.common.policy.RetryPolicyType; @@ -184,7 +185,12 @@ long getUploadBlockSize() { @SuppressForbidden(reason = "this test uses a HttpHandler to emulate an Azure endpoint") private static class AzureBlobStoreHttpHandler extends AzureHttpHandler implements BlobStoreHttpHandler { AzureBlobStoreHttpHandler(final String account, final String container) { - super(account, container, null /* no auth header validation - sometimes it's omitted in these tests (TODO why?) */); + super( + account, + container, + null /* no auth header validation - sometimes it's omitted in these tests (TODO why?) */, + MockAzureBlobStore.LeaseExpiryPredicate.NEVER_EXPIRE + ); } } diff --git a/modules/repository-azure/src/internalClusterTest/java/org/elasticsearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java b/modules/repository-azure/src/internalClusterTest/java/org/elasticsearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java index 6d5c17c392141..40be0f8ca78c4 100644 --- a/modules/repository-azure/src/internalClusterTest/java/org/elasticsearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java +++ b/modules/repository-azure/src/internalClusterTest/java/org/elasticsearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java @@ -10,6 +10,7 @@ package org.elasticsearch.repositories.azure; import fixture.azure.AzureHttpFixture; +import fixture.azure.MockAzureBlobStore; import com.azure.core.exception.HttpResponseException; import com.azure.storage.blob.BlobContainerClient; @@ -60,7 +61,8 @@ public class AzureStorageCleanupThirdPartyTests extends AbstractThirdPartyReposi System.getProperty("test.azure.container"), System.getProperty("test.azure.tenant_id"), System.getProperty("test.azure.client_id"), - AzureHttpFixture.sharedKeyForAccountPredicate(AZURE_ACCOUNT) + AzureHttpFixture.sharedKeyForAccountPredicate(AZURE_ACCOUNT), + MockAzureBlobStore.LeaseExpiryPredicate.NEVER_EXPIRE ); @Override diff --git a/modules/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java b/modules/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java index 52bc1ee1399d4..73936d82fc204 100644 --- a/modules/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java +++ b/modules/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobContainer.java @@ -180,7 +180,7 @@ protected String buildKey(String blobName) { } private boolean skipRegisterOperation(ActionListener listener) { - return skipCas(listener) || skipIfNotPrimaryOnlyLocationMode(listener); + return skipIfNotPrimaryOnlyLocationMode(listener); } private boolean skipIfNotPrimaryOnlyLocationMode(ActionListener listener) { diff --git a/modules/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java b/modules/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java index 3c64bb9f3b830..b4567a92184fc 100644 --- a/modules/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java +++ b/modules/repository-azure/src/main/java/org/elasticsearch/repositories/azure/AzureBlobStore.java @@ -40,6 +40,7 @@ import com.azure.storage.blob.models.ListBlobsOptions; import com.azure.storage.blob.options.BlobParallelUploadOptions; import com.azure.storage.blob.options.BlockBlobSimpleUploadOptions; +import com.azure.storage.blob.specialized.BlobLeaseClient; import com.azure.storage.blob.specialized.BlobLeaseClientBuilder; import com.azure.storage.blob.specialized.BlockBlobAsyncClient; @@ -1010,7 +1011,7 @@ private static BytesReference innerCompareAndExchangeRegister( } return currentValue; } finally { - leaseClient.releaseLease(); + bestEffortRelease(leaseClient); } } else { if (expected.length() == 0) { @@ -1020,6 +1021,29 @@ private static BytesReference innerCompareAndExchangeRegister( } } + /** + * Release the lease, ignoring conflicts due to expiry + * + * @see Outcomes of lease operations by lease state + * @param leaseClient The client for the lease + */ + private static void bestEffortRelease(BlobLeaseClient leaseClient) { + try { + leaseClient.releaseLease(); + } catch (BlobStorageException blobStorageException) { + if (blobStorageException.getStatusCode() == RestStatus.CONFLICT.getStatus()) { + // This is OK, we tried to release a lease that was expired/re-acquired + logger.debug( + "Ignored conflict on release: errorCode={}, message={}", + blobStorageException.getErrorCode(), + blobStorageException.getMessage() + ); + } else { + throw blobStorageException; + } + } + } + private static BytesReference downloadRegisterBlob( String containerPath, String blobKey, diff --git a/modules/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerStatsTests.java b/modules/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerStatsTests.java index 6730e5c3c81bd..812d519e60260 100644 --- a/modules/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerStatsTests.java +++ b/modules/repository-azure/src/test/java/org/elasticsearch/repositories/azure/AzureBlobContainerStatsTests.java @@ -10,6 +10,7 @@ package org.elasticsearch.repositories.azure; import fixture.azure.AzureHttpHandler; +import fixture.azure.MockAzureBlobStore; import org.elasticsearch.common.blobstore.OperationPurpose; import org.elasticsearch.common.bytes.BytesReference; @@ -26,7 +27,7 @@ public class AzureBlobContainerStatsTests extends AbstractAzureServerTestCase { @SuppressForbidden(reason = "use a http server") @Before public void configureAzureHandler() { - httpServer.createContext("/", new AzureHttpHandler(ACCOUNT, CONTAINER, null)); + httpServer.createContext("/", new AzureHttpHandler(ACCOUNT, CONTAINER, null, MockAzureBlobStore.LeaseExpiryPredicate.NEVER_EXPIRE)); } public void testOperationPurposeIsReflectedInBlobStoreStats() throws IOException { diff --git a/modules/repository-azure/src/yamlRestTest/java/org/elasticsearch/repositories/azure/RepositoryAzureClientYamlTestSuiteIT.java b/modules/repository-azure/src/yamlRestTest/java/org/elasticsearch/repositories/azure/RepositoryAzureClientYamlTestSuiteIT.java index 64dde0248ad2c..b24574da36825 100644 --- a/modules/repository-azure/src/yamlRestTest/java/org/elasticsearch/repositories/azure/RepositoryAzureClientYamlTestSuiteIT.java +++ b/modules/repository-azure/src/yamlRestTest/java/org/elasticsearch/repositories/azure/RepositoryAzureClientYamlTestSuiteIT.java @@ -10,6 +10,7 @@ package org.elasticsearch.repositories.azure; import fixture.azure.AzureHttpFixture; +import fixture.azure.MockAzureBlobStore; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; @@ -47,7 +48,8 @@ public class RepositoryAzureClientYamlTestSuiteIT extends ESClientYamlSuiteTestC AZURE_TEST_CONTAINER, AZURE_TEST_TENANT_ID, AZURE_TEST_CLIENT_ID, - decideAuthHeaderPredicate() + decideAuthHeaderPredicate(), + MockAzureBlobStore.LeaseExpiryPredicate.NEVER_EXPIRE ); private static Predicate decideAuthHeaderPredicate() { diff --git a/modules/repository-azure/src/yamlRestTest/resources/rest-api-spec/test/repository_azure/20_repository.yml b/modules/repository-azure/src/yamlRestTest/resources/rest-api-spec/test/repository_azure/20_repository.yml index a4a7d0b22a0ed..968e93cf9fc55 100644 --- a/modules/repository-azure/src/yamlRestTest/resources/rest-api-spec/test/repository_azure/20_repository.yml +++ b/modules/repository-azure/src/yamlRestTest/resources/rest-api-spec/test/repository_azure/20_repository.yml @@ -193,6 +193,20 @@ setup: container: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE client: integration_test +--- +"Register a read-only repository with a non existing container": + + - do: + catch: /repository_verification_exception/ + snapshot.create_repository: + repository: repository + body: + type: azure + settings: + container: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE + client: integration_test + readonly: true + --- "Register a repository with a non existing client": diff --git a/test/fixtures/azure-fixture/src/main/java/fixture/azure/AzureHttpFixture.java b/test/fixtures/azure-fixture/src/main/java/fixture/azure/AzureHttpFixture.java index 39105e0a27dc9..ab4d54f4fc451 100644 --- a/test/fixtures/azure-fixture/src/main/java/fixture/azure/AzureHttpFixture.java +++ b/test/fixtures/azure-fixture/src/main/java/fixture/azure/AzureHttpFixture.java @@ -45,6 +45,7 @@ public class AzureHttpFixture extends ExternalResource { private final String clientId; private final String tenantId; private final Predicate authHeaderPredicate; + private final MockAzureBlobStore.LeaseExpiryPredicate leaseExpiryPredicate; private HttpServer server; private HttpServer metadataServer; @@ -116,7 +117,8 @@ public AzureHttpFixture( String container, @Nullable String rawTenantId, @Nullable String rawClientId, - Predicate authHeaderPredicate + Predicate authHeaderPredicate, + MockAzureBlobStore.LeaseExpiryPredicate leaseExpiryPredicate ) { final var tenantId = Strings.hasText(rawTenantId) ? rawTenantId : null; final var clientId = Strings.hasText(rawClientId) ? rawClientId : null; @@ -135,6 +137,7 @@ public AzureHttpFixture( this.tenantId = tenantId; this.clientId = clientId; this.authHeaderPredicate = authHeaderPredicate; + this.leaseExpiryPredicate = leaseExpiryPredicate; } private String scheme() { @@ -193,7 +196,10 @@ protected void before() { } case HTTP -> { server = HttpServer.create(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0), 0); - server.createContext("/" + account, new AzureHttpHandler(account, container, actualAuthHeaderPredicate)); + server.createContext( + "/" + account, + new AzureHttpHandler(account, container, actualAuthHeaderPredicate, leaseExpiryPredicate) + ); server.start(); oauthTokenServiceServer = HttpServer.create(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0), 0); @@ -222,7 +228,10 @@ protected void before() { final var httpsServer = HttpsServer.create(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0), 0); this.server = httpsServer; httpsServer.setHttpsConfigurator(new HttpsConfigurator(sslContext)); - httpsServer.createContext("/" + account, new AzureHttpHandler(account, container, actualAuthHeaderPredicate)); + httpsServer.createContext( + "/" + account, + new AzureHttpHandler(account, container, actualAuthHeaderPredicate, leaseExpiryPredicate) + ); httpsServer.start(); } { diff --git a/test/fixtures/azure-fixture/src/main/java/fixture/azure/AzureHttpHandler.java b/test/fixtures/azure-fixture/src/main/java/fixture/azure/AzureHttpHandler.java index bbcfe1f75dc06..904f4581ad2c9 100644 --- a/test/fixtures/azure-fixture/src/main/java/fixture/azure/AzureHttpHandler.java +++ b/test/fixtures/azure-fixture/src/main/java/fixture/azure/AzureHttpHandler.java @@ -15,7 +15,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.regex.Regex; @@ -27,7 +26,6 @@ import org.elasticsearch.xcontent.XContentType; import java.io.BufferedReader; -import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStreamReader; import java.nio.charset.StandardCharsets; @@ -43,11 +41,11 @@ import java.util.Objects; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; import java.util.function.Predicate; import java.util.regex.Matcher; import java.util.regex.Pattern; +import static fixture.azure.MockAzureBlobStore.failTestWithAssertionError; import static org.elasticsearch.repositories.azure.AzureFixtureHelper.assertValidBlockId; /** @@ -56,17 +54,29 @@ @SuppressForbidden(reason = "Uses a HttpServer to emulate an Azure endpoint") public class AzureHttpHandler implements HttpHandler { private static final Logger logger = LogManager.getLogger(AzureHttpHandler.class); + private static final Pattern RANGE_HEADER_PATTERN = Pattern.compile("^bytes=([0-9]+)-([0-9]+)$"); + static final String X_MS_LEASE_ID = "x-ms-lease-id"; + static final String X_MS_PROPOSED_LEASE_ID = "x-ms-proposed-lease-id"; + static final String X_MS_LEASE_DURATION = "x-ms-lease-duration"; + static final String X_MS_LEASE_BREAK_PERIOD = "x-ms-lease-break-period"; + static final String X_MS_BLOB_TYPE = "x-ms-blob-type"; + static final String X_MS_BLOB_CONTENT_LENGTH = "x-ms-blob-content-length"; - private final Map blobs; private final String account; private final String container; private final Predicate authHeaderPredicate; - - public AzureHttpHandler(final String account, final String container, @Nullable Predicate authHeaderPredicate) { + private final MockAzureBlobStore mockAzureBlobStore; + + public AzureHttpHandler( + final String account, + final String container, + @Nullable Predicate authHeaderPredicate, + MockAzureBlobStore.LeaseExpiryPredicate leaseExpiryPredicate + ) { this.account = Objects.requireNonNull(account); this.container = Objects.requireNonNull(container); this.authHeaderPredicate = authHeaderPredicate; - this.blobs = new ConcurrentHashMap<>(); + this.mockAzureBlobStore = new MockAzureBlobStore(leaseExpiryPredicate); } private static List getAuthHeader(HttpExchange exchange) { @@ -134,7 +144,7 @@ public void handle(final HttpExchange exchange) throws IOException { final String blockId = params.get("blockid"); assert assertValidBlockId(blockId); - blobs.put(blockId, Streams.readFully(exchange.getRequestBody())); + mockAzureBlobStore.putBlock(blobPath(exchange), blockId, Streams.readFully(exchange.getRequestBody()), leaseId(exchange)); exchange.sendResponseHeaders(RestStatus.CREATED.getStatus(), -1); } else if (Regex.simpleMatch("PUT /" + account + "/" + container + "/*comp=blocklist*", request)) { @@ -145,83 +155,124 @@ public void handle(final HttpExchange exchange) throws IOException { .map(line -> line.substring(0, line.indexOf(""))) .toList(); - final ByteArrayOutputStream blob = new ByteArrayOutputStream(); - for (String blockId : blockIds) { - BytesReference block = blobs.remove(blockId); - assert block != null; - block.writeTo(blob); - } - blobs.put(exchange.getRequestURI().getPath(), new BytesArray(blob.toByteArray())); + mockAzureBlobStore.putBlockList(blobPath(exchange), blockIds, leaseId(exchange)); exchange.getResponseHeaders().add("x-ms-request-server-encrypted", "false"); exchange.sendResponseHeaders(RestStatus.CREATED.getStatus(), -1); + } else if (Regex.simpleMatch("PUT /" + account + "/" + container + "*comp=lease*", request)) { + // Lease Blob (https://learn.microsoft.com/en-us/rest/api/storageservices/lease-blob) + final String leaseAction = requireHeader(exchange, "x-ms-lease-action"); + + switch (leaseAction) { + case "acquire" -> { + final int leaseDurationSeconds = requireIntegerHeader(exchange, X_MS_LEASE_DURATION); + final String proposedLeaseId = exchange.getRequestHeaders().getFirst(X_MS_PROPOSED_LEASE_ID); + final String newLeaseId = mockAzureBlobStore.acquireLease( + blobPath(exchange), + leaseDurationSeconds, + proposedLeaseId + ); + exchange.getResponseHeaders().set(X_MS_LEASE_ID, newLeaseId); + exchange.sendResponseHeaders(RestStatus.CREATED.getStatus(), -1); + } + case "release" -> { + final String leaseId = requireHeader(exchange, X_MS_LEASE_ID); + mockAzureBlobStore.releaseLease(blobPath(exchange), leaseId); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), -1); + } + case "break" -> { + mockAzureBlobStore.breakLease(blobPath(exchange), getOptionalIntegerHeader(exchange, X_MS_LEASE_BREAK_PERIOD)); + exchange.sendResponseHeaders(RestStatus.ACCEPTED.getStatus(), -1); + } + case "renew", "change" -> { + failTestWithAssertionError("Attempt was made to use not-implemented lease action: " + leaseAction); + throw new MockAzureBlobStore.AzureBlobStoreError( + RestStatus.NOT_IMPLEMENTED, + "NotImplemented", + "Attempted to use unsupported lease API: " + leaseAction + ); + } + default -> { + failTestWithAssertionError("Unrecognized lease action: " + leaseAction); + throw new MockAzureBlobStore.BadRequestException( + "InvalidHeaderValue", + "Invalid x-ms-lease-action header: " + leaseAction + ); + } + } } else if (Regex.simpleMatch("PUT /" + account + "/" + container + "/*", request)) { // PUT Blob (see https://docs.microsoft.com/en-us/rest/api/storageservices/put-blob) + final String blobType = requireHeader(exchange, X_MS_BLOB_TYPE); final String ifNoneMatch = exchange.getRequestHeaders().getFirst("If-None-Match"); - if ("*".equals(ifNoneMatch)) { - if (blobs.putIfAbsent(exchange.getRequestURI().getPath(), Streams.readFully(exchange.getRequestBody())) != null) { - sendError(exchange, RestStatus.CONFLICT); - return; - } - } else { - blobs.put(exchange.getRequestURI().getPath(), Streams.readFully(exchange.getRequestBody())); - } + mockAzureBlobStore.putBlob( + blobPath(exchange), + Streams.readFully(exchange.getRequestBody()), + blobType, + ifNoneMatch, + leaseId(exchange) + ); exchange.getResponseHeaders().add("x-ms-request-server-encrypted", "false"); exchange.sendResponseHeaders(RestStatus.CREATED.getStatus(), -1); } else if (Regex.simpleMatch("HEAD /" + account + "/" + container + "/*", request)) { // Get Blob Properties (see https://docs.microsoft.com/en-us/rest/api/storageservices/get-blob-properties) - final BytesReference blob = blobs.get(exchange.getRequestURI().getPath()); - if (blob == null) { - sendError(exchange, RestStatus.NOT_FOUND); - return; - } - exchange.getResponseHeaders().add("x-ms-blob-content-length", String.valueOf(blob.length())); - exchange.getResponseHeaders().add("Content-Length", String.valueOf(blob.length())); - exchange.getResponseHeaders().add("x-ms-blob-type", "BlockBlob"); + final MockAzureBlobStore.AzureBlockBlob blob = mockAzureBlobStore.getBlob(blobPath(exchange), leaseId(exchange)); + + final Headers responseHeaders = exchange.getResponseHeaders(); + final BytesReference blobContents = blob.getContents(); + responseHeaders.add(X_MS_BLOB_CONTENT_LENGTH, String.valueOf(blobContents.length())); + responseHeaders.add("Content-Length", String.valueOf(blobContents.length())); + responseHeaders.add(X_MS_BLOB_TYPE, blob.type()); exchange.sendResponseHeaders(RestStatus.OK.getStatus(), -1); } else if (Regex.simpleMatch("GET /" + account + "/" + container + "/*", request)) { - // GET Object (https://docs.aws.amazon.com/AmazonS3/latest/API/RESTObjectGET.html) - final BytesReference blob = blobs.get(exchange.getRequestURI().getPath()); - if (blob == null) { - sendError(exchange, RestStatus.NOT_FOUND); - return; - } + // Get Blob (https://learn.microsoft.com/en-us/rest/api/storageservices/get-blob) + final MockAzureBlobStore.AzureBlockBlob blob = mockAzureBlobStore.getBlob(blobPath(exchange), leaseId(exchange)); + final BytesReference responseContent; + final RestStatus successStatus; // see Constants.HeaderConstants.STORAGE_RANGE_HEADER final String range = exchange.getRequestHeaders().getFirst("x-ms-range"); - final Matcher matcher = Pattern.compile("^bytes=([0-9]+)-([0-9]+)$").matcher(range); - if (matcher.matches() == false) { - throw new AssertionError("Range header does not match expected format: " + range); - } + if (range != null) { + final Matcher matcher = RANGE_HEADER_PATTERN.matcher(range); + if (matcher.matches() == false) { + throw new MockAzureBlobStore.BadRequestException( + "InvalidHeaderValue", + "Range header does not match expected format: " + range + ); + } - final long start = Long.parseLong(matcher.group(1)); - final long end = Long.parseLong(matcher.group(2)); + final long start = Long.parseLong(matcher.group(1)); + final long end = Long.parseLong(matcher.group(2)); - if (blob.length() <= start) { - exchange.getResponseHeaders().add("Content-Type", "application/octet-stream"); - exchange.sendResponseHeaders(RestStatus.REQUESTED_RANGE_NOT_SATISFIED.getStatus(), -1); - return; - } + final BytesReference blobContents = blob.getContents(); + if (blobContents.length() <= start) { + exchange.getResponseHeaders().add("Content-Type", "application/octet-stream"); + exchange.sendResponseHeaders(RestStatus.REQUESTED_RANGE_NOT_SATISFIED.getStatus(), -1); + return; + } - var responseBlob = blob.slice(Math.toIntExact(start), Math.toIntExact(Math.min(end - start + 1, blob.length() - start))); + responseContent = blobContents.slice( + Math.toIntExact(start), + Math.toIntExact(Math.min(end - start + 1, blobContents.length() - start)) + ); + successStatus = RestStatus.PARTIAL_CONTENT; + } else { + responseContent = blob.getContents(); + successStatus = RestStatus.OK; + } exchange.getResponseHeaders().add("Content-Type", "application/octet-stream"); - exchange.getResponseHeaders().add("x-ms-blob-content-length", String.valueOf(responseBlob.length())); - exchange.getResponseHeaders().add("x-ms-blob-type", "blockblob"); + exchange.getResponseHeaders().add(X_MS_BLOB_CONTENT_LENGTH, String.valueOf(responseContent.length())); + exchange.getResponseHeaders().add(X_MS_BLOB_TYPE, blob.type()); exchange.getResponseHeaders().add("ETag", "\"blockblob\""); - exchange.sendResponseHeaders(RestStatus.OK.getStatus(), responseBlob.length()); - responseBlob.writeTo(exchange.getResponseBody()); + exchange.sendResponseHeaders(successStatus.getStatus(), responseContent.length() == 0 ? -1 : responseContent.length()); + responseContent.writeTo(exchange.getResponseBody()); } else if (Regex.simpleMatch("DELETE /" + account + "/" + container + "/*", request)) { // Delete Blob (https://docs.microsoft.com/en-us/rest/api/storageservices/delete-blob) - final boolean deleted = blobs.entrySet().removeIf(blob -> blob.getKey().startsWith(exchange.getRequestURI().getPath())); - if (deleted) { - exchange.sendResponseHeaders(RestStatus.ACCEPTED.getStatus(), -1); - } else { - exchange.sendResponseHeaders(RestStatus.NOT_FOUND.getStatus(), -1); - } + mockAzureBlobStore.deleteBlob(blobPath(exchange), leaseId(exchange)); + exchange.sendResponseHeaders(RestStatus.ACCEPTED.getStatus(), -1); } else if (Regex.simpleMatch("GET /" + account + "/" + container + "?*restype=container*comp=list*", request)) { // List Blobs (https://docs.microsoft.com/en-us/rest/api/storageservices/list-blobs) @@ -239,11 +290,12 @@ public void handle(final HttpExchange exchange) throws IOException { list.append("").append(delimiter).append(""); } list.append(""); - for (Map.Entry blob : blobs.entrySet()) { - if (prefix != null && blob.getKey().startsWith("/" + account + "/" + container + "/" + prefix) == false) { - continue; - } - String blobPath = blob.getKey().replace("/" + account + "/" + container + "/", ""); + final Map matchingBlobs = mockAzureBlobStore.listBlobs( + prefix, + leaseId(exchange) + ); + for (Map.Entry blob : matchingBlobs.entrySet()) { + final String blobPath = blob.getKey(); if (delimiter != null) { int fromIndex = (prefix != null ? prefix.length() : 0); int delimiterPosition = blobPath.indexOf(delimiter, fromIndex); @@ -259,7 +311,7 @@ public void handle(final HttpExchange exchange) throws IOException { %s BlockBlob - """, blobPath, blob.getValue().length())); + """, blobPath, blob.getValue().getContents().length())); } if (blobPrefixes.isEmpty() == false) { blobPrefixes.forEach(p -> list.append("").append(p).append("")); @@ -294,7 +346,8 @@ public void handle(final HttpExchange exchange) throws IOException { } // Process the deletion - if (blobs.remove("/" + account + toDelete) != null) { + try { + mockAzureBlobStore.deleteBlob(toDelete, leaseId(exchange)); final String acceptedPart = Strings.format(""" --%s Content-Type: application/http @@ -307,32 +360,43 @@ public void handle(final HttpExchange exchange) throws IOException { """, responseBoundary, contentId, requestId).replaceAll("\n", "\r\n"); response.append(acceptedPart); - } else { - final String notFoundBody = Strings.format( + } catch (MockAzureBlobStore.AzureBlobStoreError e) { + final String errorResponseBody = Strings.format( """ - BlobNotFoundThe specified blob does not exist. + %s%s RequestId:%s Time:%s""", + e.getErrorCode(), + e.getMessage(), requestId, DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(ZonedDateTime.now(ZoneId.of("UTC"))) ); - final String notFoundPart = Strings.format(""" - --%s - Content-Type: application/http - Content-ID: %s - - HTTP/1.1 404 The specified blob does not exist. - x-ms-error-code: BlobNotFound - x-ms-request-id: %s - x-ms-version: 2018-11-09 - Content-Length: %d - Content-Type: application/xml - - %s - """, responseBoundary, contentId, requestId, notFoundBody.length(), notFoundBody) - .replaceAll("\n", "\r\n"); - response.append(notFoundPart); + final String errorResponsePart = Strings.format( + """ + --%s + Content-Type: application/http + Content-ID: %s + + HTTP/1.1 %s %s + x-ms-error-code: %s + x-ms-request-id: %s + x-ms-version: 2018-11-09 + Content-Length: %d + Content-Type: application/xml + + %s + """, + responseBoundary, + contentId, + e.getRestStatus().getStatus(), + e.getMessage(), + e.getErrorCode(), + requestId, + errorResponseBody.length(), + errorResponseBody + ).replaceAll("\n", "\r\n"); + response.append(errorResponsePart); } // Clear the state @@ -350,19 +414,18 @@ public void handle(final HttpExchange exchange) throws IOException { } contentId = line.split("\\s")[1]; } else if (Regex.simpleMatch("DELETE /" + container + "/*", line)) { - String blobName = RestUtils.decodeComponent(line.split("(\\s|\\?)")[1]); + final String path = RestUtils.decodeComponent(line.split("(\\s|\\?)")[1]); if (toDelete != null) { throw new IllegalStateException("Got multiple deletes in a single request?"); } - toDelete = blobName; + toDelete = stripPrefix("/" + container + "/", path); } else if (Regex.simpleMatch("DELETE /" + account + "/" + container + "/*", line)) { // possible alternative DELETE url, depending on which method is used in the batch client String path = RestUtils.decodeComponent(line.split("(\\s|\\?)")[1]); - String blobName = path.split(account)[1]; if (toDelete != null) { throw new IllegalStateException("Got multiple deletes in a single request?"); } - toDelete = blobName; + toDelete = stripPrefix("/" + account + "/" + container + "/", path); } } response.append("--").append(responseBoundary).append("--\r\n0\r\n"); @@ -372,20 +435,90 @@ public void handle(final HttpExchange exchange) throws IOException { logger.debug("--> Sending response:\n{}", response); exchange.getResponseBody().write(response.toString().getBytes(StandardCharsets.UTF_8)); } - } else { - logger.warn("--> Unrecognised request received: {}", request); - sendError(exchange, RestStatus.BAD_REQUEST); - } + } else if (Regex.simpleMatch("PUT /*/*/*master.dat", request) + && Regex.simpleMatch("PUT /" + account + "/" + container + "*", request) == false) { + // An attempt to put master.dat to a different container. This is probably + // org.elasticsearch.repositories.blobstore.BlobStoreRepository#startVerification + throw new MockAzureBlobStore.AzureBlobStoreError( + RestStatus.NOT_FOUND, + "ContainerNotFound", + "The specified container does not exist." + ); + } else if (Regex.simpleMatch("GET /*/*restype=container*comp=list*", request) + && Regex.simpleMatch("GET /" + account + "/" + container + "*", request) == false) { + // An attempt to list the contents of a different container. This is probably + // org.elasticsearch.repositories.blobstore.BlobStoreRepository#startVerification for a read-only + // repository + throw new MockAzureBlobStore.AzureBlobStoreError( + RestStatus.NOT_FOUND, + "ContainerNotFound", + "The specified container does not exist." + ); + } else { + final String message = "You sent a request that is not supported by AzureHttpHandler: " + request; + failTestWithAssertionError(message); + throw new MockAzureBlobStore.BadRequestException("UnrecognisedRequest", message); + } + } catch (MockAzureBlobStore.AzureBlobStoreError e) { + sendError(exchange, e); + } catch (Exception e) { + failTestWithAssertionError("Uncaught exception", e); + sendError(exchange, RestStatus.INTERNAL_SERVER_ERROR, "InternalError", e.getMessage()); } finally { exchange.close(); } } + private String requireHeader(HttpExchange exchange, String headerName) { + final String headerValue = exchange.getRequestHeaders().getFirst(headerName); + if (headerValue == null) { + throw new MockAzureBlobStore.BadRequestException("MissingRequiredHeader", "Missing " + headerName + " header"); + } + return headerValue; + } + + private int requireIntegerHeader(HttpExchange exchange, String headerName) { + final String headerValue = requireHeader(exchange, headerName); + try { + return Integer.parseInt(headerValue); + } catch (NumberFormatException e) { + throw new MockAzureBlobStore.BadRequestException("InvalidHeaderValue", "Invalid " + headerName + " header"); + } + } + + @Nullable + private Integer getOptionalIntegerHeader(HttpExchange exchange, String headerName) { + final String headerValue = exchange.getRequestHeaders().getFirst(headerName); + try { + return headerValue == null ? null : Integer.parseInt(headerValue); + } catch (NumberFormatException e) { + throw new MockAzureBlobStore.BadRequestException("InvalidHeaderValue", "Invalid " + headerName + " header"); + } + } + + @Nullable + private String leaseId(HttpExchange exchange) { + return exchange.getRequestHeaders().getFirst(X_MS_LEASE_ID); + } + + private String blobPath(HttpExchange exchange) { + return stripPrefix("/" + account + "/" + container + "/", exchange.getRequestURI().getPath()); + } + public Map blobs() { - return blobs; + return mockAzureBlobStore.blobs(); + } + + public static void sendError(HttpExchange exchange, MockAzureBlobStore.AzureBlobStoreError error) throws IOException { + sendError(exchange, error.getRestStatus(), error.getErrorCode(), error.getMessage()); } public static void sendError(final HttpExchange exchange, final RestStatus status) throws IOException { + final String errorCode = toAzureErrorCode(status); + sendError(exchange, status, errorCode, status.toString()); + } + + public static void sendError(HttpExchange exchange, RestStatus restStatus, String errorCode, String errorMessage) throws IOException { final Headers headers = exchange.getResponseHeaders(); headers.add("Content-Type", "application/xml"); @@ -396,20 +529,19 @@ public static void sendError(final HttpExchange exchange, final RestStatus statu headers.add("x-ms-request-id", requestId); } - final String errorCode = toAzureErrorCode(status); // see Constants.HeaderConstants.ERROR_CODE headers.add("x-ms-error-code", errorCode); if ("HEAD".equals(exchange.getRequestMethod())) { - exchange.sendResponseHeaders(status.getStatus(), -1L); + exchange.sendResponseHeaders(restStatus.getStatus(), -1L); } else { final byte[] response = (String.format(Locale.ROOT, """ %s %s - """, errorCode, status)).getBytes(StandardCharsets.UTF_8); - exchange.sendResponseHeaders(status.getStatus(), response.length); + """, errorCode, errorMessage)).getBytes(StandardCharsets.UTF_8); + exchange.sendResponseHeaders(restStatus.getStatus(), response.length); exchange.getResponseBody().write(response); } } @@ -428,4 +560,9 @@ private static String toAzureErrorCode(final RestStatus status) { ); }; } + + private String stripPrefix(String prefix, String toStrip) { + assert toStrip.startsWith(prefix); + return toStrip.substring(prefix.length()); + } } diff --git a/test/fixtures/azure-fixture/src/main/java/fixture/azure/MockAzureBlobStore.java b/test/fixtures/azure-fixture/src/main/java/fixture/azure/MockAzureBlobStore.java new file mode 100644 index 0000000000000..c694c27c1293b --- /dev/null +++ b/test/fixtures/azure-fixture/src/main/java/fixture/azure/MockAzureBlobStore.java @@ -0,0 +1,484 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package fixture.azure; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.CompositeBytesReference; +import org.elasticsearch.common.util.Maps; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.rest.RestStatus; + +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +public class MockAzureBlobStore { + + private static final Logger logger = LogManager.getLogger(MockAzureBlobStore.class); + private static final String BLOCK_BLOB_TYPE = "BlockBlob"; + private static final String PAGE_BLOB_TYPE = "PageBlob"; + private static final String APPEND_BLOB_TYPE = "AppendBlob"; + + private final LeaseExpiryPredicate leaseExpiryPredicate; + private final Map blobs; + + /** + * Provide the means of triggering lease expiration + * + * @param leaseExpiryPredicate A Predicate that takes an active lease ID and returns true when it should be expired, or null to never fail leases + */ + public MockAzureBlobStore(LeaseExpiryPredicate leaseExpiryPredicate) { + this.blobs = new ConcurrentHashMap<>(); + this.leaseExpiryPredicate = Objects.requireNonNull(leaseExpiryPredicate); + } + + public void putBlock(String path, String blockId, BytesReference content, @Nullable String leaseId) { + blobs.compute(path, (p, existing) -> { + if (existing != null) { + existing.putBlock(blockId, content, leaseId); + return existing; + } else { + final AzureBlockBlob azureBlockBlob = new AzureBlockBlob(); + azureBlockBlob.putBlock(blockId, content, leaseId); + return azureBlockBlob; + } + }); + } + + public void putBlockList(String path, List blockIds, @Nullable String leaseId) { + final AzureBlockBlob blob = getExistingBlob(path); + blob.putBlockList(blockIds, leaseId); + } + + public void putBlob(String path, BytesReference contents, String blobType, @Nullable String ifNoneMatch, @Nullable String leaseId) { + blobs.compute(path, (p, existingValue) -> { + if (existingValue != null) { + existingValue.setContents(contents, leaseId, ifNoneMatch); + return existingValue; + } else { + validateBlobType(blobType); + final AzureBlockBlob newBlob = new AzureBlockBlob(); + newBlob.setContents(contents, leaseId); + return newBlob; + } + }); + } + + private void validateBlobType(String blobType) { + if (BLOCK_BLOB_TYPE.equals(blobType)) { + return; + } + final String errorMessage; + if (PAGE_BLOB_TYPE.equals(blobType) || APPEND_BLOB_TYPE.equals(blobType)) { + errorMessage = "Only BlockBlob is supported. This is a limitation of the MockAzureBlobStore"; + } else { + errorMessage = "Invalid blobType: " + blobType; + } + // Fail the test and respond with an error + failTestWithAssertionError(errorMessage); + throw new MockAzureBlobStore.BadRequestException("InvalidHeaderValue", errorMessage); + } + + public AzureBlockBlob getBlob(String path, @Nullable String leaseId) { + final AzureBlockBlob blob = getExistingBlob(path); + // This is the public implementation of "get blob" which will 404 for uncommitted block blobs + if (blob.isCommitted() == false) { + throw new BlobNotFoundException(); + } + blob.checkLeaseForRead(leaseId); + return blob; + } + + public void deleteBlob(String path, @Nullable String leaseId) { + final AzureBlockBlob blob = getExistingBlob(path); + blob.checkLeaseForWrite(leaseId); + blobs.remove(path); + } + + public Map listBlobs(String prefix, @Nullable String leaseId) { + return blobs.entrySet().stream().filter(e -> { + if (prefix == null || e.getKey().startsWith(prefix)) { + return true; + } + return false; + }) + .filter(e -> e.getValue().isCommitted()) + .peek(e -> e.getValue().checkLeaseForRead(leaseId)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + public String acquireLease(String path, int leaseTimeSeconds, @Nullable String proposedLeaseId) { + final AzureBlockBlob blob = getExistingBlob(path); + return blob.acquireLease(proposedLeaseId, leaseTimeSeconds); + } + + public void releaseLease(String path, @Nullable String leaseId) { + final AzureBlockBlob blob = getExistingBlob(path); + blob.releaseLease(leaseId); + } + + public void breakLease(String path, @Nullable Integer leaseBreakPeriod) { + final AzureBlockBlob blob = getExistingBlob(path); + blob.breakLease(leaseBreakPeriod); + } + + public Map blobs() { + return Maps.transformValues(blobs, AzureBlockBlob::getContents); + } + + private AzureBlockBlob getExistingBlob(String path) { + final AzureBlockBlob blob = blobs.get(path); + if (blob == null) { + throw new BlobNotFoundException(); + } + return blob; + } + + static void failTestWithAssertionError(String message) { + ExceptionsHelper.maybeDieOnAnotherThread(new AssertionError(message)); + } + + static void failTestWithAssertionError(String message, Throwable throwable) { + ExceptionsHelper.maybeDieOnAnotherThread(new AssertionError(message, throwable)); + } + + public class AzureBlockBlob { + private final Object writeLock = new Object(); + private final Lease lease = new Lease(); + private final Map blocks; + private volatile BytesReference contents; + + private AzureBlockBlob() { + this.blocks = new ConcurrentHashMap<>(); + } + + public void putBlock(String blockId, BytesReference content, @Nullable String leaseId) { + synchronized (writeLock) { + lease.checkLeaseForWrite(leaseId); + this.blocks.put(blockId, content); + } + } + + public void putBlockList(List blockIds, @Nullable String leaseId) throws BadRequestException { + synchronized (writeLock) { + lease.checkLeaseForWrite(leaseId); + final List unresolvedBlocks = blockIds.stream().filter(bId -> blocks.containsKey(bId) == false).toList(); + if (unresolvedBlocks.isEmpty() == false) { + logger.warn("Block list contained non-existent block IDs: {}", unresolvedBlocks); + throw new BadRequestException("InvalidBlockList", "The specified blocklist is invalid."); + } + final BytesReference[] resolvedContents = blockIds.stream().map(blocks::get).toList().toArray(new BytesReference[0]); + contents = CompositeBytesReference.of(resolvedContents); + } + } + + private boolean matches(String ifNoneMatchHeaderValue) { + if (ifNoneMatchHeaderValue == null) { + return false; + } + // We only support * + if ("*".equals(ifNoneMatchHeaderValue)) { + return true; + } + // Fail the test, trigger an internal server error + failTestWithAssertionError("We've only implemented 'If-None-Match: *' in the MockAzureBlobStore"); + throw new AzureBlobStoreError( + RestStatus.INTERNAL_SERVER_ERROR, + "UnsupportedHeader", + "The test fixture only supports * for If-None-Match" + ); + } + + public synchronized void setContents(BytesReference contents, @Nullable String leaseId) { + synchronized (writeLock) { + lease.checkLeaseForWrite(leaseId); + this.contents = contents; + this.blocks.clear(); + } + } + + public void setContents(BytesReference contents, @Nullable String leaseId, @Nullable String ifNoneMatchHeaderValue) { + synchronized (writeLock) { + if (matches(ifNoneMatchHeaderValue)) { + throw new PreconditionFailedException( + "TargetConditionNotMet", + "The target condition specified using HTTP conditional header(s) is not met." + ); + } + setContents(contents, leaseId); + } + } + + /** + * Get the committed contents of the blob + * + * @return The last committed contents of the blob, or null if the blob is uncommitted + */ + @Nullable + public BytesReference getContents() { + return contents; + } + + public String type() { + return BLOCK_BLOB_TYPE; + } + + public boolean isCommitted() { + return contents != null; + } + + @Override + public String toString() { + return "MockAzureBlockBlob{" + "blocks=" + blocks + ", contents=" + contents + '}'; + } + + public String acquireLease(@Nullable String proposedLeaseId, int leaseTimeSeconds) { + synchronized (writeLock) { + return lease.acquire(proposedLeaseId, leaseTimeSeconds); + } + } + + public void releaseLease(String leaseId) { + synchronized (writeLock) { + lease.release(leaseId); + } + } + + public void breakLease(@Nullable Integer leaseBreakPeriod) { + synchronized (writeLock) { + lease.breakLease(leaseBreakPeriod); + } + } + + public void checkLeaseForRead(@Nullable String leaseId) { + lease.checkLeaseForRead(leaseId); + } + + public void checkLeaseForWrite(@Nullable String leaseId) { + lease.checkLeaseForWrite(leaseId); + } + } + + /** + * @see acquire/release rules + * @see read/write rules + */ + public class Lease { + + /** + * Minimal set of states, we don't support breaking/broken + */ + enum State { + Available, + Leased, + Expired, + Broken + } + + private String leaseId; + private State state = State.Available; + private int leaseDurationSeconds; + + public synchronized String acquire(@Nullable String proposedLeaseId, int leaseDurationSeconds) { + maybeExpire(proposedLeaseId); + switch (state) { + case Available, Expired, Broken -> { + final State prevState = state; + state = State.Leased; + leaseId = proposedLeaseId != null ? proposedLeaseId : UUID.randomUUID().toString(); + validateLeaseDuration(leaseDurationSeconds); + this.leaseDurationSeconds = leaseDurationSeconds; + logger.debug("Granting lease, prior state={}, leaseId={}, expires={}", prevState, leaseId); + } + case Leased -> { + if (leaseId.equals(proposedLeaseId) == false) { + logger.debug("Mismatch on acquire - proposed leaseId: {}, active leaseId: {}", proposedLeaseId, leaseId); + throw new ConflictException( + "LeaseIdMismatchWithLeaseOperation", + "The lease ID specified did not match the lease ID for the blob/container." + ); + } + validateLeaseDuration(leaseDurationSeconds); + } + } + return leaseId; + } + + public synchronized void release(String requestLeaseId) { + switch (state) { + case Available -> throw new ConflictException( + "LeaseNotPresentWithLeaseOperation", + "There is currently no lease on the blob/container." + ); + case Leased, Expired, Broken -> { + if (leaseId.equals(requestLeaseId) == false) { + logger.debug("Mismatch on release - submitted leaseId: {}, active leaseId: {}", requestLeaseId, this.leaseId); + throw new ConflictException( + "LeaseIdMismatchWithLeaseOperation", + "The lease ID specified did not match the lease ID for the blob/container." + ); + } + state = State.Available; + this.leaseId = null; + } + } + } + + public synchronized void breakLease(Integer leaseBreakPeriod) { + // We haven't implemented the "Breaking" state so we don't support 'breaks' for non-infinite leases unless break-period is 0 + if (leaseDurationSeconds != -1 && (leaseBreakPeriod == null || leaseBreakPeriod != 0)) { + failTestWithAssertionError( + "MockAzureBlobStore only supports breaking non-infinite leases with 'x-ms-lease-break-period: 0'" + ); + } + switch (state) { + case Available -> throw new ConflictException( + "LeaseNotPresentWithLeaseOperation", + "There is currently no lease on the blob/container." + ); + case Leased, Expired, Broken -> state = State.Broken; + } + } + + public synchronized void checkLeaseForWrite(@Nullable String requestLeaseId) { + maybeExpire(requestLeaseId); + switch (state) { + case Available, Expired, Broken -> { + if (requestLeaseId != null) { + throw new PreconditionFailedException( + "LeaseLost", + "A lease ID was specified, but the lease for the blob/container has expired." + ); + } + } + case Leased -> { + if (requestLeaseId == null) { + throw new PreconditionFailedException( + "LeaseIdMissing", + "There is currently a lease on the blob/container and no lease ID was specified in the request." + ); + } + if (leaseId.equals(requestLeaseId) == false) { + throw new ConflictException( + "LeaseIdMismatchWithBlobOperation", + "The lease ID specified did not match the lease ID for the blob." + ); + } + } + } + } + + public synchronized void checkLeaseForRead(@Nullable String requestLeaseId) { + maybeExpire(requestLeaseId); + switch (state) { + case Available, Expired, Broken -> { + if (requestLeaseId != null) { + throw new PreconditionFailedException( + "LeaseLost", + "A lease ID was specified, but the lease for the blob/container has expired." + ); + } + } + case Leased -> { + if (requestLeaseId != null && requestLeaseId.equals(leaseId) == false) { + throw new ConflictException( + "LeaseIdMismatchWithBlobOperation", + "The lease ID specified did not match the lease ID for the blob." + ); + } + } + } + } + + /** + * If there's an active lease, ask the predicate if we should expire the existing it + * + * @param requestLeaseId The lease of the request + */ + private void maybeExpire(String requestLeaseId) { + if (state == State.Leased && leaseExpiryPredicate.shouldExpireLease(leaseId, requestLeaseId)) { + logger.debug("Expiring lease, id={}", leaseId); + state = State.Expired; + } + } + + private void validateLeaseDuration(long leaseTimeSeconds) { + if (leaseTimeSeconds != -1 && (leaseTimeSeconds < 15 || leaseTimeSeconds > 60)) { + throw new BadRequestException( + "InvalidHeaderValue", + AzureHttpHandler.X_MS_LEASE_DURATION + " must be between 16 and 60 seconds (was " + leaseTimeSeconds + ")" + ); + } + } + } + + public static class AzureBlobStoreError extends RuntimeException { + private final RestStatus restStatus; + private final String errorCode; + + public AzureBlobStoreError(RestStatus restStatus, String errorCode, String message) { + super(message); + this.restStatus = restStatus; + this.errorCode = errorCode; + } + + public RestStatus getRestStatus() { + return restStatus; + } + + public String getErrorCode() { + return errorCode; + } + } + + public static class BlobNotFoundException extends AzureBlobStoreError { + public BlobNotFoundException() { + super(RestStatus.NOT_FOUND, "BlobNotFound", "The specified blob does not exist."); + } + } + + public static class BadRequestException extends AzureBlobStoreError { + public BadRequestException(String errorCode, String message) { + super(RestStatus.BAD_REQUEST, errorCode, message); + } + } + + public static class ConflictException extends AzureBlobStoreError { + public ConflictException(String errorCode, String message) { + super(RestStatus.CONFLICT, errorCode, message); + } + } + + public static class PreconditionFailedException extends AzureBlobStoreError { + public PreconditionFailedException(String errorCode, String message) { + super(RestStatus.PRECONDITION_FAILED, errorCode, message); + } + } + + public interface LeaseExpiryPredicate { + + LeaseExpiryPredicate NEVER_EXPIRE = (activeLeaseId, requestLeaseId) -> false; + + /** + * Should the lease be expired? + * + * @param activeLeaseId The current active lease ID + * @param requestLeaseId The request lease ID (if any) + * @return true to expire the lease, false otherwise + */ + boolean shouldExpireLease(String activeLeaseId, @Nullable String requestLeaseId); + } +} diff --git a/x-pack/plugin/repositories-metering-api/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/repositories/metering/azure/AzureRepositoriesMeteringIT.java b/x-pack/plugin/repositories-metering-api/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/repositories/metering/azure/AzureRepositoriesMeteringIT.java index 7029a38edcb5a..d21dc4b2982f1 100644 --- a/x-pack/plugin/repositories-metering-api/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/repositories/metering/azure/AzureRepositoriesMeteringIT.java +++ b/x-pack/plugin/repositories-metering-api/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/repositories/metering/azure/AzureRepositoriesMeteringIT.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.repositories.metering.azure; import fixture.azure.AzureHttpFixture; +import fixture.azure.MockAzureBlobStore; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Booleans; @@ -37,7 +38,8 @@ public class AzureRepositoriesMeteringIT extends AbstractRepositoriesMeteringAPI AZURE_TEST_CONTAINER, System.getProperty("test.azure.tenant_id"), System.getProperty("test.azure.client_id"), - AzureHttpFixture.sharedKeyForAccountPredicate(AZURE_TEST_ACCOUNT) + AzureHttpFixture.sharedKeyForAccountPredicate(AZURE_TEST_ACCOUNT), + MockAzureBlobStore.LeaseExpiryPredicate.NEVER_EXPIRE ); private static TestTrustStore trustStore = new TestTrustStore( diff --git a/x-pack/plugin/searchable-snapshots/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/searchablesnapshots/AzureSearchableSnapshotsIT.java b/x-pack/plugin/searchable-snapshots/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/searchablesnapshots/AzureSearchableSnapshotsIT.java index 610b58453716c..f65db6dab1e68 100644 --- a/x-pack/plugin/searchable-snapshots/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/searchablesnapshots/AzureSearchableSnapshotsIT.java +++ b/x-pack/plugin/searchable-snapshots/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/searchablesnapshots/AzureSearchableSnapshotsIT.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.searchablesnapshots; import fixture.azure.AzureHttpFixture; +import fixture.azure.MockAzureBlobStore; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Booleans; @@ -38,7 +39,8 @@ public class AzureSearchableSnapshotsIT extends AbstractSearchableSnapshotsRestT AZURE_TEST_CONTAINER, System.getProperty("test.azure.tenant_id"), System.getProperty("test.azure.client_id"), - AzureHttpFixture.sharedKeyForAccountPredicate(AZURE_TEST_ACCOUNT) + AzureHttpFixture.sharedKeyForAccountPredicate(AZURE_TEST_ACCOUNT), + MockAzureBlobStore.LeaseExpiryPredicate.NEVER_EXPIRE ); private static TestTrustStore trustStore = new TestTrustStore( diff --git a/x-pack/plugin/snapshot-based-recoveries/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/snapshotbasedrecoveries/recovery/AzureSnapshotBasedRecoveryIT.java b/x-pack/plugin/snapshot-based-recoveries/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/snapshotbasedrecoveries/recovery/AzureSnapshotBasedRecoveryIT.java index 591d4582d5905..8142b40166840 100644 --- a/x-pack/plugin/snapshot-based-recoveries/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/snapshotbasedrecoveries/recovery/AzureSnapshotBasedRecoveryIT.java +++ b/x-pack/plugin/snapshot-based-recoveries/qa/azure/src/javaRestTest/java/org/elasticsearch/xpack/snapshotbasedrecoveries/recovery/AzureSnapshotBasedRecoveryIT.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.snapshotbasedrecoveries.recovery; import fixture.azure.AzureHttpFixture; +import fixture.azure.MockAzureBlobStore; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Booleans; @@ -37,7 +38,8 @@ public class AzureSnapshotBasedRecoveryIT extends AbstractSnapshotBasedRecoveryR AZURE_TEST_CONTAINER, System.getProperty("test.azure.tenant_id"), System.getProperty("test.azure.client_id"), - AzureHttpFixture.sharedKeyForAccountPredicate(AZURE_TEST_ACCOUNT) + AzureHttpFixture.sharedKeyForAccountPredicate(AZURE_TEST_ACCOUNT), + MockAzureBlobStore.LeaseExpiryPredicate.NEVER_EXPIRE ); private static TestTrustStore trustStore = new TestTrustStore( diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AzureRepositoryAnalysisRestIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AzureRepositoryAnalysisRestIT.java index a9b8fe51c01cc..03906b3cf69da 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AzureRepositoryAnalysisRestIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/azure/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/AzureRepositoryAnalysisRestIT.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; import java.util.function.Predicate; import static org.hamcrest.Matchers.blankOrNullString; @@ -49,7 +50,10 @@ public class AzureRepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRes AZURE_TEST_CONTAINER, AZURE_TEST_TENANT_ID, AZURE_TEST_CLIENT_ID, - decideAuthHeaderPredicate() + decideAuthHeaderPredicate(), + // 5% of the time, in a contended lease scenario, expire the existing lease + (currentLeaseId, requestLeaseId) -> currentLeaseId.equals(requestLeaseId) == false + && ThreadLocalRandom.current().nextDouble() < 0.05 ); private static Predicate decideAuthHeaderPredicate() { @@ -78,12 +82,6 @@ private static Predicate decideAuthHeaderPredicate() { () -> "ignored;DefaultEndpointsProtocol=http;BlobEndpoint=" + fixture.getAddress(), s -> USE_FIXTURE ) - .apply(c -> { - if (USE_FIXTURE) { - // test fixture does not support CAS yet; TODO fix this - c.systemProperty("test.repository_test_kit.skip_cas", "true"); - } - }) .systemProperty( "tests.azure.credentials.disable_instance_discovery", () -> "true", From 24bc505e28cadad4a3253a458ce6493a916b22e8 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Fri, 29 Nov 2024 14:07:48 +1100 Subject: [PATCH 12/27] [Test] Increase test secret key length (#117675) Running with FIPS approved mode requires secret keys to be at least 114 bits long. Relates: #117324 Resolves: #117596 Resolves: #117709 Resolves: #117710 Resolves: #117711 Resolves: #117712 --- .../RepositoryS3RestReloadCredentialsIT.java | 19 +++++++++++++------ muted-tests.yml | 2 -- .../fixture/aws/sts/AwsStsHttpHandler.java | 3 ++- .../fixture/aws/imds/Ec2ImdsHttpHandler.java | 3 ++- .../org/elasticsearch/test/ESTestCase.java | 7 +++++++ 5 files changed, 24 insertions(+), 10 deletions(-) diff --git a/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3RestReloadCredentialsIT.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3RestReloadCredentialsIT.java index 430c0a1994967..1f09fa6b081b9 100644 --- a/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3RestReloadCredentialsIT.java +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3RestReloadCredentialsIT.java @@ -10,6 +10,7 @@ package org.elasticsearch.repositories.s3; import fixture.s3.S3HttpFixture; +import io.netty.handler.codec.http.HttpMethod; import org.elasticsearch.client.Request; import org.elasticsearch.client.ResponseException; @@ -61,8 +62,6 @@ protected String getTestRestCluster() { } public void testReloadCredentialsFromKeystore() throws IOException { - assumeFalse("doesn't work in a FIPS JVM, but that's ok", inFipsJvm()); - // Register repository (?verify=false because we don't have access to the blob store yet) final var repositoryName = randomIdentifier(); registerRepository( @@ -77,15 +76,16 @@ public void testReloadCredentialsFromKeystore() throws IOException { final var accessKey1 = randomIdentifier(); repositoryAccessKey = accessKey1; keystoreSettings.put("s3.client.default.access_key", accessKey1); - keystoreSettings.put("s3.client.default.secret_key", randomIdentifier()); + keystoreSettings.put("s3.client.default.secret_key", randomSecretKey()); cluster.updateStoredSecureSettings(); - assertOK(client().performRequest(new Request("POST", "/_nodes/reload_secure_settings"))); + + assertOK(client().performRequest(createReloadSecureSettingsRequest())); // Check access using initial credentials assertOK(client().performRequest(verifyRequest)); // Rotate credentials in blob store - final var accessKey2 = randomValueOtherThan(accessKey1, ESTestCase::randomIdentifier); + final var accessKey2 = randomValueOtherThan(accessKey1, ESTestCase::randomSecretKey); repositoryAccessKey = accessKey2; // Ensure that initial credentials now invalid @@ -99,10 +99,17 @@ public void testReloadCredentialsFromKeystore() throws IOException { // Set up refreshed credentials keystoreSettings.put("s3.client.default.access_key", accessKey2); cluster.updateStoredSecureSettings(); - assertOK(client().performRequest(new Request("POST", "/_nodes/reload_secure_settings"))); + assertOK(client().performRequest(createReloadSecureSettingsRequest())); // Check access using refreshed credentials assertOK(client().performRequest(verifyRequest)); } + private Request createReloadSecureSettingsRequest() throws IOException { + return newXContentRequest( + HttpMethod.POST, + "/_nodes/reload_secure_settings", + (b, p) -> inFipsJvm() ? b.field("secure_settings_password", "keystore-password") : b + ); + } } diff --git a/muted-tests.yml b/muted-tests.yml index d703cfaa1b9aa..c3f67f97011ee 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -216,8 +216,6 @@ tests: - class: org.elasticsearch.reservedstate.service.FileSettingsServiceTests method: testStopWorksInMiddleOfProcessing issue: https://github.com/elastic/elasticsearch/issues/117591 -- class: org.elasticsearch.repositories.s3.RepositoryS3ClientYamlTestSuiteIT - issue: https://github.com/elastic/elasticsearch/issues/117596 - class: "org.elasticsearch.xpack.esql.qa.multi_node.EsqlSpecIT" method: "test {scoring.*}" issue: https://github.com/elastic/elasticsearch/issues/117641 diff --git a/test/fixtures/aws-sts-fixture/src/main/java/fixture/aws/sts/AwsStsHttpHandler.java b/test/fixtures/aws-sts-fixture/src/main/java/fixture/aws/sts/AwsStsHttpHandler.java index 84541f5e15211..ac3299f157485 100644 --- a/test/fixtures/aws-sts-fixture/src/main/java/fixture/aws/sts/AwsStsHttpHandler.java +++ b/test/fixtures/aws-sts-fixture/src/main/java/fixture/aws/sts/AwsStsHttpHandler.java @@ -28,6 +28,7 @@ import java.util.stream.Collectors; import static org.elasticsearch.test.ESTestCase.randomIdentifier; +import static org.elasticsearch.test.ESTestCase.randomSecretKey; /** * Minimal HTTP handler that emulates the AWS STS server @@ -102,7 +103,7 @@ public void handle(final HttpExchange exchange) throws IOException { ROLE_ARN, ROLE_NAME, sessionToken, - randomIdentifier(), + randomSecretKey(), ZonedDateTime.now().plusDays(1L).format(DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ssZ")), accessKey ).getBytes(StandardCharsets.UTF_8); diff --git a/test/fixtures/ec2-imds-fixture/src/main/java/fixture/aws/imds/Ec2ImdsHttpHandler.java b/test/fixtures/ec2-imds-fixture/src/main/java/fixture/aws/imds/Ec2ImdsHttpHandler.java index a92f1bdc5f9ae..bc87eff592bec 100644 --- a/test/fixtures/ec2-imds-fixture/src/main/java/fixture/aws/imds/Ec2ImdsHttpHandler.java +++ b/test/fixtures/ec2-imds-fixture/src/main/java/fixture/aws/imds/Ec2ImdsHttpHandler.java @@ -28,6 +28,7 @@ import java.util.function.BiConsumer; import static org.elasticsearch.test.ESTestCase.randomIdentifier; +import static org.elasticsearch.test.ESTestCase.randomSecretKey; /** * Minimal HTTP handler that emulates the EC2 IMDS server @@ -84,7 +85,7 @@ public void handle(final HttpExchange exchange) throws IOException { accessKey, ZonedDateTime.now(Clock.systemUTC()).plusDays(1L).format(DateTimeFormatter.ISO_DATE_TIME), randomIdentifier(), - randomIdentifier(), + randomSecretKey(), sessionToken ).getBytes(StandardCharsets.UTF_8); exchange.getResponseHeaders().add("Content-Type", "application/json"); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 5b2beaee00bfe..d983fc854bdfd 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -1358,6 +1358,13 @@ public static String randomDateFormatterPattern() { return randomFrom(FormatNames.values()).getName(); } + /** + * Generate a random string of at least 112 bits to satisfy minimum entropy requirement when running in FIPS mode. + */ + public static String randomSecretKey() { + return randomAlphaOfLengthBetween(14, 20); + } + /** * Randomly choose between {@link EsExecutors#DIRECT_EXECUTOR_SERVICE} (which does not fork), {@link ThreadPool#generic}, and one of the * other named threadpool executors. From 5935f766df80325f748c3193e13e6e74fb5c1f37 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Fri, 29 Nov 2024 17:44:27 +1100 Subject: [PATCH 13/27] Mute org.elasticsearch.xpack.inference.InferenceCrudIT testSupportedStream #117745 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index c3f67f97011ee..40d3dcf46e1b9 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -222,6 +222,9 @@ tests: - class: "org.elasticsearch.xpack.esql.qa.single_node.EsqlSpecIT" method: "test {scoring.*}" issue: https://github.com/elastic/elasticsearch/issues/117641 +- class: org.elasticsearch.xpack.inference.InferenceCrudIT + method: testSupportedStream + issue: https://github.com/elastic/elasticsearch/issues/117745 # Examples: # From 17d280363c62dc4d35c320246d36ec8cd14e4533 Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 29 Nov 2024 09:54:38 +0000 Subject: [PATCH 14/27] Add YAML test for status in indices stats (#116711) The feature added in #81954 lacks coverage in BwC situations. This commit adds a YAML test to address that. --- .../indices.stats/15_open_closed_state.yml | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) create mode 100644 rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.stats/15_open_closed_state.yml diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.stats/15_open_closed_state.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.stats/15_open_closed_state.yml new file mode 100644 index 0000000000000..94b6a3acc83a8 --- /dev/null +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/indices.stats/15_open_closed_state.yml @@ -0,0 +1,22 @@ +--- +"Ensure index state is exposed": + - requires: + cluster_features: ["gte_v8.1.0"] + reason: index state added to stats in 8.1.0 + + - do: + indices.create: + index: openindex + - do: + indices.create: + index: closedindex + - do: + indices.close: + index: closedindex + - do: + indices.stats: + expand_wildcards: [open,closed] + forbid_closed_indices: false + + - match: { indices.openindex.status: open } + - match: { indices.closedindex.status: close } From c3f9e0172333b8edae525865c9d84b29a1c6ab8f Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 29 Nov 2024 09:58:09 +0000 Subject: [PATCH 15/27] Migrate `repository-s3` YAML tests to Java REST tests (#117628) Today these YAML tests rely on a bunch of rather complex setup organised by Gradle, and contain lots of duplication and coincident strings, mostly because that was the only way to achieve what we wanted before we could orchestrate test clusters and fixtures directly from Java test suites. We're not actually running the YAML tests in ways that take advantage of their YAMLness (e.g. in mixed-version clusters, or from other client libraries). This commit replaces these tests with Java REST tests which enormously simplifies this area of code. Relates ES-9984 --- modules/repository-s3/build.gradle | 118 +----- .../s3/S3RepositoryThirdPartyTests.java | 7 +- .../s3/AbstractRepositoryS3RestTestCase.java | 383 ++++++++++++++++++ .../RepositoryS3BasicCredentialsRestIT.java | 65 +++ .../s3/RepositoryS3EcsCredentialsRestIT.java} | 44 +- .../RepositoryS3ImdsV1CredentialsRestIT.java | 73 ++++ ...ositoryS3MinioBasicCredentialsRestIT.java} | 44 +- .../RepositoryS3SessionCredentialsRestIT.java | 72 ++++ .../s3/RepositoryS3StsCredentialsRestIT.java} | 64 +-- .../repositories/s3/S3BlobStore.java | 2 +- .../repositories/s3/S3Service.java | 8 +- .../resources/aws-web-identity-token-file | 1 - .../s3/RepositoryS3ClientYamlTestSuiteIT.java | 57 +-- ...oryS3RegionalStsClientYamlTestSuiteIT.java | 12 +- .../20_repository_permanent_credentials.yml | 265 +----------- .../30_repository_temporary_credentials.yml | 278 ------------- .../40_repository_ec2_credentials.yml | 278 ------------- .../50_repository_ecs_credentials.yml | 278 ------------- .../60_repository_sts_credentials.yml | 279 ------------- .../fixtures/minio/MinioTestContainer.java | 12 +- .../main/java/fixture/s3/S3HttpFixture.java | 4 - .../local/AbstractLocalClusterFactory.java | 2 + .../minio/MinioSearchableSnapshotsIT.java | 7 +- .../MinioRepositoryAnalysisRestIT.java | 7 +- 24 files changed, 765 insertions(+), 1595 deletions(-) create mode 100644 modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/AbstractRepositoryS3RestTestCase.java create mode 100644 modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3BasicCredentialsRestIT.java rename modules/repository-s3/src/{yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3EcsClientYamlTestSuiteIT.java => javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3EcsCredentialsRestIT.java} (59%) create mode 100644 modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3ImdsV1CredentialsRestIT.java rename modules/repository-s3/src/{yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3MinioClientYamlTestSuiteIT.java => javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3MinioBasicCredentialsRestIT.java} (50%) create mode 100644 modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3SessionCredentialsRestIT.java rename modules/repository-s3/src/{yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3StsClientYamlTestSuiteIT.java => javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3StsCredentialsRestIT.java} (53%) delete mode 100644 modules/repository-s3/src/test/resources/aws-web-identity-token-file delete mode 100644 modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml delete mode 100644 modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml delete mode 100644 modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml delete mode 100644 modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/60_repository_sts_credentials.yml diff --git a/modules/repository-s3/build.gradle b/modules/repository-s3/build.gradle index ed1777891f40d..2cfb5d23db4ff 100644 --- a/modules/repository-s3/build.gradle +++ b/modules/repository-s3/build.gradle @@ -43,19 +43,24 @@ dependencies { api 'javax.xml.bind:jaxb-api:2.2.2' testImplementation project(':test:fixtures:s3-fixture') - yamlRestTestImplementation project(":test:framework") - yamlRestTestImplementation project(':test:fixtures:s3-fixture') - yamlRestTestImplementation project(':test:fixtures:ec2-imds-fixture') - yamlRestTestImplementation project(':test:fixtures:aws-sts-fixture') - yamlRestTestImplementation project(':test:fixtures:minio-fixture') - internalClusterTestImplementation project(':test:fixtures:minio-fixture') - javaRestTestImplementation project(":test:framework") - javaRestTestImplementation project(':test:fixtures:s3-fixture') - javaRestTestImplementation project(':modules:repository-s3') + internalClusterTestImplementation project(':test:fixtures:minio-fixture') + internalClusterTestRuntimeOnly "org.slf4j:slf4j-simple:${versions.slf4j}" + yamlRestTestImplementation project(':modules:repository-s3') + yamlRestTestImplementation project(':test:fixtures:s3-fixture') + yamlRestTestImplementation project(':test:fixtures:testcontainer-utils') + yamlRestTestImplementation project(':test:framework') yamlRestTestRuntimeOnly "org.slf4j:slf4j-simple:${versions.slf4j}" - internalClusterTestRuntimeOnly "org.slf4j:slf4j-simple:${versions.slf4j}" + + javaRestTestImplementation project(':modules:repository-s3') + javaRestTestImplementation project(':test:fixtures:aws-sts-fixture') + javaRestTestImplementation project(':test:fixtures:ec2-imds-fixture') + javaRestTestImplementation project(':test:fixtures:minio-fixture') + javaRestTestImplementation project(':test:fixtures:s3-fixture') + javaRestTestImplementation project(':test:fixtures:testcontainer-utils') + javaRestTestImplementation project(':test:framework') + javaRestTestRuntimeOnly "org.slf4j:slf4j-simple:${versions.slf4j}" } restResources { @@ -82,90 +87,25 @@ def testRepositoryCreds = tasks.register("testRepositoryCreds", Test) { testClassesDirs = sourceSets.test.output.classesDirs } -tasks.named('check').configure { - dependsOn(testRepositoryCreds) -} - tasks.named('test').configure { // this is tested explicitly in separate test tasks exclude '**/RepositoryCredentialsTests.class' } boolean useFixture = false - -// We test against two repositories, one which uses the usual two-part "permanent" credentials and -// the other which uses three-part "temporary" or "session" credentials. - String s3PermanentAccessKey = System.getenv("amazon_s3_access_key") String s3PermanentSecretKey = System.getenv("amazon_s3_secret_key") String s3PermanentBucket = System.getenv("amazon_s3_bucket") String s3PermanentBasePath = System.getenv("amazon_s3_base_path") -String s3TemporaryAccessKey = System.getenv("amazon_s3_access_key_temporary") -String s3TemporarySecretKey = System.getenv("amazon_s3_secret_key_temporary") -String s3TemporarySessionToken = System.getenv("amazon_s3_session_token_temporary") -String s3TemporaryBucket = System.getenv("amazon_s3_bucket_temporary") -String s3TemporaryBasePath = System.getenv("amazon_s3_base_path_temporary") - -String s3EC2Bucket = System.getenv("amazon_s3_bucket_ec2") -String s3EC2BasePath = System.getenv("amazon_s3_base_path_ec2") - -String s3ECSBucket = System.getenv("amazon_s3_bucket_ecs") -String s3ECSBasePath = System.getenv("amazon_s3_base_path_ecs") - -String s3STSBucket = System.getenv("amazon_s3_bucket_sts") -String s3STSBasePath = System.getenv("amazon_s3_base_path_sts") - -boolean s3DisableChunkedEncoding = buildParams.random.nextBoolean() - -// If all these variables are missing then we are testing against the internal fixture instead, which has the following -// credentials hard-coded in. +// If all these variables are missing then we are testing against the internal fixture instead, which has the following credentials hard-coded in. if (!s3PermanentAccessKey && !s3PermanentSecretKey && !s3PermanentBucket && !s3PermanentBasePath) { + useFixture = true s3PermanentAccessKey = 's3_test_access_key' s3PermanentSecretKey = 's3_test_secret_key' s3PermanentBucket = 'bucket' s3PermanentBasePath = 'base_path' - useFixture = true -} -if (!s3TemporaryAccessKey && !s3TemporarySecretKey && !s3TemporaryBucket && !s3TemporaryBasePath && !s3TemporarySessionToken) { - s3TemporaryAccessKey = 'session_token_access_key' - s3TemporarySecretKey = 'session_token_secret_key' - s3TemporaryBucket = 'session_token_bucket' - s3TemporaryBasePath = 'session_token_base_path' -} - -if (!s3EC2Bucket && !s3EC2BasePath && !s3ECSBucket && !s3ECSBasePath) { - s3EC2Bucket = 'ec2_bucket' - s3EC2BasePath = 'ec2_base_path' - s3ECSBucket = 'ecs_bucket' - s3ECSBasePath = 'ecs_base_path' -} - -if (!s3STSBucket && !s3STSBasePath) { - s3STSBucket = 'sts_bucket' - s3STSBasePath = 'sts_base_path' -} - -tasks.named("processYamlRestTestResources").configure { - from("src/test/resources") { - include "aws-web-identity-token-file" - } - Map expansions = [ - 'permanent_bucket' : s3PermanentBucket, - 'permanent_base_path' : s3PermanentBasePath + "_integration_tests", - 'temporary_bucket' : s3TemporaryBucket, - 'temporary_base_path' : s3TemporaryBasePath + "_integration_tests", - 'ec2_bucket' : s3EC2Bucket, - 'ec2_base_path' : s3EC2BasePath, - 'ecs_bucket' : s3ECSBucket, - 'ecs_base_path' : s3ECSBasePath, - 'sts_bucket' : s3STSBucket, - 'sts_base_path' : s3STSBasePath, - 'disable_chunked_encoding': s3DisableChunkedEncoding - ] - inputs.properties(expansions) - filter("tokens" : expansions.collectEntries {k, v -> [k, v.toString()]} /* must be a map of strings */, ReplaceTokens.class) } tasks.named("internalClusterTest").configure { @@ -175,22 +115,7 @@ tasks.named("internalClusterTest").configure { systemProperty 'es.insecure_network_trace_enabled', 'true' } -tasks.named("yamlRestTest").configure { - systemProperty("s3PermanentAccessKey", s3PermanentAccessKey) - systemProperty("s3PermanentSecretKey", s3PermanentSecretKey) - systemProperty("s3TemporaryAccessKey", s3TemporaryAccessKey) - systemProperty("s3TemporarySecretKey", s3TemporarySecretKey) - systemProperty("s3EC2AccessKey", s3PermanentAccessKey) - - // ideally we could resolve an env path in cluster config as resource similar to configuring a config file - // not sure how common this is, but it would be nice to support - File awsWebIdentityTokenExternalLocation = file('src/test/resources/aws-web-identity-token-file') - // The web identity token can be read only from the plugin config directory because of security restrictions - // Ideally we would create a symlink, but extraConfigFile doesn't support it - nonInputProperties.systemProperty("awsWebIdentityTokenExternalLocation", awsWebIdentityTokenExternalLocation.getAbsolutePath()) -} - -// 3rd Party Tests +// 3rd Party Tests, i.e. testing against a real S3 repository tasks.register("s3ThirdPartyTest", Test) { SourceSetContainer sourceSets = project.getExtensions().getByType(SourceSetContainer.class); SourceSet internalTestSourceSet = sourceSets.getByName(InternalClusterTestPlugin.SOURCE_SET_NAME) @@ -198,13 +123,13 @@ tasks.register("s3ThirdPartyTest", Test) { setClasspath(internalTestSourceSet.getRuntimeClasspath()) include '**/S3RepositoryThirdPartyTests.class' systemProperty("tests.use.fixture", Boolean.toString(useFixture)) - - // test container accesses ~/.testcontainers.properties read - systemProperty "tests.security.manager", "false" systemProperty 'test.s3.account', s3PermanentAccessKey systemProperty 'test.s3.key', s3PermanentSecretKey systemProperty 'test.s3.bucket', s3PermanentBucket nonInputProperties.systemProperty 'test.s3.base', s3PermanentBasePath + "_third_party_tests_" + buildParams.testSeed + + // test container accesses ~/.testcontainers.properties read + systemProperty "tests.security.manager", "false" } tasks.named("thirdPartyAudit").configure { @@ -241,5 +166,6 @@ tasks.named("thirdPartyAudit").configure { tasks.named("check").configure { dependsOn(tasks.withType(Test)) + dependsOn(testRepositoryCreds) } diff --git a/modules/repository-s3/src/internalClusterTest/java/org/elasticsearch/repositories/s3/S3RepositoryThirdPartyTests.java b/modules/repository-s3/src/internalClusterTest/java/org/elasticsearch/repositories/s3/S3RepositoryThirdPartyTests.java index 3552cb8d9389a..4cebedebfba07 100644 --- a/modules/repository-s3/src/internalClusterTest/java/org/elasticsearch/repositories/s3/S3RepositoryThirdPartyTests.java +++ b/modules/repository-s3/src/internalClusterTest/java/org/elasticsearch/repositories/s3/S3RepositoryThirdPartyTests.java @@ -61,7 +61,12 @@ public class S3RepositoryThirdPartyTests extends AbstractThirdPartyRepositoryTes static final boolean USE_FIXTURE = Booleans.parseBoolean(System.getProperty("tests.use.fixture", "true")); @ClassRule - public static MinioTestContainer minio = new MinioTestContainer(USE_FIXTURE); + public static MinioTestContainer minio = new MinioTestContainer( + USE_FIXTURE, + System.getProperty("test.s3.account"), + System.getProperty("test.s3.key"), + System.getProperty("test.s3.bucket") + ); @Override protected Collection> getPlugins() { diff --git a/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/AbstractRepositoryS3RestTestCase.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/AbstractRepositoryS3RestTestCase.java new file mode 100644 index 0000000000000..2199a64521759 --- /dev/null +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/AbstractRepositoryS3RestTestCase.java @@ -0,0 +1,383 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories.s3; + +import io.netty.handler.codec.http.HttpMethod; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.cluster.routing.Murmur3HashFunction; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.rest.ESRestTestCase; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Set; +import java.util.function.UnaryOperator; +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +public abstract class AbstractRepositoryS3RestTestCase extends ESRestTestCase { + + public record TestRepository(String repositoryName, String clientName, String bucketName, String basePath) { + + public Closeable register() throws IOException { + return register(UnaryOperator.identity()); + } + + public Closeable register(UnaryOperator settingsUnaryOperator) throws IOException { + assertOK(client().performRequest(getRegisterRequest(settingsUnaryOperator))); + return () -> assertOK(client().performRequest(new Request("DELETE", "/_snapshot/" + repositoryName()))); + } + + private Request getRegisterRequest(UnaryOperator settingsUnaryOperator) throws IOException { + return newXContentRequest( + HttpMethod.PUT, + "/_snapshot/" + repositoryName(), + (b, p) -> b.field("type", S3Repository.TYPE) + .startObject("settings") + .value( + settingsUnaryOperator.apply( + Settings.builder() + .put("bucket", bucketName()) + .put("base_path", basePath()) + .put("client", clientName()) + .put("canned_acl", "private") + .put("storage_class", "standard") + .put("disable_chunked_encoding", randomBoolean()) + .build() + ) + ) + .endObject() + ); + } + } + + protected abstract String getBucketName(); + + protected abstract String getBasePath(); + + protected abstract String getClientName(); + + protected static String getIdentifierPrefix(String testSuiteName) { + return testSuiteName + "-" + Integer.toString(Murmur3HashFunction.hash(testSuiteName + System.getProperty("tests.seed")), 16) + "-"; + } + + private TestRepository newTestRepository() { + return new TestRepository(randomIdentifier(), getClientName(), getBucketName(), getBasePath()); + } + + private static UnaryOperator readonlyOperator(Boolean readonly) { + return readonly == null + ? UnaryOperator.identity() + : s -> Settings.builder().put(s).put(BlobStoreRepository.READONLY_SETTING_KEY, readonly).build(); + } + + public void testGetRepository() throws IOException { + testGetRepository(null); + } + + public void testGetRepositoryReadonlyTrue() throws IOException { + testGetRepository(Boolean.TRUE); + } + + public void testGetRepositoryReadonlyFalse() throws IOException { + testGetRepository(Boolean.FALSE); + } + + private void testGetRepository(Boolean readonly) throws IOException { + final var repository = newTestRepository(); + try (var ignored = repository.register(readonlyOperator(readonly))) { + final var repositoryName = repository.repositoryName(); + final var responseObjectPath = assertOKAndCreateObjectPath( + client().performRequest(new Request("GET", "/_snapshot/" + repositoryName)) + ); + + assertEquals("s3", responseObjectPath.evaluate(repositoryName + ".type")); + assertNotNull(responseObjectPath.evaluate(repositoryName + ".settings")); + assertEquals(repository.bucketName(), responseObjectPath.evaluate(repositoryName + ".settings.bucket")); + assertEquals(repository.clientName(), responseObjectPath.evaluate(repositoryName + ".settings.client")); + assertEquals(repository.basePath(), responseObjectPath.evaluate(repositoryName + ".settings.base_path")); + assertEquals("private", responseObjectPath.evaluate(repositoryName + ".settings.canned_acl")); + assertEquals("standard", responseObjectPath.evaluate(repositoryName + ".settings.storage_class")); + assertNull(responseObjectPath.evaluate(repositoryName + ".settings.access_key")); + assertNull(responseObjectPath.evaluate(repositoryName + ".settings.secret_key")); + assertNull(responseObjectPath.evaluate(repositoryName + ".settings.session_token")); + + if (readonly == null) { + assertNull(responseObjectPath.evaluate(repositoryName + ".settings." + BlobStoreRepository.READONLY_SETTING_KEY)); + } else { + assertEquals( + Boolean.toString(readonly), + responseObjectPath.evaluate(repositoryName + ".settings." + BlobStoreRepository.READONLY_SETTING_KEY) + ); + } + } + } + + public void testNonexistentBucket() throws Exception { + testNonexistentBucket(null); + } + + public void testNonexistentBucketReadonlyTrue() throws Exception { + testNonexistentBucket(Boolean.TRUE); + } + + public void testNonexistentBucketReadonlyFalse() throws Exception { + testNonexistentBucket(Boolean.FALSE); + } + + private void testNonexistentBucket(Boolean readonly) throws Exception { + final var repository = new TestRepository( + randomIdentifier(), + getClientName(), + randomValueOtherThan(getBucketName(), ESTestCase::randomIdentifier), + getBasePath() + ); + final var registerRequest = repository.getRegisterRequest(readonlyOperator(readonly)); + + final var responseException = expectThrows(ResponseException.class, () -> client().performRequest(registerRequest)); + assertEquals(RestStatus.INTERNAL_SERVER_ERROR.getStatus(), responseException.getResponse().getStatusLine().getStatusCode()); + assertThat( + responseException.getMessage(), + allOf(containsString("repository_verification_exception"), containsString("is not accessible on master node")) + ); + } + + public void testNonexistentClient() throws Exception { + testNonexistentClient(null); + } + + public void testNonexistentClientReadonlyTrue() throws Exception { + testNonexistentClient(Boolean.TRUE); + } + + public void testNonexistentClientReadonlyFalse() throws Exception { + testNonexistentClient(Boolean.FALSE); + } + + private void testNonexistentClient(Boolean readonly) throws Exception { + final var repository = new TestRepository( + randomIdentifier(), + randomValueOtherThanMany(c -> c.equals(getClientName()) || c.equals("default"), ESTestCase::randomIdentifier), + getBucketName(), + getBasePath() + ); + final var registerRequest = repository.getRegisterRequest(readonlyOperator(readonly)); + + final var responseException = expectThrows(ResponseException.class, () -> client().performRequest(registerRequest)); + assertEquals(RestStatus.INTERNAL_SERVER_ERROR.getStatus(), responseException.getResponse().getStatusLine().getStatusCode()); + assertThat( + responseException.getMessage(), + allOf( + containsString("repository_verification_exception"), + containsString("is not accessible on master node"), + containsString("illegal_argument_exception"), + containsString("Unknown s3 client name") + ) + ); + } + + public void testNonexistentSnapshot() throws Exception { + testNonexistentSnapshot(null); + } + + public void testNonexistentSnapshotReadonlyTrue() throws Exception { + testNonexistentSnapshot(Boolean.TRUE); + } + + public void testNonexistentSnapshotReadonlyFalse() throws Exception { + testNonexistentSnapshot(Boolean.FALSE); + } + + private void testNonexistentSnapshot(Boolean readonly) throws Exception { + final var repository = newTestRepository(); + try (var ignored = repository.register(readonlyOperator(readonly))) { + final var repositoryName = repository.repositoryName(); + + final var getSnapshotRequest = new Request("GET", "/_snapshot/" + repositoryName + "/" + randomIdentifier()); + final var getSnapshotException = expectThrows(ResponseException.class, () -> client().performRequest(getSnapshotRequest)); + assertEquals(RestStatus.NOT_FOUND.getStatus(), getSnapshotException.getResponse().getStatusLine().getStatusCode()); + assertThat(getSnapshotException.getMessage(), containsString("snapshot_missing_exception")); + + final var restoreRequest = new Request("POST", "/_snapshot/" + repositoryName + "/" + randomIdentifier() + "/_restore"); + if (randomBoolean()) { + restoreRequest.addParameter("wait_for_completion", Boolean.toString(randomBoolean())); + } + final var restoreException = expectThrows(ResponseException.class, () -> client().performRequest(restoreRequest)); + assertEquals(RestStatus.INTERNAL_SERVER_ERROR.getStatus(), restoreException.getResponse().getStatusLine().getStatusCode()); + assertThat(restoreException.getMessage(), containsString("snapshot_restore_exception")); + + if (readonly != Boolean.TRUE) { + final var deleteRequest = new Request("DELETE", "/_snapshot/" + repositoryName + "/" + randomIdentifier()); + final var deleteException = expectThrows(ResponseException.class, () -> client().performRequest(deleteRequest)); + assertEquals(RestStatus.NOT_FOUND.getStatus(), deleteException.getResponse().getStatusLine().getStatusCode()); + assertThat(deleteException.getMessage(), containsString("snapshot_missing_exception")); + } + } + } + + public void testUsageStats() throws Exception { + testUsageStats(null); + } + + public void testUsageStatsReadonlyTrue() throws Exception { + testUsageStats(Boolean.TRUE); + } + + public void testUsageStatsReadonlyFalse() throws Exception { + testUsageStats(Boolean.FALSE); + } + + private void testUsageStats(Boolean readonly) throws Exception { + final var repository = newTestRepository(); + try (var ignored = repository.register(readonlyOperator(readonly))) { + final var responseObjectPath = assertOKAndCreateObjectPath(client().performRequest(new Request("GET", "/_cluster/stats"))); + assertThat(responseObjectPath.evaluate("repositories.s3.count"), equalTo(1)); + + if (readonly == Boolean.TRUE) { + assertThat(responseObjectPath.evaluate("repositories.s3.read_only"), equalTo(1)); + assertNull(responseObjectPath.evaluate("repositories.s3.read_write")); + } else { + assertNull(responseObjectPath.evaluate("repositories.s3.read_only")); + assertThat(responseObjectPath.evaluate("repositories.s3.read_write"), equalTo(1)); + } + } + } + + public void testSnapshotAndRestore() throws Exception { + final var repository = newTestRepository(); + try (var ignored = repository.register()) { + final var repositoryName = repository.repositoryName(); + final var indexName = randomIdentifier(); + final var snapshotsToDelete = new ArrayList(2); + + try { + indexDocuments(indexName, """ + {"index":{"_id":"1"}} + {"snapshot":"one"} + {"index":{"_id":"2"}} + {"snapshot":"one"} + {"index":{"_id":"3"}} + {"snapshot":"one"} + """, 3); + + // create the first snapshot + final var snapshot1Name = randomIdentifier(); + createSnapshot(repositoryName, snapshotsToDelete, snapshot1Name); + + // check the first snapshot's status + { + final var snapshotStatusResponse = assertOKAndCreateObjectPath( + client().performRequest(new Request("GET", "/_snapshot/" + repositoryName + "/" + snapshot1Name + "/_status")) + ); + assertEquals(snapshot1Name, snapshotStatusResponse.evaluate("snapshots.0.snapshot")); + assertEquals("SUCCESS", snapshotStatusResponse.evaluate("snapshots.0.state")); + } + + // add more documents to the index + indexDocuments(indexName, """ + {"index":{"_id":"4"}} + {"snapshot":"one"} + {"index":{"_id":"5"}} + {"snapshot":"one"} + {"index":{"_id":"6"}} + {"snapshot":"one"} + {"index":{"_id":"7"}} + {"snapshot":"one"} + """, 7); + + // create the second snapshot + final var snapshot2Name = randomValueOtherThan(snapshot1Name, ESTestCase::randomIdentifier); + createSnapshot(repositoryName, snapshotsToDelete, snapshot2Name); + + // list the snapshots + { + final var listSnapshotsResponse = assertOKAndCreateObjectPath( + client().performRequest( + new Request("GET", "/_snapshot/" + repositoryName + "/" + snapshot1Name + "," + snapshot2Name) + ) + ); + assertEquals(2, listSnapshotsResponse.evaluateArraySize("snapshots")); + assertEquals( + Set.of(snapshot1Name, snapshot2Name), + Set.of( + listSnapshotsResponse.evaluate("snapshots.0.snapshot"), + listSnapshotsResponse.evaluate("snapshots.1.snapshot") + ) + ); + assertEquals("SUCCESS", listSnapshotsResponse.evaluate("snapshots.0.state")); + assertEquals("SUCCESS", listSnapshotsResponse.evaluate("snapshots.1.state")); + } + + // delete and restore the index from snapshot 2 + deleteAndRestoreIndex(indexName, repositoryName, snapshot2Name, 7); + + // delete and restore the index from snapshot 1 + deleteAndRestoreIndex(indexName, repositoryName, snapshot1Name, 3); + } finally { + if (snapshotsToDelete.isEmpty() == false) { + assertOK( + client().performRequest( + new Request( + "DELETE", + "/_snapshot/" + repositoryName + "/" + snapshotsToDelete.stream().collect(Collectors.joining(",")) + ) + ) + ); + } + } + } + } + + private static void deleteAndRestoreIndex(String indexName, String repositoryName, String snapshot2Name, int expectedDocCount) + throws IOException { + assertOK(client().performRequest(new Request("DELETE", "/" + indexName))); + final var restoreRequest = new Request("POST", "/_snapshot/" + repositoryName + "/" + snapshot2Name + "/_restore"); + restoreRequest.addParameter("wait_for_completion", "true"); + assertOK(client().performRequest(restoreRequest)); + assertIndexDocCount(indexName, expectedDocCount); + } + + private static void indexDocuments(String indexName, String body, int expectedDocCount) throws IOException { + // create and populate an index + final var indexDocsRequest = new Request("POST", "/" + indexName + "/_bulk"); + indexDocsRequest.addParameter("refresh", "true"); + indexDocsRequest.setJsonEntity(body); + assertFalse(assertOKAndCreateObjectPath(client().performRequest(indexDocsRequest)).evaluate("errors")); + + // check the index contents + assertIndexDocCount(indexName, expectedDocCount); + } + + private static void createSnapshot(String repositoryName, ArrayList snapshotsToDelete, String snapshotName) throws IOException { + final var createSnapshotRequest = new Request("POST", "/_snapshot/" + repositoryName + "/" + snapshotName); + createSnapshotRequest.addParameter("wait_for_completion", "true"); + final var createSnapshotResponse = assertOKAndCreateObjectPath(client().performRequest(createSnapshotRequest)); + snapshotsToDelete.add(snapshotName); + assertEquals(snapshotName, createSnapshotResponse.evaluate("snapshot.snapshot")); + assertEquals("SUCCESS", createSnapshotResponse.evaluate("snapshot.state")); + assertThat(createSnapshotResponse.evaluate("snapshot.shards.failed"), equalTo(0)); + } + + private static void assertIndexDocCount(String indexName, int expectedCount) throws IOException { + assertThat( + assertOKAndCreateObjectPath(client().performRequest(new Request("GET", "/" + indexName + "/_count"))).evaluate("count"), + equalTo(expectedCount) + ); + } +} diff --git a/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3BasicCredentialsRestIT.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3BasicCredentialsRestIT.java new file mode 100644 index 0000000000000..45844703683bb --- /dev/null +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3BasicCredentialsRestIT.java @@ -0,0 +1,65 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories.s3; + +import fixture.s3.S3HttpFixture; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; + +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.fixtures.testcontainers.TestContainersThreadFilter; +import org.junit.ClassRule; +import org.junit.rules.RuleChain; +import org.junit.rules.TestRule; + +@ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) // https://github.com/elastic/elasticsearch/issues/102482 +public class RepositoryS3BasicCredentialsRestIT extends AbstractRepositoryS3RestTestCase { + + private static final String PREFIX = getIdentifierPrefix("RepositoryS3BasicCredentialsRestIT"); + private static final String BUCKET = PREFIX + "bucket"; + private static final String BASE_PATH = PREFIX + "base_path"; + private static final String ACCESS_KEY = PREFIX + "access-key"; + private static final String SECRET_KEY = PREFIX + "secret-key"; + private static final String CLIENT = "basic_credentials_client"; + + private static final S3HttpFixture s3Fixture = new S3HttpFixture(true, BUCKET, BASE_PATH, S3HttpFixture.fixedAccessKey(ACCESS_KEY)); + + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .module("repository-s3") + .keystore("s3.client." + CLIENT + ".access_key", ACCESS_KEY) + .keystore("s3.client." + CLIENT + ".secret_key", SECRET_KEY) + .setting("s3.client." + CLIENT + ".endpoint", s3Fixture::getAddress) + .build(); + + @ClassRule + public static TestRule ruleChain = RuleChain.outerRule(s3Fixture).around(cluster); + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + @Override + protected String getBucketName() { + return BUCKET; + } + + @Override + protected String getBasePath() { + return BASE_PATH; + } + + @Override + protected String getClientName() { + return CLIENT; + } +} diff --git a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3EcsClientYamlTestSuiteIT.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3EcsCredentialsRestIT.java similarity index 59% rename from modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3EcsClientYamlTestSuiteIT.java rename to modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3EcsCredentialsRestIT.java index bbd003f506ead..267ba6e6b3a13 100644 --- a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3EcsClientYamlTestSuiteIT.java +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3EcsCredentialsRestIT.java @@ -13,18 +13,25 @@ import fixture.s3.DynamicS3Credentials; import fixture.s3.S3HttpFixture; -import com.carrotsearch.randomizedtesting.annotations.Name; -import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; import org.elasticsearch.test.cluster.ElasticsearchCluster; -import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; +import org.elasticsearch.test.fixtures.testcontainers.TestContainersThreadFilter; import org.junit.ClassRule; import org.junit.rules.RuleChain; import org.junit.rules.TestRule; import java.util.Set; -public class RepositoryS3EcsClientYamlTestSuiteIT extends AbstractRepositoryS3ClientYamlTestSuiteIT { +@ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) // https://github.com/elastic/elasticsearch/issues/102482 +public class RepositoryS3EcsCredentialsRestIT extends AbstractRepositoryS3RestTestCase { + + private static final String PREFIX = getIdentifierPrefix("RepositoryS3EcsCredentialsRestIT"); + private static final String BUCKET = PREFIX + "bucket"; + private static final String BASE_PATH = PREFIX + "base_path"; + private static final String CLIENT = "ecs_credentials_client"; private static final DynamicS3Credentials dynamicS3Credentials = new DynamicS3Credentials(); @@ -33,33 +40,34 @@ public class RepositoryS3EcsClientYamlTestSuiteIT extends AbstractRepositoryS3Cl Set.of("/ecs_credentials_endpoint") ); - private static final S3HttpFixture s3Fixture = new S3HttpFixture( - true, - "ecs_bucket", - "ecs_base_path", - dynamicS3Credentials::isAuthorized - ); + private static final S3HttpFixture s3Fixture = new S3HttpFixture(true, BUCKET, BASE_PATH, dynamicS3Credentials::isAuthorized); public static ElasticsearchCluster cluster = ElasticsearchCluster.local() .module("repository-s3") - .setting("s3.client.integration_test_ecs.endpoint", s3Fixture::getAddress) + .setting("s3.client." + CLIENT + ".endpoint", s3Fixture::getAddress) .environment("AWS_CONTAINER_CREDENTIALS_FULL_URI", () -> ec2ImdsHttpFixture.getAddress() + "/ecs_credentials_endpoint") .build(); @ClassRule public static TestRule ruleChain = RuleChain.outerRule(s3Fixture).around(ec2ImdsHttpFixture).around(cluster); - @ParametersFactory - public static Iterable parameters() throws Exception { - return createParameters(new String[] { "repository_s3/50_repository_ecs_credentials" }); + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); } - public RepositoryS3EcsClientYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { - super(testCandidate); + @Override + protected String getBucketName() { + return BUCKET; } @Override - protected String getTestRestCluster() { - return cluster.getHttpAddresses(); + protected String getBasePath() { + return BASE_PATH; + } + + @Override + protected String getClientName() { + return CLIENT; } } diff --git a/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3ImdsV1CredentialsRestIT.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3ImdsV1CredentialsRestIT.java new file mode 100644 index 0000000000000..de9c9b6ae0695 --- /dev/null +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3ImdsV1CredentialsRestIT.java @@ -0,0 +1,73 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories.s3; + +import fixture.aws.imds.Ec2ImdsHttpFixture; +import fixture.s3.DynamicS3Credentials; +import fixture.s3.S3HttpFixture; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; + +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.fixtures.testcontainers.TestContainersThreadFilter; +import org.junit.ClassRule; +import org.junit.rules.RuleChain; +import org.junit.rules.TestRule; + +import java.util.Set; + +@ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) // https://github.com/elastic/elasticsearch/issues/102482 +public class RepositoryS3ImdsV1CredentialsRestIT extends AbstractRepositoryS3RestTestCase { + + private static final String PREFIX = getIdentifierPrefix("RepositoryS3ImdsV1CredentialsRestIT"); + private static final String BUCKET = PREFIX + "bucket"; + private static final String BASE_PATH = PREFIX + "base_path"; + private static final String CLIENT = "imdsv1_credentials_client"; + + private static final DynamicS3Credentials dynamicS3Credentials = new DynamicS3Credentials(); + + private static final Ec2ImdsHttpFixture ec2ImdsHttpFixture = new Ec2ImdsHttpFixture( + dynamicS3Credentials::addValidCredentials, + Set.of() + ); + + private static final S3HttpFixture s3Fixture = new S3HttpFixture(true, BUCKET, BASE_PATH, dynamicS3Credentials::isAuthorized); + + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .module("repository-s3") + .setting("s3.client." + CLIENT + ".endpoint", s3Fixture::getAddress) + .systemProperty("com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", ec2ImdsHttpFixture::getAddress) + .build(); + + @ClassRule + public static TestRule ruleChain = RuleChain.outerRule(ec2ImdsHttpFixture).around(s3Fixture).around(cluster); + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + @Override + protected String getBucketName() { + return BUCKET; + } + + @Override + protected String getBasePath() { + return BASE_PATH; + } + + @Override + protected String getClientName() { + return CLIENT; + } +} diff --git a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3MinioClientYamlTestSuiteIT.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3MinioBasicCredentialsRestIT.java similarity index 50% rename from modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3MinioClientYamlTestSuiteIT.java rename to modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3MinioBasicCredentialsRestIT.java index d2b1413295ceb..93915e8491d5b 100644 --- a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3MinioClientYamlTestSuiteIT.java +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3MinioBasicCredentialsRestIT.java @@ -9,44 +9,56 @@ package org.elasticsearch.repositories.s3; -import com.carrotsearch.randomizedtesting.annotations.Name; -import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; import org.elasticsearch.test.cluster.ElasticsearchCluster; import org.elasticsearch.test.fixtures.minio.MinioTestContainer; import org.elasticsearch.test.fixtures.testcontainers.TestContainersThreadFilter; -import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.junit.ClassRule; import org.junit.rules.RuleChain; import org.junit.rules.TestRule; @ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) -public class RepositoryS3MinioClientYamlTestSuiteIT extends AbstractRepositoryS3ClientYamlTestSuiteIT { +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) // https://github.com/elastic/elasticsearch/issues/102482 +public class RepositoryS3MinioBasicCredentialsRestIT extends AbstractRepositoryS3RestTestCase { - public static MinioTestContainer minio = new MinioTestContainer(); + private static final String PREFIX = getIdentifierPrefix("RepositoryS3MinioBasicCredentialsRestIT"); + private static final String BUCKET = PREFIX + "bucket"; + private static final String BASE_PATH = PREFIX + "base_path"; + private static final String ACCESS_KEY = PREFIX + "access-key"; + private static final String SECRET_KEY = PREFIX + "secret-key"; + private static final String CLIENT = "minio_client"; + + private static final MinioTestContainer minioFixture = new MinioTestContainer(true, ACCESS_KEY, SECRET_KEY, BUCKET); public static ElasticsearchCluster cluster = ElasticsearchCluster.local() .module("repository-s3") - .keystore("s3.client.integration_test_permanent.access_key", System.getProperty("s3PermanentAccessKey")) - .keystore("s3.client.integration_test_permanent.secret_key", System.getProperty("s3PermanentSecretKey")) - .setting("s3.client.integration_test_permanent.endpoint", () -> minio.getAddress()) + .keystore("s3.client." + CLIENT + ".access_key", ACCESS_KEY) + .keystore("s3.client." + CLIENT + ".secret_key", SECRET_KEY) + .setting("s3.client." + CLIENT + ".endpoint", minioFixture::getAddress) .build(); @ClassRule - public static TestRule ruleChain = RuleChain.outerRule(minio).around(cluster); + public static TestRule ruleChain = RuleChain.outerRule(minioFixture).around(cluster); + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } - @ParametersFactory - public static Iterable parameters() throws Exception { - return createParameters(new String[] { "repository_s3/10_basic", "repository_s3/20_repository_permanent_credentials" }); + @Override + protected String getBucketName() { + return BUCKET; } - public RepositoryS3MinioClientYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { - super(testCandidate); + @Override + protected String getBasePath() { + return BASE_PATH; } @Override - protected String getTestRestCluster() { - return cluster.getHttpAddresses(); + protected String getClientName() { + return CLIENT; } } diff --git a/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3SessionCredentialsRestIT.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3SessionCredentialsRestIT.java new file mode 100644 index 0000000000000..84a327ee131ae --- /dev/null +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3SessionCredentialsRestIT.java @@ -0,0 +1,72 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories.s3; + +import fixture.s3.S3HttpFixture; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; + +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.fixtures.testcontainers.TestContainersThreadFilter; +import org.junit.ClassRule; +import org.junit.rules.RuleChain; +import org.junit.rules.TestRule; + +@ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) // https://github.com/elastic/elasticsearch/issues/102482 +public class RepositoryS3SessionCredentialsRestIT extends AbstractRepositoryS3RestTestCase { + + private static final String PREFIX = getIdentifierPrefix("RepositoryS3SessionCredentialsRestIT"); + private static final String BUCKET = PREFIX + "bucket"; + private static final String BASE_PATH = PREFIX + "base_path"; + private static final String ACCESS_KEY = PREFIX + "access-key"; + private static final String SECRET_KEY = PREFIX + "secret-key"; + private static final String SESSION_TOKEN = PREFIX + "session-token"; + private static final String CLIENT = "session_credentials_client"; + + private static final S3HttpFixture s3Fixture = new S3HttpFixture( + true, + BUCKET, + BASE_PATH, + S3HttpFixture.fixedAccessKeyAndToken(ACCESS_KEY, SESSION_TOKEN) + ); + + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .module("repository-s3") + .keystore("s3.client." + CLIENT + ".access_key", ACCESS_KEY) + .keystore("s3.client." + CLIENT + ".secret_key", SECRET_KEY) + .keystore("s3.client." + CLIENT + ".session_token", SESSION_TOKEN) + .setting("s3.client." + CLIENT + ".endpoint", s3Fixture::getAddress) + .build(); + + @ClassRule + public static TestRule ruleChain = RuleChain.outerRule(s3Fixture).around(cluster); + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + @Override + protected String getBucketName() { + return BUCKET; + } + + @Override + protected String getBasePath() { + return BASE_PATH; + } + + @Override + protected String getClientName() { + return CLIENT; + } +} diff --git a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3StsClientYamlTestSuiteIT.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3StsCredentialsRestIT.java similarity index 53% rename from modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3StsClientYamlTestSuiteIT.java rename to modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3StsCredentialsRestIT.java index 7c4d719485113..de80e4179ef5e 100644 --- a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3StsClientYamlTestSuiteIT.java +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3StsCredentialsRestIT.java @@ -13,43 +13,53 @@ import fixture.s3.DynamicS3Credentials; import fixture.s3.S3HttpFixture; -import com.carrotsearch.randomizedtesting.annotations.Name; -import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; import org.elasticsearch.test.cluster.ElasticsearchCluster; import org.elasticsearch.test.cluster.util.resource.Resource; -import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; +import org.elasticsearch.test.fixtures.testcontainers.TestContainersThreadFilter; import org.junit.ClassRule; import org.junit.rules.RuleChain; import org.junit.rules.TestRule; -public class RepositoryS3StsClientYamlTestSuiteIT extends AbstractRepositoryS3ClientYamlTestSuiteIT { +@ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) // https://github.com/elastic/elasticsearch/issues/102482 +public class RepositoryS3StsCredentialsRestIT extends AbstractRepositoryS3RestTestCase { + + private static final String PREFIX = getIdentifierPrefix("RepositoryS3StsCredentialsRestIT"); + private static final String BUCKET = PREFIX + "bucket"; + private static final String BASE_PATH = PREFIX + "base_path"; + private static final String CLIENT = "sts_credentials_client"; private static final DynamicS3Credentials dynamicS3Credentials = new DynamicS3Credentials(); - private static final S3HttpFixture s3HttpFixture = new S3HttpFixture( - true, - "sts_bucket", - "sts_base_path", - dynamicS3Credentials::isAuthorized - ); + private static final S3HttpFixture s3HttpFixture = new S3HttpFixture(true, BUCKET, BASE_PATH, dynamicS3Credentials::isAuthorized); - private static final AwsStsHttpFixture stsHttpFixture = new AwsStsHttpFixture(dynamicS3Credentials::addValidCredentials, """ + private static final String WEB_IDENTITY_TOKEN_FILE_CONTENTS = """ Atza|IQEBLjAsAhRFiXuWpUXuRvQ9PZL3GMFcYevydwIUFAHZwXZXXXXXXXXJnrulxKDHwy87oGKPznh0D6bEQZTSCzyoCtL_8S07pLpr0zMbn6w1lfVZKNTBdDans\ FBmtGnIsIapjI6xKR02Yc_2bQ8LZbUXSGm6Ry6_BG7PrtLZtj_dfCTj92xNGed-CrKqjG7nPBjNIL016GGvuS5gSvPRUxWES3VYfm1wl7WTI7jn-Pcb6M-buCgHhFO\ - zTQxod27L9CqnOLio7N3gZAGpsp6n1-AJBOCJckcyXe2c6uD0srOJeZlKUm2eTDVMf8IehDVI0r1QOnTV6KzzAI3OY87Vd_cVMQ"""); + zTQxod27L9CqnOLio7N3gZAGpsp6n1-AJBOCJckcyXe2c6uD0srOJeZlKUm2eTDVMf8IehDVI0r1QOnTV6KzzAI3OY87Vd_cVMQ"""; + + private static final AwsStsHttpFixture stsHttpFixture = new AwsStsHttpFixture( + dynamicS3Credentials::addValidCredentials, + WEB_IDENTITY_TOKEN_FILE_CONTENTS + ); public static ElasticsearchCluster cluster = ElasticsearchCluster.local() .module("repository-s3") - .setting("s3.client.integration_test_sts.endpoint", s3HttpFixture::getAddress) + .setting("s3.client." + CLIENT + ".endpoint", s3HttpFixture::getAddress) .systemProperty( "com.amazonaws.sdk.stsMetadataServiceEndpointOverride", () -> stsHttpFixture.getAddress() + "/assume-role-with-web-identity" ) - .configFile("repository-s3/aws-web-identity-token-file", Resource.fromClasspath("aws-web-identity-token-file")) - .environment("AWS_WEB_IDENTITY_TOKEN_FILE", System.getProperty("awsWebIdentityTokenExternalLocation")) - // // The AWS STS SDK requires the role and session names to be set. We can verify that they are sent to S3S in the - // // S3HttpFixtureWithSTS fixture + .configFile( + S3Service.CustomWebIdentityTokenCredentialsProvider.WEB_IDENTITY_TOKEN_FILE_LOCATION, + Resource.fromString(WEB_IDENTITY_TOKEN_FILE_CONTENTS) + ) + .environment("AWS_WEB_IDENTITY_TOKEN_FILE", S3Service.CustomWebIdentityTokenCredentialsProvider.WEB_IDENTITY_TOKEN_FILE_LOCATION) + // The AWS STS SDK requires the role and session names to be set. We can verify that they are sent to S3S in the + // S3HttpFixtureWithSTS fixture .environment("AWS_ROLE_ARN", "arn:aws:iam::123456789012:role/FederatedWebIdentityRole") .environment("AWS_ROLE_SESSION_NAME", "sts-fixture-test") .build(); @@ -57,17 +67,23 @@ public class RepositoryS3StsClientYamlTestSuiteIT extends AbstractRepositoryS3Cl @ClassRule public static TestRule ruleChain = RuleChain.outerRule(s3HttpFixture).around(stsHttpFixture).around(cluster); - @ParametersFactory - public static Iterable parameters() throws Exception { - return createParameters(new String[] { "repository_s3/60_repository_sts_credentials" }); + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + @Override + protected String getBucketName() { + return BUCKET; } - public RepositoryS3StsClientYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { - super(testCandidate); + @Override + protected String getBasePath() { + return BASE_PATH; } @Override - protected String getTestRestCluster() { - return cluster.getHttpAddresses(); + protected String getClientName() { + return CLIENT; } } diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobStore.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobStore.java index 5fb3254df819b..d08bd40275fec 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobStore.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobStore.java @@ -450,7 +450,7 @@ private static DeleteObjectsRequest bulkDelete(OperationPurpose purpose, S3BlobS @Override public void close() throws IOException { - this.service.close(); + service.onBlobStoreClose(); } @Override diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 1ebd6f920d518..1a66f5782fc03 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -303,6 +303,10 @@ private synchronized void releaseCachedClients() { IdleConnectionReaper.shutdown(); } + public void onBlobStoreClose() { + releaseCachedClients(); + } + @Override public void close() throws IOException { releaseCachedClients(); @@ -345,6 +349,8 @@ static class CustomWebIdentityTokenCredentialsProvider implements AWSCredentials private static final String STS_HOSTNAME = "https://sts.amazonaws.com"; + static final String WEB_IDENTITY_TOKEN_FILE_LOCATION = "repository-s3/aws-web-identity-token-file"; + private STSAssumeRoleWithWebIdentitySessionCredentialsProvider credentialsProvider; private AWSSecurityTokenService stsClient; private String stsRegion; @@ -363,7 +369,7 @@ static class CustomWebIdentityTokenCredentialsProvider implements AWSCredentials } // Make sure that a readable symlink to the token file exists in the plugin config directory // AWS_WEB_IDENTITY_TOKEN_FILE exists but we only use Web Identity Tokens if a corresponding symlink exists and is readable - Path webIdentityTokenFileSymlink = environment.configFile().resolve("repository-s3/aws-web-identity-token-file"); + Path webIdentityTokenFileSymlink = environment.configFile().resolve(WEB_IDENTITY_TOKEN_FILE_LOCATION); if (Files.exists(webIdentityTokenFileSymlink) == false) { LOGGER.warn( "Cannot use AWS Web Identity Tokens: AWS_WEB_IDENTITY_TOKEN_FILE is defined but no corresponding symlink exists " diff --git a/modules/repository-s3/src/test/resources/aws-web-identity-token-file b/modules/repository-s3/src/test/resources/aws-web-identity-token-file deleted file mode 100644 index 15cb29eac2ff6..0000000000000 --- a/modules/repository-s3/src/test/resources/aws-web-identity-token-file +++ /dev/null @@ -1 +0,0 @@ -Atza|IQEBLjAsAhRFiXuWpUXuRvQ9PZL3GMFcYevydwIUFAHZwXZXXXXXXXXJnrulxKDHwy87oGKPznh0D6bEQZTSCzyoCtL_8S07pLpr0zMbn6w1lfVZKNTBdDansFBmtGnIsIapjI6xKR02Yc_2bQ8LZbUXSGm6Ry6_BG7PrtLZtj_dfCTj92xNGed-CrKqjG7nPBjNIL016GGvuS5gSvPRUxWES3VYfm1wl7WTI7jn-Pcb6M-buCgHhFOzTQxod27L9CqnOLio7N3gZAGpsp6n1-AJBOCJckcyXe2c6uD0srOJeZlKUm2eTDVMf8IehDVI0r1QOnTV6KzzAI3OY87Vd_cVMQ diff --git a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3ClientYamlTestSuiteIT.java b/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3ClientYamlTestSuiteIT.java index a3b154b4bdfed..3d34934e54945 100644 --- a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3ClientYamlTestSuiteIT.java +++ b/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3ClientYamlTestSuiteIT.java @@ -9,8 +9,6 @@ package org.elasticsearch.repositories.s3; -import fixture.aws.imds.Ec2ImdsHttpFixture; -import fixture.s3.DynamicS3Credentials; import fixture.s3.S3HttpFixture; import com.carrotsearch.randomizedtesting.annotations.Name; @@ -18,7 +16,6 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; -import org.elasticsearch.cluster.routing.Murmur3HashFunction; import org.elasticsearch.test.cluster.ElasticsearchCluster; import org.elasticsearch.test.fixtures.testcontainers.TestContainersThreadFilter; import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; @@ -26,67 +23,33 @@ import org.junit.rules.RuleChain; import org.junit.rules.TestRule; -import java.util.Set; - @ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) @ThreadLeakScope(ThreadLeakScope.Scope.NONE) // https://github.com/elastic/elasticsearch/issues/102482 public class RepositoryS3ClientYamlTestSuiteIT extends AbstractRepositoryS3ClientYamlTestSuiteIT { - private static final String HASHED_SEED = Integer.toString(Murmur3HashFunction.hash(System.getProperty("tests.seed"))); - private static final String TEMPORARY_SESSION_TOKEN = "session_token-" + HASHED_SEED; - - private static final S3HttpFixture s3Fixture = new S3HttpFixture(); - - private static final S3HttpFixture s3HttpFixtureWithSessionToken = new S3HttpFixture( - true, - "session_token_bucket", - "session_token_base_path_integration_tests", - S3HttpFixture.fixedAccessKeyAndToken(System.getProperty("s3TemporaryAccessKey"), TEMPORARY_SESSION_TOKEN) - ); - - private static final DynamicS3Credentials dynamicS3Credentials = new DynamicS3Credentials(); - - private static final Ec2ImdsHttpFixture ec2ImdsHttpFixture = new Ec2ImdsHttpFixture( - dynamicS3Credentials::addValidCredentials, - Set.of() - ); + private static final String ACCESS_KEY = "RepositoryS3ClientYamlTestSuiteIT-access-key"; + private static final String SECRET_KEY = "RepositoryS3ClientYamlTestSuiteIT-secret-key"; - private static final S3HttpFixture s3HttpFixtureWithImdsSessionToken = new S3HttpFixture( + private static final S3HttpFixture s3Fixture = new S3HttpFixture( true, - "ec2_bucket", - "ec2_base_path", - dynamicS3Credentials::isAuthorized + "bucket", + "base_path_integration_tests", + S3HttpFixture.fixedAccessKey(ACCESS_KEY) ); public static ElasticsearchCluster cluster = ElasticsearchCluster.local() .module("repository-s3") - .keystore("s3.client.integration_test_permanent.access_key", System.getProperty("s3PermanentAccessKey")) - .keystore("s3.client.integration_test_permanent.secret_key", System.getProperty("s3PermanentSecretKey")) - .keystore("s3.client.integration_test_temporary.access_key", System.getProperty("s3TemporaryAccessKey")) - .keystore("s3.client.integration_test_temporary.secret_key", System.getProperty("s3TemporarySecretKey")) - .keystore("s3.client.integration_test_temporary.session_token", TEMPORARY_SESSION_TOKEN) + .keystore("s3.client.integration_test_permanent.access_key", ACCESS_KEY) + .keystore("s3.client.integration_test_permanent.secret_key", SECRET_KEY) .setting("s3.client.integration_test_permanent.endpoint", s3Fixture::getAddress) - .setting("s3.client.integration_test_temporary.endpoint", s3HttpFixtureWithSessionToken::getAddress) - .setting("s3.client.integration_test_ec2.endpoint", s3HttpFixtureWithImdsSessionToken::getAddress) - .systemProperty("com.amazonaws.sdk.ec2MetadataServiceEndpointOverride", ec2ImdsHttpFixture::getAddress) .build(); @ClassRule - public static TestRule ruleChain = RuleChain.outerRule(s3Fixture) - .around(s3HttpFixtureWithSessionToken) - .around(s3HttpFixtureWithImdsSessionToken) - .around(ec2ImdsHttpFixture) - .around(cluster); + public static TestRule ruleChain = RuleChain.outerRule(s3Fixture).around(cluster); @ParametersFactory public static Iterable parameters() throws Exception { - return createParameters( - new String[] { - "repository_s3/10_basic", - "repository_s3/20_repository_permanent_credentials", - "repository_s3/30_repository_temporary_credentials", - "repository_s3/40_repository_ec2_credentials" } - ); + return createParameters(new String[] { "repository_s3/10_basic", "repository_s3/20_repository_permanent_credentials" }); } public RepositoryS3ClientYamlTestSuiteIT(@Name("yaml") ClientYamlTestCandidate testCandidate) { diff --git a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3RegionalStsClientYamlTestSuiteIT.java b/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3RegionalStsClientYamlTestSuiteIT.java index 2baba66a8a4d0..ac356083983eb 100644 --- a/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3RegionalStsClientYamlTestSuiteIT.java +++ b/modules/repository-s3/src/yamlRestTest/java/org/elasticsearch/repositories/s3/RepositoryS3RegionalStsClientYamlTestSuiteIT.java @@ -21,10 +21,11 @@ public class RepositoryS3RegionalStsClientYamlTestSuiteIT extends AbstractReposi @ClassRule public static ElasticsearchCluster cluster = ElasticsearchCluster.local() .module("repository-s3") - .configFile("repository-s3/aws-web-identity-token-file", Resource.fromClasspath("aws-web-identity-token-file")) - .environment("AWS_WEB_IDENTITY_TOKEN_FILE", System.getProperty("awsWebIdentityTokenExternalLocation")) - // The AWS STS SDK requires the role and session names to be set. We can verify that they are sent to S3S in the - // S3HttpFixtureWithSTS fixture + .configFile(S3Service.CustomWebIdentityTokenCredentialsProvider.WEB_IDENTITY_TOKEN_FILE_LOCATION, Resource.fromString(""" + Atza|IQEBLjAsAhRFiXuWpUXuRvQ9PZL3GMFcYevydwIUFAHZwXZXXXXXXXXJnrulxKDHwy87oGKPznh0D6bEQZTSCzyoCtL_8S07pLpr0zMbn6w1lfVZKNTBdDans\ + FBmtGnIsIapjI6xKR02Yc_2bQ8LZbUXSGm6Ry6_BG7PrtLZtj_dfCTj92xNGed-CrKqjG7nPBjNIL016GGvuS5gSvPRUxWES3VYfm1wl7WTI7jn-Pcb6M-buCgHhFO\ + zTQxod27L9CqnOLio7N3gZAGpsp6n1-AJBOCJckcyXe2c6uD0srOJeZlKUm2eTDVMf8IehDVI0r1QOnTV6KzzAI3OY87Vd_cVMQ""")) + .environment("AWS_WEB_IDENTITY_TOKEN_FILE", S3Service.CustomWebIdentityTokenCredentialsProvider.WEB_IDENTITY_TOKEN_FILE_LOCATION) .environment("AWS_ROLE_ARN", "arn:aws:iam::123456789012:role/FederatedWebIdentityRole") .environment("AWS_ROLE_SESSION_NAME", "sts-fixture-test") .environment("AWS_STS_REGIONAL_ENDPOINTS", "regional") @@ -33,6 +34,9 @@ public class RepositoryS3RegionalStsClientYamlTestSuiteIT extends AbstractReposi @ParametersFactory public static Iterable parameters() throws Exception { + // Run just the basic sanity test to make sure ES starts up and loads the S3 repository with a regional endpoint without an error. + // It would be great to make actual requests against a test fixture, but setting the region means using a production endpoint. + // See #102230 for more details. return createParameters(new String[] { "repository_s3/10_basic" }); } diff --git a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml b/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml index e88a0861ec01c..6f6fdaed8c666 100644 --- a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml +++ b/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml @@ -10,12 +10,11 @@ setup: body: type: s3 settings: - bucket: @permanent_bucket@ + bucket: bucket client: integration_test_permanent - base_path: "@permanent_base_path@" + base_path: base_path_integration_tests canned_acl: private storage_class: standard - disable_chunked_encoding: @disable_chunked_encoding@ # Remove the snapshots, if a previous test failed to delete them. This is # useful for third party tests that runs the test against a real external service. @@ -40,9 +39,9 @@ setup: body: type: s3 settings: - bucket: @permanent_bucket@ + bucket: bucket client: integration_test_permanent - base_path: "@permanent_base_path@" + base_path: base_path_integration_tests endpoint: 127.0.0.1:5 canned_acl: private storage_class: standard @@ -55,9 +54,9 @@ setup: body: type: s3 settings: - bucket: @permanent_bucket@ + bucket: bucket client: integration_test_permanent - base_path: "@permanent_base_path@" + base_path: base_path_integration_tests endpoint: 127.0.0.1:5 canned_acl: private storage_class: standard @@ -106,258 +105,6 @@ setup: - match: { snapshot.include_global_state: true } - match: { snapshot.shards.failed: 0 } ---- -"Snapshot and Restore with repository-s3 using permanent credentials": - - # Get repository - - do: - snapshot.get_repository: - repository: repository_permanent - - - match: { repository_permanent.settings.bucket : @permanent_bucket@ } - - match: { repository_permanent.settings.client : "integration_test_permanent" } - - match: { repository_permanent.settings.base_path : "@permanent_base_path@" } - - match: { repository_permanent.settings.canned_acl : "private" } - - match: { repository_permanent.settings.storage_class : "standard" } - - is_false: repository_permanent.settings.access_key - - is_false: repository_permanent.settings.secret_key - - is_false: repository_permanent.settings.session_token - - # Index documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: "1" - - snapshot: one - - index: - _index: docs - _id: "2" - - snapshot: one - - index: - _index: docs - _id: "3" - - snapshot: one - - - do: - count: - index: docs - - - match: {count: 3} - - # Create a first snapshot - - do: - snapshot.create: - repository: repository_permanent - snapshot: snapshot-one - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-one } - - match: { snapshot.state : SUCCESS } - - match: { snapshot.include_global_state: true } - - match: { snapshot.shards.failed : 0 } - - - do: - snapshot.status: - repository: repository_permanent - snapshot: snapshot-one - - - is_true: snapshots - - match: { snapshots.0.snapshot: snapshot-one } - - match: { snapshots.0.state : SUCCESS } - - # Index more documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: "4" - - snapshot: two - - index: - _index: docs - _id: "5" - - snapshot: two - - index: - _index: docs - _id: "6" - - snapshot: two - - index: - _index: docs - _id: "7" - - snapshot: two - - - do: - count: - index: docs - - - match: {count: 7} - - # Create a second snapshot - - do: - snapshot.create: - repository: repository_permanent - snapshot: snapshot-two - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-two } - - match: { snapshot.state : SUCCESS } - - match: { snapshot.shards.failed : 0 } - - - do: - snapshot.get: - repository: repository_permanent - snapshot: snapshot-one,snapshot-two - - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } - - # Delete the index - - do: - indices.delete: - index: docs - - # Restore the second snapshot - - do: - snapshot.restore: - repository: repository_permanent - snapshot: snapshot-two - wait_for_completion: true - - - do: - count: - index: docs - - - match: {count: 7} - - # Delete the index again - - do: - indices.delete: - index: docs - - # Restore the first snapshot - - do: - snapshot.restore: - repository: repository_permanent - snapshot: snapshot-one - wait_for_completion: true - - - do: - count: - index: docs - - - match: {count: 3} - - # Remove the snapshots - - do: - snapshot.delete: - repository: repository_permanent - snapshot: snapshot-two - - - do: - snapshot.delete: - repository: repository_permanent - snapshot: snapshot-one - ---- -"Register a repository with a non existing bucket": - - - do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_permanent - body: - type: s3 - settings: - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_permanent - ---- -"Register a repository with a non existing client": - - - do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_permanent - body: - type: s3 - settings: - bucket: repository_permanent - client: unknown - ---- -"Register a read-only repository with a non existing bucket": - -- do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_permanent - body: - type: s3 - settings: - readonly: true - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_permanent - ---- -"Register a read-only repository with a non existing client": - -- do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_permanent - body: - type: s3 - settings: - readonly: true - bucket: repository_permanent - client: unknown - ---- -"Get a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.get: - repository: repository_permanent - snapshot: missing - ---- -"Delete a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.delete: - repository: repository_permanent - snapshot: missing - ---- -"Restore a non existing snapshot": - - - do: - catch: /snapshot_restore_exception/ - snapshot.restore: - repository: repository_permanent - snapshot: missing - wait_for_completion: true - ---- -"Usage stats": - - requires: - cluster_features: - - repositories.supports_usage_stats - reason: requires this feature - - - do: - cluster.stats: {} - - - gte: { repositories.s3.count: 1 } - - gte: { repositories.s3.read_write: 1 } - --- teardown: diff --git a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml b/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml deleted file mode 100644 index 501af980e17e3..0000000000000 --- a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml +++ /dev/null @@ -1,278 +0,0 @@ -# Integration tests for repository-s3 - ---- -setup: - - # Register repository with temporary credentials - - do: - snapshot.create_repository: - repository: repository_temporary - body: - type: s3 - settings: - bucket: @temporary_bucket@ - client: integration_test_temporary - base_path: "@temporary_base_path@" - canned_acl: private - storage_class: standard - disable_chunked_encoding: @disable_chunked_encoding@ - ---- -"Snapshot and Restore with repository-s3 using temporary credentials": - - # Get repository - - do: - snapshot.get_repository: - repository: repository_temporary - - - match: { repository_temporary.settings.bucket : @temporary_bucket@ } - - match: { repository_temporary.settings.client : "integration_test_temporary" } - - match: { repository_temporary.settings.base_path : "@temporary_base_path@" } - - match: { repository_temporary.settings.canned_acl : "private" } - - match: { repository_temporary.settings.storage_class : "standard" } - - is_false: repository_temporary.settings.access_key - - is_false: repository_temporary.settings.secret_key - - is_false: repository_temporary.settings.session_token - - # Index documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: "1" - - snapshot: one - - index: - _index: docs - _id: "2" - - snapshot: one - - index: - _index: docs - _id: "3" - - snapshot: one - - - do: - count: - index: docs - - - match: {count: 3} - - # Create a first snapshot - - do: - snapshot.create: - repository: repository_temporary - snapshot: snapshot-one - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-one } - - match: { snapshot.state : SUCCESS } - - match: { snapshot.include_global_state: true } - - match: { snapshot.shards.failed : 0 } - - - do: - snapshot.status: - repository: repository_temporary - snapshot: snapshot-one - - - is_true: snapshots - - match: { snapshots.0.snapshot: snapshot-one } - - match: { snapshots.0.state : SUCCESS } - - # Index more documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: "4" - - snapshot: two - - index: - _index: docs - _id: "5" - - snapshot: two - - index: - _index: docs - _id: "6" - - snapshot: two - - index: - _index: docs - _id: "7" - - snapshot: two - - - do: - count: - index: docs - - - match: {count: 7} - - # Create a second snapshot - - do: - snapshot.create: - repository: repository_temporary - snapshot: snapshot-two - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-two } - - match: { snapshot.state : SUCCESS } - - match: { snapshot.shards.failed : 0 } - - - do: - snapshot.get: - repository: repository_temporary - snapshot: snapshot-one,snapshot-two - - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } - - # Delete the index - - do: - indices.delete: - index: docs - - # Restore the second snapshot - - do: - snapshot.restore: - repository: repository_temporary - snapshot: snapshot-two - wait_for_completion: true - - - do: - count: - index: docs - - - match: {count: 7} - - # Delete the index again - - do: - indices.delete: - index: docs - - # Restore the first snapshot - - do: - snapshot.restore: - repository: repository_temporary - snapshot: snapshot-one - wait_for_completion: true - - - do: - count: - index: docs - - - match: {count: 3} - - # Remove the snapshots - - do: - snapshot.delete: - repository: repository_temporary - snapshot: snapshot-two - - - do: - snapshot.delete: - repository: repository_temporary - snapshot: snapshot-one - ---- -"Register a repository with a non existing bucket": - - - do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_temporary - body: - type: s3 - settings: - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_temporary - ---- -"Register a repository with a non existing client": - - - do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_temporary - body: - type: s3 - settings: - bucket: repository_temporary - client: unknown - ---- -"Register a read-only repository with a non existing bucket": - -- do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_temporary - body: - type: s3 - settings: - readonly: true - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_temporary - ---- -"Register a read-only repository with a non existing client": - -- do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_temporary - body: - type: s3 - settings: - readonly: true - bucket: repository_temporary - client: unknown - ---- -"Get a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.get: - repository: repository_temporary - snapshot: missing - ---- -"Delete a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.delete: - repository: repository_temporary - snapshot: missing - ---- -"Restore a non existing snapshot": - - - do: - catch: /snapshot_restore_exception/ - snapshot.restore: - repository: repository_temporary - snapshot: missing - wait_for_completion: true - ---- -"Usage stats": - - requires: - cluster_features: - - repositories.supports_usage_stats - reason: requires this feature - - - do: - cluster.stats: {} - - - gte: { repositories.s3.count: 1 } - - gte: { repositories.s3.read_write: 1 } - ---- -teardown: - - # Remove our repository - - do: - snapshot.delete_repository: - repository: repository_temporary diff --git a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml b/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml deleted file mode 100644 index 129f0ba5d7588..0000000000000 --- a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml +++ /dev/null @@ -1,278 +0,0 @@ -# Integration tests for repository-s3 - ---- -setup: - - # Register repository with ec2 credentials - - do: - snapshot.create_repository: - repository: repository_ec2 - body: - type: s3 - settings: - bucket: @ec2_bucket@ - client: integration_test_ec2 - base_path: "@ec2_base_path@" - canned_acl: private - storage_class: standard - disable_chunked_encoding: @disable_chunked_encoding@ - ---- -"Snapshot and Restore with repository-s3 using ec2 credentials": - - # Get repository - - do: - snapshot.get_repository: - repository: repository_ec2 - - - match: { repository_ec2.settings.bucket : @ec2_bucket@ } - - match: { repository_ec2.settings.client : "integration_test_ec2" } - - match: { repository_ec2.settings.base_path : "@ec2_base_path@" } - - match: { repository_ec2.settings.canned_acl : "private" } - - match: { repository_ec2.settings.storage_class : "standard" } - - is_false: repository_ec2.settings.access_key - - is_false: repository_ec2.settings.secret_key - - is_false: repository_ec2.settings.session_token - - # Index documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: "1" - - snapshot: one - - index: - _index: docs - _id: "2" - - snapshot: one - - index: - _index: docs - _id: "3" - - snapshot: one - - - do: - count: - index: docs - - - match: {count: 3} - - # Create a first snapshot - - do: - snapshot.create: - repository: repository_ec2 - snapshot: snapshot-one - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-one } - - match: { snapshot.state : SUCCESS } - - match: { snapshot.include_global_state: true } - - match: { snapshot.shards.failed : 0 } - - - do: - snapshot.status: - repository: repository_ec2 - snapshot: snapshot-one - - - is_true: snapshots - - match: { snapshots.0.snapshot: snapshot-one } - - match: { snapshots.0.state : SUCCESS } - - # Index more documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: "4" - - snapshot: two - - index: - _index: docs - _id: "5" - - snapshot: two - - index: - _index: docs - _id: "6" - - snapshot: two - - index: - _index: docs - _id: "7" - - snapshot: two - - - do: - count: - index: docs - - - match: {count: 7} - - # Create a second snapshot - - do: - snapshot.create: - repository: repository_ec2 - snapshot: snapshot-two - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-two } - - match: { snapshot.state : SUCCESS } - - match: { snapshot.shards.failed : 0 } - - - do: - snapshot.get: - repository: repository_ec2 - snapshot: snapshot-one,snapshot-two - - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } - - # Delete the index - - do: - indices.delete: - index: docs - - # Restore the second snapshot - - do: - snapshot.restore: - repository: repository_ec2 - snapshot: snapshot-two - wait_for_completion: true - - - do: - count: - index: docs - - - match: {count: 7} - - # Delete the index again - - do: - indices.delete: - index: docs - - # Restore the first snapshot - - do: - snapshot.restore: - repository: repository_ec2 - snapshot: snapshot-one - wait_for_completion: true - - - do: - count: - index: docs - - - match: {count: 3} - - # Remove the snapshots - - do: - snapshot.delete: - repository: repository_ec2 - snapshot: snapshot-two - - - do: - snapshot.delete: - repository: repository_ec2 - snapshot: snapshot-one - ---- -"Register a repository with a non existing bucket": - - - do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_ec2 - body: - type: s3 - settings: - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_temporary - ---- -"Register a repository with a non existing client": - - - do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_ec2 - body: - type: s3 - settings: - bucket: repository_ec2 - client: unknown - ---- -"Register a read-only repository with a non existing bucket": - -- do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_ec2 - body: - type: s3 - settings: - readonly: true - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_temporary - ---- -"Register a read-only repository with a non existing client": - -- do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_ec2 - body: - type: s3 - settings: - readonly: true - bucket: repository_ec2 - client: unknown - ---- -"Get a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.get: - repository: repository_ec2 - snapshot: missing - ---- -"Delete a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.delete: - repository: repository_ec2 - snapshot: missing - ---- -"Restore a non existing snapshot": - - - do: - catch: /snapshot_restore_exception/ - snapshot.restore: - repository: repository_ec2 - snapshot: missing - wait_for_completion: true - ---- -"Usage stats": - - requires: - cluster_features: - - repositories.supports_usage_stats - reason: requires this feature - - - do: - cluster.stats: {} - - - gte: { repositories.s3.count: 1 } - - gte: { repositories.s3.read_write: 1 } - ---- -teardown: - - # Remove our repository - - do: - snapshot.delete_repository: - repository: repository_ec2 diff --git a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml b/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml deleted file mode 100644 index de334b4b3df96..0000000000000 --- a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml +++ /dev/null @@ -1,278 +0,0 @@ -# Integration tests for repository-s3 - ---- -setup: - - # Register repository with ecs credentials - - do: - snapshot.create_repository: - repository: repository_ecs - body: - type: s3 - settings: - bucket: @ecs_bucket@ - client: integration_test_ecs - base_path: "@ecs_base_path@" - canned_acl: private - storage_class: standard - disable_chunked_encoding: @disable_chunked_encoding@ - ---- -"Snapshot and Restore with repository-s3 using ecs credentials": - - # Get repository - - do: - snapshot.get_repository: - repository: repository_ecs - - - match: { repository_ecs.settings.bucket : @ecs_bucket@ } - - match: { repository_ecs.settings.client : "integration_test_ecs" } - - match: { repository_ecs.settings.base_path : "@ecs_base_path@" } - - match: { repository_ecs.settings.canned_acl : "private" } - - match: { repository_ecs.settings.storage_class : "standard" } - - is_false: repository_ecs.settings.access_key - - is_false: repository_ecs.settings.secret_key - - is_false: repository_ecs.settings.session_token - - # Index documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: "1" - - snapshot: one - - index: - _index: docs - _id: "2" - - snapshot: one - - index: - _index: docs - _id: "3" - - snapshot: one - - - do: - count: - index: docs - - - match: {count: 3} - - # Create a first snapshot - - do: - snapshot.create: - repository: repository_ecs - snapshot: snapshot-one - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-one } - - match: { snapshot.state : SUCCESS } - - match: { snapshot.include_global_state: true } - - match: { snapshot.shards.failed : 0 } - - - do: - snapshot.status: - repository: repository_ecs - snapshot: snapshot-one - - - is_true: snapshots - - match: { snapshots.0.snapshot: snapshot-one } - - match: { snapshots.0.state : SUCCESS } - - # Index more documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: "4" - - snapshot: two - - index: - _index: docs - _id: "5" - - snapshot: two - - index: - _index: docs - _id: "6" - - snapshot: two - - index: - _index: docs - _id: "7" - - snapshot: two - - - do: - count: - index: docs - - - match: {count: 7} - - # Create a second snapshot - - do: - snapshot.create: - repository: repository_ecs - snapshot: snapshot-two - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-two } - - match: { snapshot.state : SUCCESS } - - match: { snapshot.shards.failed : 0 } - - - do: - snapshot.get: - repository: repository_ecs - snapshot: snapshot-one,snapshot-two - - - is_true: snapshots - - match: { snapshots.0.state : SUCCESS } - - match: { snapshots.1.state : SUCCESS } - - # Delete the index - - do: - indices.delete: - index: docs - - # Restore the second snapshot - - do: - snapshot.restore: - repository: repository_ecs - snapshot: snapshot-two - wait_for_completion: true - - - do: - count: - index: docs - - - match: {count: 7} - - # Delete the index again - - do: - indices.delete: - index: docs - - # Restore the first snapshot - - do: - snapshot.restore: - repository: repository_ecs - snapshot: snapshot-one - wait_for_completion: true - - - do: - count: - index: docs - - - match: {count: 3} - - # Remove the snapshots - - do: - snapshot.delete: - repository: repository_ecs - snapshot: snapshot-two - - - do: - snapshot.delete: - repository: repository_ecs - snapshot: snapshot-one - ---- -"Register a repository with a non existing bucket": - - - do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_ecs - body: - type: s3 - settings: - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_ecs - ---- -"Register a repository with a non existing client": - - - do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_ecs - body: - type: s3 - settings: - bucket: repository_ecs - client: unknown - ---- -"Register a read-only repository with a non existing bucket": - -- do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_ecs - body: - type: s3 - settings: - readonly: true - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_ecs - ---- -"Register a read-only repository with a non existing client": - -- do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_ecs - body: - type: s3 - settings: - readonly: true - bucket: repository_ecs - client: unknown - ---- -"Get a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.get: - repository: repository_ecs - snapshot: missing - ---- -"Delete a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.delete: - repository: repository_ecs - snapshot: missing - ---- -"Restore a non existing snapshot": - - - do: - catch: /snapshot_restore_exception/ - snapshot.restore: - repository: repository_ecs - snapshot: missing - wait_for_completion: true - ---- -"Usage stats": - - requires: - cluster_features: - - repositories.supports_usage_stats - reason: requires this feature - - - do: - cluster.stats: {} - - - gte: { repositories.s3.count: 1 } - - gte: { repositories.s3.read_write: 1 } - ---- -teardown: - - # Remove our repository - - do: - snapshot.delete_repository: - repository: repository_ecs diff --git a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/60_repository_sts_credentials.yml b/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/60_repository_sts_credentials.yml deleted file mode 100644 index 09a8526017960..0000000000000 --- a/modules/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/60_repository_sts_credentials.yml +++ /dev/null @@ -1,279 +0,0 @@ -# Integration tests for repository-s3 - ---- -setup: - - # Register repository with sts credentials - - do: - snapshot.create_repository: - repository: repository_sts - body: - type: s3 - settings: - bucket: @sts_bucket@ - client: integration_test_sts - base_path: "@sts_base_path@" - canned_acl: private - storage_class: standard - disable_chunked_encoding: @disable_chunked_encoding@ - ---- -"Snapshot and Restore repository-s3 using sts credentials": - - # Get repository - - do: - snapshot.get_repository: - repository: repository_sts - - - match: { repository_sts.settings.bucket: @sts_bucket@ } - - match: { repository_sts.settings.client: "integration_test_sts" } - - match: { repository_sts.settings.base_path: "@sts_base_path@" } - - match: { repository_sts.settings.canned_acl: "private" } - - match: { repository_sts.settings.storage_class: "standard" } - - is_false: repository_sts.settings.access_key - - is_false: repository_sts.settings.secret_key - - is_false: repository_sts.settings.session_token - - # Index documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: 1 - - snapshot: one - - index: - _index: docs - _id: 2 - - snapshot: one - - index: - _index: docs - _id: 3 - - snapshot: one - - - do: - count: - index: docs - - - match: { count: 3 } - - # Create a first snapshot - - do: - snapshot.create: - repository: repository_sts - snapshot: snapshot-one - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-one } - - match: { snapshot.state: SUCCESS } - - match: { snapshot.include_global_state: true } - - match: { snapshot.shards.failed: 0 } - - - do: - snapshot.status: - repository: repository_sts - snapshot: snapshot-one - - - is_true: snapshots - - match: { snapshots.0.snapshot: snapshot-one } - - match: { snapshots.0.state: SUCCESS } - - # Index more documents - - do: - bulk: - refresh: true - body: - - index: - _index: docs - _id: 4 - - snapshot: two - - index: - _index: docs - _id: 5 - - snapshot: two - - index: - _index: docs - _id: 6 - - snapshot: two - - index: - _index: docs - _id: 7 - - snapshot: two - - - do: - count: - index: docs - - - match: { count: 7 } - - # Create a second snapshot - - do: - snapshot.create: - repository: repository_sts - snapshot: snapshot-two - wait_for_completion: true - - - match: { snapshot.snapshot: snapshot-two } - - match: { snapshot.state: SUCCESS } - - match: { snapshot.shards.failed: 0 } - - - do: - snapshot.get: - repository: repository_sts - snapshot: snapshot-one,snapshot-two - - - is_true: snapshots - - match: { snapshots.0.state: SUCCESS } - - match: { snapshots.1.state: SUCCESS } - - # Delete the index - - do: - indices.delete: - index: docs - - # Restore the second snapshot - - do: - snapshot.restore: - repository: repository_sts - snapshot: snapshot-two - wait_for_completion: true - - - do: - count: - index: docs - - - match: { count: 7 } - - # Delete the index again - - do: - indices.delete: - index: docs - - # Restore the first snapshot - - do: - snapshot.restore: - repository: repository_sts - snapshot: snapshot-one - wait_for_completion: true - - - do: - count: - index: docs - - - match: { count: 3 } - - # Remove the snapshots - - do: - snapshot.delete: - repository: repository_sts - snapshot: snapshot-two - - - do: - snapshot.delete: - repository: repository_sts - snapshot: snapshot-one - ---- - -"Register a repository with a non existing bucket": - - - do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_sts - body: - type: s3 - settings: - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_sts - ---- -"Register a repository with a non existing client": - - - do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_sts - body: - type: s3 - settings: - bucket: repository_sts - client: unknown - ---- -"Register a read-only repository with a non existing bucket": - - - do: - catch: /repository_verification_exception/ - snapshot.create_repository: - repository: repository_sts - body: - type: s3 - settings: - readonly: true - bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE - client: integration_test_sts - ---- -"Register a read-only repository with a non existing client": - - - do: - catch: /illegal_argument_exception/ - snapshot.create_repository: - repository: repository_sts - body: - type: s3 - settings: - readonly: true - bucket: repository_sts - client: unknown - ---- -"Get a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.get: - repository: repository_sts - snapshot: missing - ---- -"Delete a non existing snapshot": - - - do: - catch: /snapshot_missing_exception/ - snapshot.delete: - repository: repository_sts - snapshot: missing - ---- -"Restore a non existing snapshot": - - - do: - catch: /snapshot_restore_exception/ - snapshot.restore: - repository: repository_sts - snapshot: missing - wait_for_completion: true - ---- -"Usage stats": - - requires: - cluster_features: - - repositories.supports_usage_stats - reason: requires this feature - - - do: - cluster.stats: {} - - - gte: { repositories.s3.count: 1 } - - gte: { repositories.s3.read_write: 1 } - ---- -teardown: - - # Remove our repository - - do: - snapshot.delete_repository: - repository: repository_sts diff --git a/test/fixtures/minio-fixture/src/main/java/org/elasticsearch/test/fixtures/minio/MinioTestContainer.java b/test/fixtures/minio-fixture/src/main/java/org/elasticsearch/test/fixtures/minio/MinioTestContainer.java index 285bbb91983cc..3ee18d71a5a79 100644 --- a/test/fixtures/minio-fixture/src/main/java/org/elasticsearch/test/fixtures/minio/MinioTestContainer.java +++ b/test/fixtures/minio-fixture/src/main/java/org/elasticsearch/test/fixtures/minio/MinioTestContainer.java @@ -18,17 +18,13 @@ public final class MinioTestContainer extends DockerEnvironmentAwareTestContaine public static final String DOCKER_BASE_IMAGE = "minio/minio:RELEASE.2021-03-01T04-20-55Z"; private final boolean enabled; - public MinioTestContainer() { - this(true); - } - - public MinioTestContainer(boolean enabled) { + public MinioTestContainer(boolean enabled, String accessKey, String secretKey, String bucketName) { super( new ImageFromDockerfile("es-minio-testfixture").withDockerfileFromBuilder( builder -> builder.from(DOCKER_BASE_IMAGE) - .env("MINIO_ACCESS_KEY", "s3_test_access_key") - .env("MINIO_SECRET_KEY", "s3_test_secret_key") - .run("mkdir -p /minio/data/bucket") + .env("MINIO_ACCESS_KEY", accessKey) + .env("MINIO_SECRET_KEY", secretKey) + .run("mkdir -p /minio/data/" + bucketName) .cmd("server", "/minio/data") .build() ) diff --git a/test/fixtures/s3-fixture/src/main/java/fixture/s3/S3HttpFixture.java b/test/fixtures/s3-fixture/src/main/java/fixture/s3/S3HttpFixture.java index 36f8fedcb3335..ab70f043043cc 100644 --- a/test/fixtures/s3-fixture/src/main/java/fixture/s3/S3HttpFixture.java +++ b/test/fixtures/s3-fixture/src/main/java/fixture/s3/S3HttpFixture.java @@ -33,10 +33,6 @@ public class S3HttpFixture extends ExternalResource { private final String basePath; private final BiPredicate authorizationPredicate; - public S3HttpFixture() { - this(true); - } - public S3HttpFixture(boolean enabled) { this(enabled, "bucket", "base_path_integration_tests", fixedAccessKey("s3_test_access_key")); } diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/AbstractLocalClusterFactory.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/AbstractLocalClusterFactory.java index 717cf96ad6a92..2dac2ee232aa5 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/AbstractLocalClusterFactory.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/AbstractLocalClusterFactory.java @@ -49,6 +49,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Properties; import java.util.Set; import java.util.UUID; @@ -473,6 +474,7 @@ private void createKeystore() { private void addKeystoreSettings() { spec.resolveKeystore().forEach((key, value) -> { + Objects.requireNonNull(value, "keystore setting for '" + key + "' may not be null"); String input = spec.getKeystorePassword() == null || spec.getKeystorePassword().isEmpty() ? value : spec.getKeystorePassword() + "\n" + value; diff --git a/x-pack/plugin/searchable-snapshots/qa/minio/src/javaRestTest/java/org/elasticsearch/xpack/searchablesnapshots/minio/MinioSearchableSnapshotsIT.java b/x-pack/plugin/searchable-snapshots/qa/minio/src/javaRestTest/java/org/elasticsearch/xpack/searchablesnapshots/minio/MinioSearchableSnapshotsIT.java index 5c2b19fe75a07..53f1a9a88e10e 100644 --- a/x-pack/plugin/searchable-snapshots/qa/minio/src/javaRestTest/java/org/elasticsearch/xpack/searchablesnapshots/minio/MinioSearchableSnapshotsIT.java +++ b/x-pack/plugin/searchable-snapshots/qa/minio/src/javaRestTest/java/org/elasticsearch/xpack/searchablesnapshots/minio/MinioSearchableSnapshotsIT.java @@ -21,7 +21,12 @@ @ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) public class MinioSearchableSnapshotsIT extends AbstractSearchableSnapshotsRestTestCase { - public static final MinioTestContainer minioFixture = new MinioTestContainer(); + public static final MinioTestContainer minioFixture = new MinioTestContainer( + true, + "s3_test_access_key", + "s3_test_secret_key", + "bucket" + ); public static ElasticsearchCluster cluster = ElasticsearchCluster.local() .distribution(DistributionType.DEFAULT) diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/MinioRepositoryAnalysisRestIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/MinioRepositoryAnalysisRestIT.java index b0068bd7bfdaf..3b5edaf768057 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/MinioRepositoryAnalysisRestIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/minio/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/MinioRepositoryAnalysisRestIT.java @@ -20,7 +20,12 @@ @ThreadLeakFilters(filters = { TestContainersThreadFilter.class }) public class MinioRepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRestTestCase { - public static final MinioTestContainer minioFixture = new MinioTestContainer(); + public static final MinioTestContainer minioFixture = new MinioTestContainer( + true, + "s3_test_access_key", + "s3_test_secret_key", + "bucket" + ); public static ElasticsearchCluster cluster = ElasticsearchCluster.local() .distribution(DistributionType.DEFAULT) From d729558529cafc80d705296328140b45830aa974 Mon Sep 17 00:00:00 2001 From: Jan Kuipers <148754765+jan-elastic@users.noreply.github.com> Date: Fri, 29 Nov 2024 11:00:54 +0100 Subject: [PATCH 16/27] Correct categorization analyzer in ES|QL categorize (#117695) * Correct categorization analyzer in ES|QL categorize * close categorizer if constructing analyzer fails * Rename capability CATEGORIZE_V4 * add comments --- x-pack/plugin/esql/compute/build.gradle | 4 +- .../compute/src/main/java/module-info.java | 1 + .../aggregation/blockhash/BlockHash.java | 10 +- .../blockhash/CategorizeRawBlockHash.java | 34 ++--- .../operator/HashAggregationOperator.java | 6 +- .../GroupingAggregatorFunctionTestCase.java | 4 +- .../blockhash/CategorizeBlockHashTests.java | 76 +++++++---- .../HashAggregationOperatorTests.java | 3 +- .../src/main/resources/categorize.csv-spec | 123 ++++++++++-------- .../xpack/esql/action/EsqlCapabilities.java | 2 +- .../AbstractPhysicalOperationProviders.java | 9 +- .../planner/EsPhysicalOperationProviders.java | 4 +- .../xpack/esql/plugin/ComputeService.java | 2 +- .../xpack/esql/analysis/VerifierTests.java | 6 +- .../optimizer/LogicalPlanOptimizerTests.java | 4 +- .../planner/LocalExecutionPlannerTests.java | 4 +- .../TestPhysicalOperationProviders.java | 20 ++- 17 files changed, 199 insertions(+), 113 deletions(-) diff --git a/x-pack/plugin/esql/compute/build.gradle b/x-pack/plugin/esql/compute/build.gradle index 609c778df5929..8e866cec3f421 100644 --- a/x-pack/plugin/esql/compute/build.gradle +++ b/x-pack/plugin/esql/compute/build.gradle @@ -11,11 +11,13 @@ base { dependencies { compileOnly project(':server') compileOnly project('ann') + compileOnly project(xpackModule('core')) compileOnly project(xpackModule('ml')) annotationProcessor project('gen') implementation 'com.carrotsearch:hppc:0.8.1' - testImplementation project(':test:framework') + testImplementation(project(':modules:analysis-common')) + testImplementation(project(':test:framework')) testImplementation(project(xpackModule('esql-core'))) testImplementation(project(xpackModule('core'))) testImplementation(project(xpackModule('ml'))) diff --git a/x-pack/plugin/esql/compute/src/main/java/module-info.java b/x-pack/plugin/esql/compute/src/main/java/module-info.java index 573d9e048a4d4..1b3253694b298 100644 --- a/x-pack/plugin/esql/compute/src/main/java/module-info.java +++ b/x-pack/plugin/esql/compute/src/main/java/module-info.java @@ -19,6 +19,7 @@ requires org.elasticsearch.ml; requires org.elasticsearch.tdigest; requires org.elasticsearch.geo; + requires org.elasticsearch.xcore; requires hppc; exports org.elasticsearch.compute; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java index ef0f3ceb112c4..ea76c3bd0a0aa 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java @@ -25,6 +25,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.ReleasableIterator; +import org.elasticsearch.index.analysis.AnalysisRegistry; import java.util.Iterator; import java.util.List; @@ -169,14 +170,19 @@ public static BlockHash buildPackedValuesBlockHash(List groups, Block /** * Builds a BlockHash for the Categorize grouping function. */ - public static BlockHash buildCategorizeBlockHash(List groups, AggregatorMode aggregatorMode, BlockFactory blockFactory) { + public static BlockHash buildCategorizeBlockHash( + List groups, + AggregatorMode aggregatorMode, + BlockFactory blockFactory, + AnalysisRegistry analysisRegistry + ) { if (groups.size() != 1) { throw new IllegalArgumentException("only a single CATEGORIZE group can used"); } return aggregatorMode.isInputPartial() ? new CategorizedIntermediateBlockHash(groups.get(0).channel, blockFactory, aggregatorMode.isOutputPartial()) - : new CategorizeRawBlockHash(groups.get(0).channel, blockFactory, aggregatorMode.isOutputPartial()); + : new CategorizeRawBlockHash(groups.get(0).channel, blockFactory, aggregatorMode.isOutputPartial(), analysisRegistry); } /** diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java index 0d0a2fef2f82b..47dd7f650dffa 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java @@ -7,7 +7,6 @@ package org.elasticsearch.compute.aggregation.blockhash; -import org.apache.lucene.analysis.core.WhitespaceTokenizer; import org.apache.lucene.util.BytesRef; import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; import org.elasticsearch.compute.data.Block; @@ -19,13 +18,14 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; -import org.elasticsearch.index.analysis.CharFilterFactory; -import org.elasticsearch.index.analysis.CustomAnalyzer; -import org.elasticsearch.index.analysis.TokenFilterFactory; -import org.elasticsearch.index.analysis.TokenizerFactory; +import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.xpack.core.ml.job.config.CategorizationAnalyzerConfig; import org.elasticsearch.xpack.ml.aggs.categorization.TokenListCategorizer; import org.elasticsearch.xpack.ml.job.categorization.CategorizationAnalyzer; +import java.io.IOException; +import java.util.List; + /** * BlockHash implementation for {@code Categorize} grouping function. *

@@ -33,19 +33,23 @@ *

*/ public class CategorizeRawBlockHash extends AbstractCategorizeBlockHash { + private static final CategorizationAnalyzerConfig ANALYZER_CONFIG = CategorizationAnalyzerConfig.buildStandardCategorizationAnalyzer( + List.of() + ); + private final CategorizeEvaluator evaluator; - CategorizeRawBlockHash(int channel, BlockFactory blockFactory, boolean outputPartial) { + CategorizeRawBlockHash(int channel, BlockFactory blockFactory, boolean outputPartial, AnalysisRegistry analysisRegistry) { super(blockFactory, channel, outputPartial); - CategorizationAnalyzer analyzer = new CategorizationAnalyzer( - // TODO: should be the same analyzer as used in Production - new CustomAnalyzer( - TokenizerFactory.newFactory("whitespace", WhitespaceTokenizer::new), - new CharFilterFactory[0], - new TokenFilterFactory[0] - ), - true - ); + + CategorizationAnalyzer analyzer; + try { + analyzer = new CategorizationAnalyzer(analysisRegistry, ANALYZER_CONFIG); + } catch (IOException e) { + categorizer.close(); + throw new RuntimeException(e); + } + this.evaluator = new CategorizeEvaluator(analyzer, categorizer, blockFactory); } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/HashAggregationOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/HashAggregationOperator.java index a69e8ca767014..6f8386ec08de1 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/HashAggregationOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/HashAggregationOperator.java @@ -24,6 +24,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; @@ -42,14 +43,15 @@ public record HashAggregationOperatorFactory( List groups, AggregatorMode aggregatorMode, List aggregators, - int maxPageSize + int maxPageSize, + AnalysisRegistry analysisRegistry ) implements OperatorFactory { @Override public Operator get(DriverContext driverContext) { if (groups.stream().anyMatch(BlockHash.GroupSpec::isCategorize)) { return new HashAggregationOperator( aggregators, - () -> BlockHash.buildCategorizeBlockHash(groups, aggregatorMode, driverContext.blockFactory()), + () -> BlockHash.buildCategorizeBlockHash(groups, aggregatorMode, driverContext.blockFactory(), analysisRegistry), driverContext ); } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunctionTestCase.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunctionTestCase.java index 1e97bdf5a2e79..58925a5ca36fc 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunctionTestCase.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/GroupingAggregatorFunctionTestCase.java @@ -54,7 +54,6 @@ import static org.elasticsearch.compute.data.BlockTestUtils.append; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.in; /** * Shared tests for testing grouped aggregations. @@ -107,7 +106,8 @@ private Operator.OperatorFactory simpleWithMode( List.of(new BlockHash.GroupSpec(0, ElementType.LONG)), mode, List.of(supplier.groupingAggregatorFactory(mode)), - randomPageSize() + randomPageSize(), + null ); } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java index dd7a87dc4a574..8a3c723557151 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java @@ -8,8 +8,10 @@ package org.elasticsearch.compute.aggregation.blockhash; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.analysis.common.CommonAnalysisPlugin; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.MockBigArrays; @@ -35,7 +37,15 @@ import org.elasticsearch.compute.operator.LocalSourceOperator; import org.elasticsearch.compute.operator.PageConsumerOperator; import org.elasticsearch.core.Releasables; - +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.indices.analysis.AnalysisModule; +import org.elasticsearch.plugins.scanners.StablePluginsRegistry; +import org.elasticsearch.xpack.ml.MachineLearning; +import org.junit.Before; + +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -50,6 +60,19 @@ public class CategorizeBlockHashTests extends BlockHashTestCase { + private AnalysisRegistry analysisRegistry; + + @Before + private void initAnalysisRegistry() throws IOException { + analysisRegistry = new AnalysisModule( + TestEnvironment.newEnvironment( + Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()).build() + ), + List.of(new MachineLearning(Settings.EMPTY), new CommonAnalysisPlugin()), + new StablePluginsRegistry() + ).getAnalysisRegistry(); + } + public void testCategorizeRaw() { final Page page; boolean withNull = randomBoolean(); @@ -72,7 +95,7 @@ public void testCategorizeRaw() { page = new Page(builder.build()); } - try (BlockHash hash = new CategorizeRawBlockHash(0, blockFactory, true)) { + try (BlockHash hash = new CategorizeRawBlockHash(0, blockFactory, true, analysisRegistry)) { hash.add(page, new GroupingAggregatorFunction.AddInput() { @Override public void add(int positionOffset, IntBlock groupIds) { @@ -145,8 +168,8 @@ public void testCategorizeIntermediate() { // Fill intermediatePages with the intermediate state from the raw hashes try ( - BlockHash rawHash1 = new CategorizeRawBlockHash(0, blockFactory, true); - BlockHash rawHash2 = new CategorizeRawBlockHash(0, blockFactory, true) + BlockHash rawHash1 = new CategorizeRawBlockHash(0, blockFactory, true, analysisRegistry); + BlockHash rawHash2 = new CategorizeRawBlockHash(0, blockFactory, true, analysisRegistry); ) { rawHash1.add(page1, new GroupingAggregatorFunction.AddInput() { @Override @@ -267,14 +290,16 @@ public void testCategorize_withDriver() { BytesRefVector.Builder textsBuilder = driverContext.blockFactory().newBytesRefVectorBuilder(10); LongVector.Builder countsBuilder = driverContext.blockFactory().newLongVectorBuilder(10) ) { - textsBuilder.appendBytesRef(new BytesRef("a")); - textsBuilder.appendBytesRef(new BytesRef("b")); + // Note that just using "a" or "aaa" doesn't work, because the ml_standard + // tokenizer drops numbers, including hexadecimal ones. + textsBuilder.appendBytesRef(new BytesRef("aaazz")); + textsBuilder.appendBytesRef(new BytesRef("bbbzz")); textsBuilder.appendBytesRef(new BytesRef("words words words goodbye jan")); textsBuilder.appendBytesRef(new BytesRef("words words words goodbye nik")); textsBuilder.appendBytesRef(new BytesRef("words words words goodbye tom")); textsBuilder.appendBytesRef(new BytesRef("words words words hello jan")); - textsBuilder.appendBytesRef(new BytesRef("c")); - textsBuilder.appendBytesRef(new BytesRef("d")); + textsBuilder.appendBytesRef(new BytesRef("ccczz")); + textsBuilder.appendBytesRef(new BytesRef("dddzz")); countsBuilder.appendLong(1); countsBuilder.appendLong(2); countsBuilder.appendLong(800); @@ -293,10 +318,10 @@ public void testCategorize_withDriver() { ) { textsBuilder.appendBytesRef(new BytesRef("words words words hello nik")); textsBuilder.appendBytesRef(new BytesRef("words words words hello nik")); - textsBuilder.appendBytesRef(new BytesRef("c")); + textsBuilder.appendBytesRef(new BytesRef("ccczz")); textsBuilder.appendBytesRef(new BytesRef("words words words goodbye chris")); - textsBuilder.appendBytesRef(new BytesRef("d")); - textsBuilder.appendBytesRef(new BytesRef("e")); + textsBuilder.appendBytesRef(new BytesRef("dddzz")); + textsBuilder.appendBytesRef(new BytesRef("eeezz")); countsBuilder.appendLong(9); countsBuilder.appendLong(90); countsBuilder.appendLong(3); @@ -320,7 +345,8 @@ public void testCategorize_withDriver() { new SumLongAggregatorFunctionSupplier(List.of(1)).groupingAggregatorFactory(AggregatorMode.INITIAL), new MaxLongAggregatorFunctionSupplier(List.of(1)).groupingAggregatorFactory(AggregatorMode.INITIAL) ), - 16 * 1024 + 16 * 1024, + analysisRegistry ).get(driverContext) ), new PageConsumerOperator(intermediateOutput::add), @@ -339,7 +365,8 @@ public void testCategorize_withDriver() { new SumLongAggregatorFunctionSupplier(List.of(1)).groupingAggregatorFactory(AggregatorMode.INITIAL), new MaxLongAggregatorFunctionSupplier(List.of(1)).groupingAggregatorFactory(AggregatorMode.INITIAL) ), - 16 * 1024 + 16 * 1024, + analysisRegistry ).get(driverContext) ), new PageConsumerOperator(intermediateOutput::add), @@ -360,7 +387,8 @@ public void testCategorize_withDriver() { new SumLongAggregatorFunctionSupplier(List.of(1, 2)).groupingAggregatorFactory(AggregatorMode.FINAL), new MaxLongAggregatorFunctionSupplier(List.of(3, 4)).groupingAggregatorFactory(AggregatorMode.FINAL) ), - 16 * 1024 + 16 * 1024, + analysisRegistry ).get(driverContext) ), new PageConsumerOperator(finalOutput::add), @@ -385,15 +413,15 @@ public void testCategorize_withDriver() { sums, equalTo( Map.of( - ".*?a.*?", + ".*?aaazz.*?", 1L, - ".*?b.*?", + ".*?bbbzz.*?", 2L, - ".*?c.*?", + ".*?ccczz.*?", 33L, - ".*?d.*?", + ".*?dddzz.*?", 44L, - ".*?e.*?", + ".*?eeezz.*?", 5L, ".*?words.+?words.+?words.+?goodbye.*?", 8888L, @@ -406,15 +434,15 @@ public void testCategorize_withDriver() { maxs, equalTo( Map.of( - ".*?a.*?", + ".*?aaazz.*?", 1L, - ".*?b.*?", + ".*?bbbzz.*?", 2L, - ".*?c.*?", + ".*?ccczz.*?", 30L, - ".*?d.*?", + ".*?dddzz.*?", 40L, - ".*?e.*?", + ".*?eeezz.*?", 5L, ".*?words.+?words.+?words.+?goodbye.*?", 8000L, diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java index b2f4ad594936e..953c7d1c313f1 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java @@ -59,7 +59,8 @@ protected Operator.OperatorFactory simpleWithMode(AggregatorMode mode) { new SumLongAggregatorFunctionSupplier(sumChannels).groupingAggregatorFactory(mode), new MaxLongAggregatorFunctionSupplier(maxChannels).groupingAggregatorFactory(mode) ), - randomPageSize() + randomPageSize(), + null ); } diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/categorize.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/categorize.csv-spec index 547c430ed7518..e45b10d1aa122 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/categorize.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/categorize.csv-spec @@ -1,5 +1,5 @@ standard aggs -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS count=COUNT(), @@ -17,7 +17,7 @@ count:long | sum:long | avg:double | count_distinct:long | category:keyw ; values aggs -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS values=MV_SORT(VALUES(message)), @@ -33,7 +33,7 @@ values:keyword | top ; mv -required_capability: categorize_v3 +required_capability: categorize_v4 FROM mv_sample_data | STATS COUNT(), SUM(event_duration) BY category=CATEGORIZE(message) @@ -48,7 +48,7 @@ COUNT():long | SUM(event_duration):long | category:keyword ; row mv -required_capability: categorize_v3 +required_capability: categorize_v4 ROW message = ["connected to a", "connected to b", "disconnected"], str = ["a", "b", "c"] | STATS COUNT(), VALUES(str) BY category=CATEGORIZE(message) @@ -60,8 +60,20 @@ COUNT():long | VALUES(str):keyword | category:keyword 1 | [a, b, c] | .*?disconnected.*? ; +skips stopwords +required_capability: categorize_v4 + +ROW message = ["Mon Tue connected to a", "Jul Aug connected to b September ", "UTC connected GMT to c UTC"] + | STATS COUNT() BY category=CATEGORIZE(message) + | SORT category +; + +COUNT():long | category:keyword + 3 | .*?connected.+?to.*? +; + with multiple indices -required_capability: categorize_v3 +required_capability: categorize_v4 required_capability: union_types FROM sample_data* @@ -76,7 +88,7 @@ COUNT():long | category:keyword ; mv with many values -required_capability: categorize_v3 +required_capability: categorize_v4 FROM employees | STATS COUNT() BY category=CATEGORIZE(job_positions) @@ -93,7 +105,7 @@ COUNT():long | category:keyword ; mv with many values and SUM -required_capability: categorize_v3 +required_capability: categorize_v4 FROM employees | STATS SUM(languages) BY category=CATEGORIZE(job_positions) @@ -108,7 +120,7 @@ SUM(languages):long | category:keyword ; mv with many values and nulls and SUM -required_capability: categorize_v3 +required_capability: categorize_v4 FROM employees | STATS SUM(languages) BY category=CATEGORIZE(job_positions) @@ -122,7 +134,7 @@ SUM(languages):long | category:keyword ; mv via eval -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | EVAL message = MV_APPEND(message, "Banana") @@ -138,7 +150,7 @@ COUNT():long | category:keyword ; mv via eval const -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | EVAL message = ["Banana", "Bread"] @@ -152,7 +164,7 @@ COUNT():long | category:keyword ; mv via eval const without aliases -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | EVAL message = ["Banana", "Bread"] @@ -166,7 +178,7 @@ COUNT():long | CATEGORIZE(message):keyword ; mv const in parameter -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY c = CATEGORIZE(["Banana", "Bread"]) @@ -179,7 +191,7 @@ COUNT():long | c:keyword ; agg alias shadowing -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS c = COUNT() BY c = CATEGORIZE(["Banana", "Bread"]) @@ -194,7 +206,7 @@ c:keyword ; chained aggregations using categorize -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(message) @@ -203,13 +215,13 @@ FROM sample_data ; COUNT():long | category:keyword - 1 | .*?\.\*\?Connected\.\+\?to\.\*\?.*? - 1 | .*?\.\*\?Connection\.\+\?error\.\*\?.*? - 1 | .*?\.\*\?Disconnected\.\*\?.*? + 1 | .*?Connected.+?to.*? + 1 | .*?Connection.+?error.*? + 1 | .*?Disconnected.*? ; stats without aggs -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS BY category=CATEGORIZE(message) @@ -223,7 +235,7 @@ category:keyword ; text field -required_capability: categorize_v3 +required_capability: categorize_v4 FROM hosts | STATS COUNT() BY category=CATEGORIZE(host_group) @@ -231,14 +243,17 @@ FROM hosts ; COUNT():long | category:keyword - 2 | .*?DB.+?servers.*? 2 | .*?Gateway.+?instances.*? 5 | .*?Kubernetes.+?cluster.*? + 2 | .*?servers.*? 1 | null + +// Note: DB is removed from "DB servers", because the ml_standard +// tokenizer drops numbers, including hexadecimal ones. ; on TO_UPPER -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(TO_UPPER(message)) @@ -252,7 +267,7 @@ COUNT():long | category:keyword ; on CONCAT -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(CONCAT(message, " banana")) @@ -266,7 +281,7 @@ COUNT():long | category:keyword ; on CONCAT with unicode -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(CONCAT(message, " 👍🏽😊")) @@ -274,13 +289,13 @@ FROM sample_data ; COUNT():long | category:keyword - 3 | .*?Connected.+?to.+?👍🏽😊.*? - 3 | .*?Connection.+?error.+?👍🏽😊.*? - 1 | .*?Disconnected.+?👍🏽😊.*? + 3 | .*?Connected.+?to.*? + 3 | .*?Connection.+?error.*? + 1 | .*?Disconnected.*? ; on REVERSE(CONCAT()) -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(REVERSE(CONCAT(message, " 👍🏽😊"))) @@ -288,13 +303,13 @@ FROM sample_data ; COUNT():long | category:keyword - 1 | .*?😊👍🏽.+?detcennocsiD.*? - 3 | .*?😊👍🏽.+?ot.+?detcennoC.*? - 3 | .*?😊👍🏽.+?rorre.+?noitcennoC.*? + 1 | .*?detcennocsiD.*? + 3 | .*?ot.+?detcennoC.*? + 3 | .*?rorre.+?noitcennoC.*? ; and then TO_LOWER -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(message) @@ -309,7 +324,7 @@ COUNT():long | category:keyword ; on const empty string -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY category=CATEGORIZE("") @@ -321,7 +336,7 @@ COUNT():long | category:keyword ; on const empty string from eval -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | EVAL x = "" @@ -334,7 +349,7 @@ COUNT():long | category:keyword ; on null -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | EVAL x = null @@ -347,7 +362,7 @@ COUNT():long | SUM(event_duration):long | category:keyword ; on null string -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | EVAL x = null::string @@ -360,7 +375,7 @@ COUNT():long | category:keyword ; filtering out all data -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | WHERE @timestamp < "2023-10-23T00:00:00Z" @@ -372,7 +387,7 @@ COUNT():long | category:keyword ; filtering out all data with constant -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS COUNT() BY category=CATEGORIZE(message) @@ -383,7 +398,7 @@ COUNT():long | category:keyword ; drop output columns -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS count=COUNT() BY category=CATEGORIZE(message) @@ -398,7 +413,7 @@ x:integer ; category value processing -required_capability: categorize_v3 +required_capability: categorize_v4 ROW message = ["connected to a", "connected to b", "disconnected"] | STATS COUNT() BY category=CATEGORIZE(message) @@ -412,21 +427,21 @@ COUNT():long | category:keyword ; row aliases -required_capability: categorize_v3 +required_capability: categorize_v4 -ROW message = "connected to a" +ROW message = "connected to xyz" | EVAL x = message | STATS COUNT() BY category=CATEGORIZE(x) | EVAL y = category | SORT y ; -COUNT():long | category:keyword | y:keyword - 1 | .*?connected.+?to.+?a.*? | .*?connected.+?to.+?a.*? +COUNT():long | category:keyword | y:keyword + 1 | .*?connected.+?to.+?xyz.*? | .*?connected.+?to.+?xyz.*? ; from aliases -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | EVAL x = message @@ -442,9 +457,9 @@ COUNT():long | category:keyword | y:keyword ; row aliases with keep -required_capability: categorize_v3 +required_capability: categorize_v4 -ROW message = "connected to a" +ROW message = "connected to xyz" | EVAL x = message | KEEP x | STATS COUNT() BY category=CATEGORIZE(x) @@ -454,11 +469,11 @@ ROW message = "connected to a" ; COUNT():long | y:keyword - 1 | .*?connected.+?to.+?a.*? + 1 | .*?connected.+?to.+?xyz.*? ; from aliases with keep -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | EVAL x = message @@ -476,9 +491,9 @@ COUNT():long | y:keyword ; row rename -required_capability: categorize_v3 +required_capability: categorize_v4 -ROW message = "connected to a" +ROW message = "connected to xyz" | RENAME message as x | STATS COUNT() BY category=CATEGORIZE(x) | RENAME category as y @@ -486,11 +501,11 @@ ROW message = "connected to a" ; COUNT():long | y:keyword - 1 | .*?connected.+?to.+?a.*? + 1 | .*?connected.+?to.+?xyz.*? ; from rename -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | RENAME message as x @@ -506,7 +521,7 @@ COUNT():long | y:keyword ; row drop -required_capability: categorize_v3 +required_capability: categorize_v4 ROW message = "connected to a" | STATS c = COUNT() BY category=CATEGORIZE(message) @@ -519,7 +534,7 @@ c:long ; from drop -required_capability: categorize_v3 +required_capability: categorize_v4 FROM sample_data | STATS c = COUNT() BY category=CATEGORIZE(message) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 77a3e2840977f..373be23cdf847 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -403,7 +403,7 @@ public enum Cap { /** * Supported the text categorization function "CATEGORIZE". */ - CATEGORIZE_V3(Build.current().isSnapshot()), + CATEGORIZE_V4(Build.current().isSnapshot()), /** * QSTR function diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AbstractPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AbstractPhysicalOperationProviders.java index a7418654f6b0e..69e2d1c45aa3c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AbstractPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/AbstractPhysicalOperationProviders.java @@ -18,6 +18,7 @@ import org.elasticsearch.compute.operator.EvalOperator; import org.elasticsearch.compute.operator.HashAggregationOperator.HashAggregationOperatorFactory; import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; import org.elasticsearch.xpack.esql.core.InvalidArgumentException; import org.elasticsearch.xpack.esql.core.expression.Alias; @@ -46,6 +47,11 @@ public abstract class AbstractPhysicalOperationProviders implements PhysicalOperationProviders { private final AggregateMapper aggregateMapper = new AggregateMapper(); + private final AnalysisRegistry analysisRegistry; + + AbstractPhysicalOperationProviders(AnalysisRegistry analysisRegistry) { + this.analysisRegistry = analysisRegistry; + } @Override public final PhysicalOperation groupingPhysicalOperation( @@ -173,7 +179,8 @@ else if (aggregatorMode.isOutputPartial()) { groupSpecs.stream().map(GroupSpec::toHashGroupSpec).toList(), aggregatorMode, aggregatorFactories, - context.pageSize(aggregateExec.estimatedRowSize()) + context.pageSize(aggregateExec.estimatedRowSize()), + analysisRegistry ); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java index 15f5b6579098d..7bf7d0e2d08eb 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java @@ -34,6 +34,7 @@ import org.elasticsearch.compute.operator.SourceOperator; import org.elasticsearch.index.IndexMode; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.FieldNamesFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; @@ -98,7 +99,8 @@ public interface ShardContext extends org.elasticsearch.compute.lucene.ShardCont private final List shardContexts; - public EsPhysicalOperationProviders(List shardContexts) { + public EsPhysicalOperationProviders(List shardContexts, AnalysisRegistry analysisRegistry) { + super(analysisRegistry); this.shardContexts = shardContexts; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java index 73266551f169c..b06dd3cdb64d3 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java @@ -452,7 +452,7 @@ void runCompute(CancellableTask task, ComputeContext context, PhysicalPlan plan, context.exchangeSink(), enrichLookupService, lookupFromIndexService, - new EsPhysicalOperationProviders(contexts) + new EsPhysicalOperationProviders(contexts, searchService.getIndicesService().getAnalysis()) ); LOGGER.debug("Received physical plan:\n{}", plan); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index dd14e8dd82123..d4fca2a0a2540 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1846,7 +1846,7 @@ public void testIntervalAsString() { } public void testCategorizeSingleGrouping() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V4.isEnabled()); query("from test | STATS COUNT(*) BY CATEGORIZE(first_name)"); query("from test | STATS COUNT(*) BY cat = CATEGORIZE(first_name)"); @@ -1875,7 +1875,7 @@ public void testCategorizeSingleGrouping() { } public void testCategorizeNestedGrouping() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V4.isEnabled()); query("from test | STATS COUNT(*) BY CATEGORIZE(LENGTH(first_name)::string)"); @@ -1890,7 +1890,7 @@ public void testCategorizeNestedGrouping() { } public void testCategorizeWithinAggregations() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V4.isEnabled()); query("from test | STATS MV_COUNT(cat), COUNT(*) BY cat = CATEGORIZE(first_name)"); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index e98f2b88b33c9..57d0c7432f97b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -1212,7 +1212,7 @@ public void testCombineProjectionWithAggregationFirstAndAliasedGroupingUsedInAgg * \_EsRelation[test][_meta_field{f}#23, emp_no{f}#17, first_name{f}#18, ..] */ public void testCombineProjectionWithCategorizeGrouping() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V4.isEnabled()); var plan = plan(""" from test @@ -3949,7 +3949,7 @@ public void testNestedExpressionsInGroups() { * \_EsRelation[test][_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, ge..] */ public void testNestedExpressionsInGroupsWithCategorize() { - assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V3.isEnabled()); + assumeTrue("requires Categorize capability", EsqlCapabilities.Cap.CATEGORIZE_V4.isEnabled()); var plan = optimizedPlan(""" from test diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java index ff9e45a9f9233..5d8da21c6faad 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java @@ -156,7 +156,7 @@ private Configuration config() { randomZone(), randomLocale(random()), "test_user", - "test_cluser", + "test_cluster", pragmas, EsqlPlugin.QUERY_RESULT_TRUNCATION_MAX_SIZE.getDefault(null), EsqlPlugin.QUERY_RESULT_TRUNCATION_DEFAULT_SIZE.getDefault(null), @@ -187,7 +187,7 @@ private EsPhysicalOperationProviders esPhysicalOperationProviders() throws IOExc ); } releasables.add(searcher); - return new EsPhysicalOperationProviders(shardContexts); + return new EsPhysicalOperationProviders(shardContexts, null); } private IndexReader reader() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java index c811643c8daea..e91fc6e49312d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java @@ -8,7 +8,9 @@ package org.elasticsearch.xpack.esql.planner; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.analysis.common.CommonAnalysisPlugin; import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.compute.Describable; import org.elasticsearch.compute.aggregation.GroupingAggregator; @@ -28,7 +30,11 @@ import org.elasticsearch.compute.operator.OrdinalsGroupingOperator; import org.elasticsearch.compute.operator.SourceOperator; import org.elasticsearch.compute.operator.SourceOperator.SourceOperatorFactory; +import org.elasticsearch.env.Environment; +import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.indices.analysis.AnalysisModule; +import org.elasticsearch.plugins.scanners.StablePluginsRegistry; import org.elasticsearch.xpack.esql.EsqlIllegalArgumentException; import org.elasticsearch.xpack.esql.TestBlockFactory; import org.elasticsearch.xpack.esql.core.expression.Attribute; @@ -39,7 +45,9 @@ import org.elasticsearch.xpack.esql.plan.physical.FieldExtractExec; import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.LocalExecutionPlannerContext; import org.elasticsearch.xpack.esql.planner.LocalExecutionPlanner.PhysicalOperation; +import org.elasticsearch.xpack.ml.MachineLearning; +import java.io.IOException; import java.util.List; import java.util.Random; import java.util.function.Function; @@ -48,6 +56,7 @@ import static com.carrotsearch.randomizedtesting.generators.RandomNumbers.randomIntBetween; import static java.util.stream.Collectors.joining; +import static org.apache.lucene.tests.util.LuceneTestCase.createTempDir; import static org.elasticsearch.index.mapper.MappedFieldType.FieldExtractPreference.DOC_VALUES; import static org.elasticsearch.index.mapper.MappedFieldType.FieldExtractPreference.NONE; @@ -56,7 +65,16 @@ public class TestPhysicalOperationProviders extends AbstractPhysicalOperationPro private final Page testData; private final List columnNames; - public TestPhysicalOperationProviders(Page testData, List columnNames) { + public TestPhysicalOperationProviders(Page testData, List columnNames) throws IOException { + super( + new AnalysisModule( + TestEnvironment.newEnvironment( + Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()).build() + ), + List.of(new MachineLearning(Settings.EMPTY), new CommonAnalysisPlugin()), + new StablePluginsRegistry() + ).getAnalysisRegistry() + ); this.testData = testData; this.columnNames = columnNames; } From 2226d6cbfa434206826207da46e95969fc77776c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 29 Nov 2024 11:24:28 +0100 Subject: [PATCH 17/27] Add _field_names disabling to archival index tests (#117703) Disabling the "_field_names" field in mappings was possible until 8.x and now issues a deprecation warning. We need to maintain the ability to read these mappings for archival indices so this change adds this case to one of the index mappings in tests and checks for the deprecation warning for it. --- .../test/java/org/elasticsearch/oldrepos/OldMappingsIT.java | 6 +++++- .../test/resources/org/elasticsearch/oldrepos/custom.json | 3 +++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldMappingsIT.java b/x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldMappingsIT.java index 67dbdec6b8399..95bc92d4f185a 100644 --- a/x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldMappingsIT.java +++ b/x-pack/qa/repository-old-versions/src/test/java/org/elasticsearch/oldrepos/OldMappingsIT.java @@ -13,6 +13,7 @@ import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.WarningsHandler; @@ -166,7 +167,10 @@ public void setupIndex() throws IOException { createRestoreRequest.addParameter("wait_for_completion", "true"); createRestoreRequest.setJsonEntity("{\"indices\":\"" + indices.stream().collect(Collectors.joining(",")) + "\"}"); createRestoreRequest.setOptions(RequestOptions.DEFAULT.toBuilder().setWarningsHandler(WarningsHandler.PERMISSIVE)); - assertOK(client().performRequest(createRestoreRequest)); + Response response = client().performRequest(createRestoreRequest); + // check deprecation warning for "_field_name" disabling + assertTrue(response.getWarnings().stream().filter(s -> s.contains("Disabling _field_names is not necessary")).count() > 0); + assertOK(response); } private Request createIndex(String indexName, String file) throws IOException { diff --git a/x-pack/qa/repository-old-versions/src/test/resources/org/elasticsearch/oldrepos/custom.json b/x-pack/qa/repository-old-versions/src/test/resources/org/elasticsearch/oldrepos/custom.json index ae52ccbcce330..ad1c6b0dc59ae 100644 --- a/x-pack/qa/repository-old-versions/src/test/resources/org/elasticsearch/oldrepos/custom.json +++ b/x-pack/qa/repository-old-versions/src/test/resources/org/elasticsearch/oldrepos/custom.json @@ -1,4 +1,7 @@ "_default_": { + "_field_names": { + "enabled": false + }, "properties": { "apache2": { "properties": { From b7c38a1451d13fa7402ff7055231451f43ac3ac6 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Fri, 29 Nov 2024 21:54:34 +1100 Subject: [PATCH 18/27] Mute org.elasticsearch.xpack.esql.ccq.MultiClusterSpecIT test {scoring.QstrWithFieldAndScoringSortedEval} #117751 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 40d3dcf46e1b9..96631d15f374f 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -225,6 +225,9 @@ tests: - class: org.elasticsearch.xpack.inference.InferenceCrudIT method: testSupportedStream issue: https://github.com/elastic/elasticsearch/issues/117745 +- class: org.elasticsearch.xpack.esql.ccq.MultiClusterSpecIT + method: test {scoring.QstrWithFieldAndScoringSortedEval} + issue: https://github.com/elastic/elasticsearch/issues/117751 # Examples: # From 045f6a31f994f51d87a217be60251e060132c8a1 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 29 Nov 2024 11:55:51 +0100 Subject: [PATCH 19/27] Add INDEX_REFRESH_BLOCK (#117543) This change adds a new ClusterBlockLevel called REFRESH. This level is used in a new ClusterBlock.INDEX_REFRESH_BLOCK which is automatically added to new indices that are created from empty store, with replicas, and only on serverless deployments that have a feature flag enabled. This block is also only added when all nodes of a cluster are in a recent enough transport version. If for some reason the new ClusterBlock is sent over the wire to a node with an old transport version, the REFRESH cluster block level will be removed from the set of level blocked. In the future, the REFRESH cluster block will be used: to block refreshes on shards until an unpromotable shard is started to allow skipping shards when searching Relates ES-10131 --- .../org/elasticsearch/TransportVersions.java | 1 + .../cluster/block/ClusterBlock.java | 24 +++++- .../cluster/block/ClusterBlockLevel.java | 3 +- .../cluster/metadata/IndexMetadata.java | 9 ++ .../metadata/MetadataCreateIndexService.java | 54 ++++++++++++ .../cluster/ClusterStateTests.java | 18 ++-- .../cluster/block/ClusterBlockTests.java | 49 +++++++++-- .../MetadataCreateIndexServiceTests.java | 86 ++++++++++++++++++- 8 files changed, 228 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index a1315ccf66701..b38a285907937 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -211,6 +211,7 @@ static TransportVersion def(int id) { public static final TransportVersion ESQL_REMOVE_NODE_LEVEL_PLAN = def(8_800_00_0); public static final TransportVersion LOGSDB_TELEMETRY_CUSTOM_CUTOFF_DATE = def(8_801_00_0); public static final TransportVersion SOURCE_MODE_TELEMETRY = def(8_802_00_0); + public static final TransportVersion NEW_REFRESH_CLUSTER_BLOCK = def(8_803_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java index 4e47925d383c2..25c6a1ff5b67f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java +++ b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java @@ -9,6 +9,7 @@ package org.elasticsearch.cluster.block; +import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; @@ -21,6 +22,7 @@ import java.util.EnumSet; import java.util.Locale; import java.util.Objects; +import java.util.function.Predicate; public class ClusterBlock implements Writeable, ToXContentFragment { @@ -142,7 +144,12 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(id); out.writeOptionalString(uuid); out.writeString(description); - out.writeEnumSet(levels); + if (out.getTransportVersion().onOrAfter(TransportVersions.NEW_REFRESH_CLUSTER_BLOCK)) { + out.writeEnumSet(levels); + } else { + // do not send ClusterBlockLevel.REFRESH to old nodes + out.writeEnumSet(filterLevels(levels, level -> ClusterBlockLevel.REFRESH.equals(level) == false)); + } out.writeBoolean(retryable); out.writeBoolean(disableStatePersistence); RestStatus.writeTo(out, status); @@ -185,4 +192,19 @@ public int hashCode() { public boolean isAllowReleaseResources() { return allowReleaseResources; } + + static EnumSet filterLevels(EnumSet levels, Predicate predicate) { + assert levels != null; + int size = levels.size(); + if (size == 0 || (size == 1 && predicate.test(levels.iterator().next()))) { + return levels; + } + var filteredLevels = EnumSet.noneOf(ClusterBlockLevel.class); + for (ClusterBlockLevel level : levels) { + if (predicate.test(level)) { + filteredLevels.add(level); + } + } + return filteredLevels; + } } diff --git a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlockLevel.java b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlockLevel.java index f6330fb18e5e6..262044b091ac7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlockLevel.java +++ b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlockLevel.java @@ -15,7 +15,8 @@ public enum ClusterBlockLevel { READ, WRITE, METADATA_READ, - METADATA_WRITE; + METADATA_WRITE, + REFRESH; public static final EnumSet ALL = EnumSet.allOf(ClusterBlockLevel.class); public static final EnumSet READ_WRITE = EnumSet.of(READ, WRITE); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java index 6456240c2317e..b7c1ee5fbad96 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexMetadata.java @@ -140,6 +140,15 @@ public class IndexMetadata implements Diffable, ToXContentFragmen RestStatus.TOO_MANY_REQUESTS, EnumSet.of(ClusterBlockLevel.WRITE) ); + public static final ClusterBlock INDEX_REFRESH_BLOCK = new ClusterBlock( + 14, + "index refresh blocked, waiting for shard(s) to be started", + true, + false, + false, + RestStatus.REQUEST_TIMEOUT, + EnumSet.of(ClusterBlockLevel.REFRESH) + ); // 'event.ingested' (part of Elastic Common Schema) range is tracked in cluster state, along with @timestamp public static final String EVENT_INGESTED_FIELD_NAME = "event.ingested"; diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java index 1f014a526b9a6..52e4d75ac5116 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexService.java @@ -28,6 +28,7 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.RoutingTable; @@ -127,6 +128,16 @@ public class MetadataCreateIndexService { public static final int MAX_INDEX_NAME_BYTES = 255; + /** + * Name of the setting used to allow blocking refreshes on newly created indices. + */ + public static final String USE_INDEX_REFRESH_BLOCK_SETTING_NAME = "stateless.indices.use_refresh_block_upon_index_creation"; + + @FunctionalInterface + interface ClusterBlocksTransformer { + void apply(ClusterBlocks.Builder clusterBlocks, IndexMetadata indexMetadata, TransportVersion minClusterTransportVersion); + } + private final Settings settings; private final ClusterService clusterService; private final IndicesService indicesService; @@ -139,6 +150,7 @@ public class MetadataCreateIndexService { private final boolean forbidPrivateIndexSettings; private final Set indexSettingProviders; private final ThreadPool threadPool; + private final ClusterBlocksTransformer blocksTransformerUponIndexCreation; public MetadataCreateIndexService( final Settings settings, @@ -166,6 +178,7 @@ public MetadataCreateIndexService( this.shardLimitValidator = shardLimitValidator; this.indexSettingProviders = indexSettingProviders.getIndexSettingProviders(); this.threadPool = threadPool; + this.blocksTransformerUponIndexCreation = createClusterBlocksTransformerForIndexCreation(settings); } /** @@ -540,8 +553,10 @@ private ClusterState applyCreateIndexWithTemporaryService( currentState, indexMetadata, metadataTransformer, + blocksTransformerUponIndexCreation, allocationService.getShardRoutingRoleStrategy() ); + assert assertHasRefreshBlock(indexMetadata, updated, updated.getMinTransportVersion()); if (request.performReroute()) { updated = allocationService.reroute(updated, "index [" + indexMetadata.getIndex().getName() + "] created", rerouteListener); } @@ -1294,6 +1309,7 @@ static ClusterState clusterStateCreateIndex( ClusterState currentState, IndexMetadata indexMetadata, BiConsumer metadataTransformer, + ClusterBlocksTransformer blocksTransformer, ShardRoutingRoleStrategy shardRoutingRoleStrategy ) { final Metadata newMetadata; @@ -1307,6 +1323,9 @@ static ClusterState clusterStateCreateIndex( var blocksBuilder = ClusterBlocks.builder().blocks(currentState.blocks()); blocksBuilder.updateBlocks(indexMetadata); + if (blocksTransformer != null) { + blocksTransformer.apply(blocksBuilder, indexMetadata, currentState.getMinTransportVersion()); + } var routingTableBuilder = RoutingTable.builder(shardRoutingRoleStrategy, currentState.routingTable()) .addAsNew(newMetadata.index(indexMetadata.getIndex().getName())); @@ -1745,4 +1764,39 @@ public static void validateStoreTypeSetting(Settings indexSettings) { ); } } + + private static boolean useRefreshBlock(Settings settings) { + return DiscoveryNode.isStateless(settings) && settings.getAsBoolean(USE_INDEX_REFRESH_BLOCK_SETTING_NAME, false); + } + + static ClusterBlocksTransformer createClusterBlocksTransformerForIndexCreation(Settings settings) { + if (useRefreshBlock(settings) == false) { + return (clusterBlocks, indexMetadata, minClusterTransportVersion) -> {}; + } + logger.debug("applying refresh block on index creation"); + return (clusterBlocks, indexMetadata, minClusterTransportVersion) -> { + if (applyRefreshBlock(indexMetadata, minClusterTransportVersion)) { + // Applies the INDEX_REFRESH_BLOCK to the index. This block will remain in cluster state until an unpromotable shard is + // started or a configurable delay is elapsed. + clusterBlocks.addIndexBlock(indexMetadata.getIndex().getName(), IndexMetadata.INDEX_REFRESH_BLOCK); + } + }; + } + + private static boolean applyRefreshBlock(IndexMetadata indexMetadata, TransportVersion minClusterTransportVersion) { + return 0 < indexMetadata.getNumberOfReplicas() // index has replicas + && indexMetadata.getResizeSourceIndex() == null // index is not a split/shrink index + && indexMetadata.getInSyncAllocationIds().values().stream().allMatch(Set::isEmpty) // index is a new index + && minClusterTransportVersion.onOrAfter(TransportVersions.NEW_REFRESH_CLUSTER_BLOCK); + } + + private boolean assertHasRefreshBlock(IndexMetadata indexMetadata, ClusterState clusterState, TransportVersion minTransportVersion) { + var hasRefreshBlock = clusterState.blocks().hasIndexBlock(indexMetadata.getIndex().getName(), IndexMetadata.INDEX_REFRESH_BLOCK); + if (useRefreshBlock(settings) == false || applyRefreshBlock(indexMetadata, minTransportVersion) == false) { + assert hasRefreshBlock == false : indexMetadata.getIndex(); + } else { + assert hasRefreshBlock : indexMetadata.getIndex(); + } + return true; + } } diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java b/server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java index 9613086aa9f57..668aea70c23f2 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java @@ -167,7 +167,8 @@ public void testToXContent() throws IOException { "read", "write", "metadata_read", - "metadata_write" + "metadata_write", + "refresh" ] } }, @@ -180,7 +181,8 @@ public void testToXContent() throws IOException { "read", "write", "metadata_read", - "metadata_write" + "metadata_write", + "refresh" ] } } @@ -440,7 +442,8 @@ public void testToXContent_FlatSettingTrue_ReduceMappingFalse() throws IOExcepti "read", "write", "metadata_read", - "metadata_write" + "metadata_write", + "refresh" ] } }, @@ -453,7 +456,8 @@ public void testToXContent_FlatSettingTrue_ReduceMappingFalse() throws IOExcepti "read", "write", "metadata_read", - "metadata_write" + "metadata_write", + "refresh" ] } } @@ -712,7 +716,8 @@ public void testToXContent_FlatSettingFalse_ReduceMappingTrue() throws IOExcepti "read", "write", "metadata_read", - "metadata_write" + "metadata_write", + "refresh" ] } }, @@ -725,7 +730,8 @@ public void testToXContent_FlatSettingFalse_ReduceMappingTrue() throws IOExcepti "read", "write", "metadata_read", - "metadata_write" + "metadata_write", + "refresh" ] } } diff --git a/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java b/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java index 311f2ec36af5c..0237fff8fdda5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java @@ -10,19 +10,22 @@ package org.elasticsearch.cluster.block; import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; -import java.util.Arrays; import java.util.Collections; -import java.util.List; +import java.util.EnumSet; import java.util.Map; import static java.util.EnumSet.copyOf; +import static org.elasticsearch.test.TransportVersionUtils.getFirstVersion; +import static org.elasticsearch.test.TransportVersionUtils.getPreviousVersion; import static org.elasticsearch.test.TransportVersionUtils.randomVersion; +import static org.elasticsearch.test.TransportVersionUtils.randomVersionBetween; import static org.hamcrest.CoreMatchers.endsWith; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; @@ -36,7 +39,7 @@ public void testSerialization() throws Exception { int iterations = randomIntBetween(5, 20); for (int i = 0; i < iterations; i++) { TransportVersion version = randomVersion(random()); - ClusterBlock clusterBlock = randomClusterBlock(); + ClusterBlock clusterBlock = randomClusterBlock(version); BytesStreamOutput out = new BytesStreamOutput(); out.setTransportVersion(version); @@ -50,13 +53,41 @@ public void testSerialization() throws Exception { } } + public void testSerializationBwc() throws Exception { + var out = new BytesStreamOutput(); + out.setTransportVersion( + randomVersionBetween(random(), getFirstVersion(), getPreviousVersion(TransportVersions.NEW_REFRESH_CLUSTER_BLOCK)) + ); + + var clusterBlock = randomClusterBlock(TransportVersions.NEW_REFRESH_CLUSTER_BLOCK); + clusterBlock.writeTo(out); + + var in = out.bytes().streamInput(); + in.setTransportVersion(randomVersion()); + + assertClusterBlockEquals( + new ClusterBlock( + clusterBlock.id(), + clusterBlock.uuid(), + clusterBlock.description(), + clusterBlock.retryable(), + clusterBlock.disableStatePersistence(), + clusterBlock.isAllowReleaseResources(), + clusterBlock.status(), + // ClusterBlockLevel.REFRESH should not be sent over the wire to nodes with version < NEW_REFRESH_CLUSTER_BLOCK + ClusterBlock.filterLevels(clusterBlock.levels(), level -> ClusterBlockLevel.REFRESH.equals(level) == false) + ), + new ClusterBlock(in) + ); + } + public void testToStringDanglingComma() { - final ClusterBlock clusterBlock = randomClusterBlock(); + final ClusterBlock clusterBlock = randomClusterBlock(randomVersion(random())); assertThat(clusterBlock.toString(), not(endsWith(","))); } public void testGlobalBlocksCheckedIfNoIndicesSpecified() { - ClusterBlock globalBlock = randomClusterBlock(); + ClusterBlock globalBlock = randomClusterBlock(randomVersion(random())); ClusterBlocks clusterBlocks = new ClusterBlocks(Collections.singleton(globalBlock), Map.of()); ClusterBlockException exception = clusterBlocks.indicesBlockedException(randomFrom(globalBlock.levels()), new String[0]); assertNotNull(exception); @@ -113,9 +144,13 @@ public void testGetIndexBlockWithId() { assertThat(builder.build().getIndexBlockWithId("index", randomValueOtherThan(blockId, ESTestCase::randomInt)), nullValue()); } - private static ClusterBlock randomClusterBlock() { + private static ClusterBlock randomClusterBlock(TransportVersion version) { final String uuid = randomBoolean() ? UUIDs.randomBase64UUID() : null; - final List levels = Arrays.asList(ClusterBlockLevel.values()); + final EnumSet levels = ClusterBlock.filterLevels( + EnumSet.allOf(ClusterBlockLevel.class), + // Filter out ClusterBlockLevel.REFRESH for versions < TransportVersions.NEW_REFRESH_CLUSTER_BLOCK + level -> ClusterBlockLevel.REFRESH.equals(level) == false || version.onOrAfter(TransportVersions.NEW_REFRESH_CLUSTER_BLOCK) + ); return new ClusterBlock( randomInt(), uuid, diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java index 96a74d2e23aad..1876a1f2da556 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -36,6 +36,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.version.CompatibilityVersions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.settings.IndexScopedSettings; @@ -66,6 +67,7 @@ import org.elasticsearch.snapshots.EmptySnapshotsInfoService; import org.elasticsearch.test.ClusterServiceUtils; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.TransportVersionUtils; import org.elasticsearch.test.gateway.TestGatewayAllocator; import org.elasticsearch.test.index.IndexVersionUtils; import org.elasticsearch.threadpool.TestThreadPool; @@ -105,6 +107,8 @@ import static org.elasticsearch.cluster.metadata.MetadataCreateIndexService.resolveAndValidateAliases; import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.elasticsearch.indices.ShardLimitValidatorTests.createTestShardLimitService; +import static org.hamcrest.Matchers.aMapWithSize; +import static org.hamcrest.Matchers.anEmptyMap; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.equalTo; @@ -1133,7 +1137,7 @@ public void testClusterStateCreateIndexThrowsWriteIndexValidationException() thr assertThat( expectThrows( IllegalStateException.class, - () -> clusterStateCreateIndex(currentClusterState, newIndex, null, TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY) + () -> clusterStateCreateIndex(currentClusterState, newIndex, null, null, TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY) ).getMessage(), startsWith("alias [alias1] has more than one write index [") ); @@ -1153,6 +1157,7 @@ public void testClusterStateCreateIndex() { currentClusterState, newIndexMetadata, null, + null, TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY ); assertThat(updatedClusterState.blocks().getIndexBlockWithId("test", INDEX_READ_ONLY_BLOCK.id()), is(INDEX_READ_ONLY_BLOCK)); @@ -1198,6 +1203,7 @@ public void testClusterStateCreateIndexWithMetadataTransaction() { currentClusterState, newIndexMetadata, metadataTransformer, + null, TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY ); assertTrue(updatedClusterState.metadata().findAllAliases(new String[] { "my-index" }).containsKey("my-index")); @@ -1547,6 +1553,84 @@ public void testDeprecateSimpleFS() { ); } + public void testClusterStateCreateIndexWithClusterBlockTransformer() { + { + var emptyClusterState = ClusterState.builder(ClusterState.EMPTY_STATE).build(); + var updatedClusterState = clusterStateCreateIndex( + emptyClusterState, + IndexMetadata.builder("test") + .settings(settings(IndexVersion.current())) + .numberOfShards(1) + .numberOfReplicas(randomIntBetween(1, 3)) + .build(), + null, + MetadataCreateIndexService.createClusterBlocksTransformerForIndexCreation(Settings.EMPTY), + TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY + ); + assertThat(updatedClusterState.blocks().indices(), is(anEmptyMap())); + assertThat(updatedClusterState.blocks().hasIndexBlock("test", IndexMetadata.INDEX_REFRESH_BLOCK), is(false)); + assertThat(updatedClusterState.routingTable().index("test"), is(notNullValue())); + } + { + var minTransportVersion = TransportVersionUtils.randomCompatibleVersion(random()); + var emptyClusterState = ClusterState.builder(ClusterState.EMPTY_STATE) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("_node_id")).build()) + .putCompatibilityVersions("_node_id", new CompatibilityVersions(minTransportVersion, Map.of())) + .build(); + var settings = Settings.builder() + .put(DiscoveryNode.STATELESS_ENABLED_SETTING_NAME, true) + .put(MetadataCreateIndexService.USE_INDEX_REFRESH_BLOCK_SETTING_NAME, true) + .build(); + int nbReplicas = randomIntBetween(0, 1); + var updatedClusterState = clusterStateCreateIndex( + emptyClusterState, + IndexMetadata.builder("test") + .settings(settings(IndexVersion.current())) + .numberOfShards(1) + .numberOfReplicas(nbReplicas) + .build() + .withTimestampRanges(IndexLongFieldRange.UNKNOWN, IndexLongFieldRange.UNKNOWN, minTransportVersion), + null, + MetadataCreateIndexService.createClusterBlocksTransformerForIndexCreation(settings), + TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY + ); + + var expectRefreshBlock = 0 < nbReplicas && minTransportVersion.onOrAfter(TransportVersions.NEW_REFRESH_CLUSTER_BLOCK); + assertThat(updatedClusterState.blocks().indices(), is(aMapWithSize(expectRefreshBlock ? 1 : 0))); + assertThat(updatedClusterState.blocks().hasIndexBlock("test", IndexMetadata.INDEX_REFRESH_BLOCK), is(expectRefreshBlock)); + assertThat(updatedClusterState.routingTable().index("test"), is(notNullValue())); + } + } + + public void testCreateClusterBlocksTransformerForIndexCreation() { + boolean isStateless = randomBoolean(); + boolean useRefreshBlock = randomBoolean(); + var minTransportVersion = TransportVersionUtils.randomCompatibleVersion(random()); + + var applier = MetadataCreateIndexService.createClusterBlocksTransformerForIndexCreation( + Settings.builder() + .put(DiscoveryNode.STATELESS_ENABLED_SETTING_NAME, isStateless) + .put(MetadataCreateIndexService.USE_INDEX_REFRESH_BLOCK_SETTING_NAME, useRefreshBlock) + .build() + ); + assertThat(applier, notNullValue()); + + var blocks = ClusterBlocks.builder().blocks(ClusterState.EMPTY_STATE.blocks()); + applier.apply( + blocks, + IndexMetadata.builder("test") + .settings(settings(IndexVersion.current())) + .numberOfShards(1) + .numberOfReplicas(randomIntBetween(1, 3)) + .build(), + minTransportVersion + ); + assertThat( + blocks.hasIndexBlock("test", IndexMetadata.INDEX_REFRESH_BLOCK), + is(isStateless && useRefreshBlock && minTransportVersion.onOrAfter(TransportVersions.NEW_REFRESH_CLUSTER_BLOCK)) + ); + } + private IndexTemplateMetadata addMatchingTemplate(Consumer configurator) { IndexTemplateMetadata.Builder builder = templateMetadataBuilder("template1", "te*"); configurator.accept(builder); From ad83d9b35ddc01229a5b2b5de21b122f9d1b2106 Mon Sep 17 00:00:00 2001 From: Panagiotis Bailis Date: Fri, 29 Nov 2024 14:50:01 +0200 Subject: [PATCH 20/27] Updating retriever-examples documentation to run validation tests on the provided snippets (#116643) --- docs/reference/search/rrf.asciidoc | 98 +- .../retrievers-examples.asciidoc | 1270 ++++++++++++++--- 2 files changed, 1149 insertions(+), 219 deletions(-) diff --git a/docs/reference/search/rrf.asciidoc b/docs/reference/search/rrf.asciidoc index edd3b67e3de04..a942c0162a80a 100644 --- a/docs/reference/search/rrf.asciidoc +++ b/docs/reference/search/rrf.asciidoc @@ -105,7 +105,7 @@ The `rrf` retriever does not currently support: * <> Using unsupported features as part of a search with an `rrf` retriever results in an exception. -+ + IMPORTANT: It is best to avoid providing a <> as part of the request, as RRF creates one internally that is shared by all sub-retrievers to ensure consistent results. @@ -703,3 +703,99 @@ So for the same params as above, we would now have: * `from=0, size=2` would return [`1`, `5`] with ranks `[1, 2]` * `from=2, size=2` would return an empty result set as it would fall outside the available `rank_window_size` results. + +==== Aggregations in RRF + +The `rrf` retriever supports aggregations from all specified sub-retrievers. Important notes about aggregations: + +* They operate on the complete result set from all sub-retrievers +* They are not limited by the `rank_window_size` parameter +* They process the union of all matching documents + +For example, consider the following document set: +[source,js] +---- +{ + "_id": 1, "termA": "foo", + "_id": 2, "termA": "foo", "termB": "bar", + "_id": 3, "termA": "aardvark", "termB": "bar", + "_id": 4, "termA": "foo", "termB": "bar" +} +---- +// NOTCONSOLE + +Perform a term aggregation on the `termA` field using an `rrf` retriever: +[source,js] +---- +{ + "retriever": { + "rrf": { + "retrievers": [ + { + "standard": { + "query": { + "term": { + "termB": "bar" + } + } + } + }, + { + "standard": { + "query": { + "match_all": { } + } + } + } + ], + "rank_window_size": 1 + } + }, + "size": 1, + "aggs": { + "termA_agg": { + "terms": { + "field": "termA" + } + } + } +} +---- +// NOTCONSOLE + +The aggregation results will include *all* matching documents, regardless of `rank_window_size`. +[source, js] +---- +{ + "foo": 3, + "aardvark": 1 +} + +---- +// NOTCONSOLE + +==== Highlighting in RRF + +Using the `rrf` retriever, you can add <> to show relevant text snippets in your search results. Highlighted snippets are computed based +on the matching text queries defined on the sub-retrievers. + +IMPORTANT: Highlighting on vector fields, using either the `knn` retriever or a `knn` query, is not supported. + +A more specific example of highlighting in RRF can also be found in the <> page. + +==== Inner hits in RRF + +The `rrf` retriever supports <> functionality, allowing you to retrieve +related nested or parent/child documents alongside your main search results. Inner hits can be +specified as part of any nested sub-retriever and will be propagated to the top-level parent +retriever. Note that the inner hit computation will take place only at end of `rrf` retriever's +evaluation on the top matching documents, and not as part of the query execution of the nested +sub-retrievers. + +[IMPORTANT] +==== +When defining multiple `inner_hits` sections across sub-retrievers: + +* Each `inner_hits` section must have a unique name +* Names must be unique across all sub-retrievers in the search request +==== diff --git a/docs/reference/search/search-your-data/retrievers-examples.asciidoc b/docs/reference/search/search-your-data/retrievers-examples.asciidoc index 8cd1a4bf5ce98..ad1cc32dcee01 100644 --- a/docs/reference/search/search-your-data/retrievers-examples.asciidoc +++ b/docs/reference/search/search-your-data/retrievers-examples.asciidoc @@ -1,31 +1,16 @@ [[retrievers-examples]] -=== Retrievers examples Learn how to combine different retrievers in these hands-on examples. -To demonstrate the full functionality of retrievers, these examples require access to a <> set up using the <>. + +=== Retrievers examples [discrete] [[retrievers-examples-setup]] ==== Add example data -To begin with, we'll set up the necessary services and have them in place for later use. - -[source,js] ----- -// Setup rerank task stored as `my-rerank-model` -PUT _inference/rerank/my-rerank-model -{ - "service": "cohere", - "service_settings": { - "model_id": "rerank-english-v3.0", - "api_key": "{{COHERE_API_KEY}}" - } -} ----- -//NOTCONSOLE +To begin with, lets create the `retrievers_example` index, and add some documents to it. -Now that we have our reranking service in place, lets create the `retrievers_example` index, and add some documents to it. -[source,js] +[source,console] ---- PUT retrievers_example { @@ -49,11 +34,7 @@ PUT retrievers_example } } } ----- -//NOTCONSOLE -[source,js] ----- POST /retrievers_example/_doc/1 { "vector": [0.23, 0.67, 0.89], @@ -94,10 +75,12 @@ POST /retrievers_example/_doc/5 "topic": ["documentation", "observability", "elastic"] } +POST /retrievers_example/_refresh + ---- -//NOTCONSOLE +// TESTSETUP -Now that we also have our documents in place, let's try to run some queries using retrievers. +Now that we have our documents in place, let's try to run some queries using retrievers. [discrete] [[retrievers-examples-combining-standard-knn-retrievers-with-rrf]] @@ -112,170 +95,272 @@ To implement this in the retriever framework, we start with the top-level elemen retriever. This retriever operates on top of two other retrievers: a `knn` retriever and a `standard` retriever. Our query structure would look like this: -[source,js] +[source,console] ---- GET /retrievers_example/_search { - "retriever":{ - "rrf": { - "retrievers":[ - { - "standard":{ - "query":{ - "query_string":{ - "query": "(information retrieval) OR (artificial intelligence)", - "default_field": "text" - } - } - } - }, - { - "knn": { - "field": "vector", - "query_vector": [ - 0.23, - 0.67, - 0.89 - ], - "k": 3, - "num_candidates": 5 - } - } - ], - "rank_window_size": 10, - "rank_constant": 1 - } - }, - "_source": ["text", "topic"] + "retriever": { + "rrf": { + "retrievers": [ + { + "standard": { + "query": { + "query_string": { + "query": "(information retrieval) OR (artificial intelligence)", + "default_field": "text" + } + } + } + }, + { + "knn": { + "field": "vector", + "query_vector": [ + 0.23, + 0.67, + 0.89 + ], + "k": 3, + "num_candidates": 5 + } + } + ], + "rank_window_size": 10, + "rank_constant": 1 + } + }, + "_source": false +} +---- +// TEST + +This returns the following response based on the final rrf score for each result. + +.Example response +[%collapsible] +============== +[source,console-result] +---- +{ + "took": 42, + "timed_out": false, + "_shards": { + "total": 1, + "successful": 1, + "skipped": 0, + "failed": 0 + }, + "hits": { + "total": { + "value": 3, + "relation": "eq" + }, + "max_score": 0.8333334, + "hits": [ + { + "_index": "retrievers_example", + "_id": "1", + "_score": 0.8333334 + }, + { + "_index": "retrievers_example", + "_id": "2", + "_score": 0.8333334 + }, + { + "_index": "retrievers_example", + "_id": "3", + "_score": 0.25 + } + ] + } } ---- -//NOTCONSOLE +// TESTRESPONSE[s/"took": 42/"took": $body.took/] +============== [discrete] [[retrievers-examples-collapsing-retriever-results]] ==== Example: Grouping results by year with `collapse` In our result set, we have many documents with the same `year` value. We can clean this -up using the `collapse` parameter with our retriever. This enables grouping results by -any field and returns only the highest-scoring document from each group. In this example +up using the `collapse` parameter with our retriever. This, as with the standard <> feature, +enables grouping results by any field and returns only the highest-scoring document from each group. In this example we'll collapse our results based on the `year` field. -[source,js] +[source,console] ---- GET /retrievers_example/_search { - "retriever":{ - "rrf": { - "retrievers":[ - { - "standard":{ - "query":{ - "query_string":{ - "query": "(information retrieval) OR (artificial intelligence)", - "default_field": "text" - } - } - } - }, - { - "knn": { - "field": "vector", - "query_vector": [ - 0.23, - 0.67, - 0.89 - ], - "k": 3, - "num_candidates": 5 - } - } - ], - "rank_window_size": 10, - "rank_constant": 1 - } - }, - "collapse": { - "field": "year", - "inner_hits": { - "name": "topic related documents", - "_source": ["text", "year"] - } - }, - "_source": ["text", "topic"] + "retriever": { + "rrf": { + "retrievers": [ + { + "standard": { + "query": { + "query_string": { + "query": "(information retrieval) OR (artificial intelligence)", + "default_field": "text" + } + } + } + }, + { + "knn": { + "field": "vector", + "query_vector": [ + 0.23, + 0.67, + 0.89 + ], + "k": 3, + "num_candidates": 5 + } + } + ], + "rank_window_size": 10, + "rank_constant": 1 + } + }, + "collapse": { + "field": "year", + "inner_hits": { + "name": "topic related documents", + "_source": [ + "year" + ] + } + }, + "_source": false } ---- -//NOTCONSOLE +// TEST[continued] -[discrete] -[[retrievers-examples-text-similarity-reranker-on-top-of-rrf]] -==== Example: Rerank results of an RRF retriever +This returns the following response with collapsed results. -Previously, we used a `text_similarity_reranker` retriever within an `rrf` retriever. -Because retrievers support full composability, we can also rerank the results of an -`rrf` retriever. Let's apply this to our first example. - -[source,js] +.Example response +[%collapsible] +============== +[source,console-result] ---- -GET retrievers_example/_search { - "retriever": { - "text_similarity_reranker": { - "retriever": { - "rrf": { - "retrievers": [ - { - "standard":{ - "query":{ - "query_string":{ - "query": "(information retrieval) OR (artificial intelligence)", - "default_field": "text" - } - } - } - }, - { - "knn": { - "field": "vector", - "query_vector": [ - 0.23, - 0.67, - 0.89 - ], - "k": 3, - "num_candidates": 5 - } - } - ], - "rank_window_size": 10, - "rank_constant": 1 - } - }, - "field": "text", - "inference_id": "my-rerank-model", - "inference_text": "What are the state of the art applications of AI in information retrieval?" - } - }, - "_source": ["text", "topic"] + "took": 42, + "timed_out": false, + "_shards": { + "total": 1, + "successful": 1, + "skipped": 0, + "failed": 0 + }, + "hits": { + "total": { + "value": 3, + "relation": "eq" + }, + "max_score": 0.8333334, + "hits": [ + { + "_index": "retrievers_example", + "_id": "1", + "_score": 0.8333334, + "fields": { + "year": [ + 2024 + ] + }, + "inner_hits": { + "topic related documents": { + "hits": { + "total": { + "value": 2, + "relation": "eq" + }, + "max_score": 0.8333334, + "hits": [ + { + "_index": "retrievers_example", + "_id": "1", + "_score": 0.8333334, + "_source": { + "year": 2024 + } + }, + { + "_index": "retrievers_example", + "_id": "3", + "_score": 0.25, + "_source": { + "year": 2024 + } + } + ] + } + } + } + }, + { + "_index": "retrievers_example", + "_id": "2", + "_score": 0.8333334, + "fields": { + "year": [ + 2023 + ] + }, + "inner_hits": { + "topic related documents": { + "hits": { + "total": { + "value": 1, + "relation": "eq" + }, + "max_score": 0.8333334, + "hits": [ + { + "_index": "retrievers_example", + "_id": "2", + "_score": 0.8333334, + "_source": { + "year": 2023 + } + } + ] + } + } + } + } + ] + } } - ---- -//NOTCONSOLE +// TESTRESPONSE[s/"took": 42/"took": $body.took/] +============== [discrete] -[[retrievers-examples-rrf-ranking-on-text-similarity-reranker-results]] -==== Example: RRF with semantic reranker +[[retrievers-examples-highlighting-retriever-results]] +==== Example: Highlighting results based on nested sub-retrievers -For this example, we'll replace our semantic query with the `my-rerank-model` -reranker we previously configured. Since this is a reranker, it needs an initial pool of -documents to work with. In this case, we'll filter for documents about `ai` topics. +Highlighting is now also available for nested sub-retrievers matches. For example, consider the same +`rrf` retriever as above, with a `knn` and `standard` retriever as its sub-retrievers. We can specify a `highlight` +section, as defined in <> documentation, and compute highlights for the top results. -[source,js] +[source,console] ---- GET /retrievers_example/_search { "retriever": { "rrf": { "retrievers": [ + { + "standard": { + "query": { + "query_string": { + "query": "(information retrieval) OR (artificial intelligence)", + "default_field": "text" + } + } + } + }, { "knn": { "field": "vector", @@ -287,21 +372,221 @@ GET /retrievers_example/_search "k": 3, "num_candidates": 5 } - }, + } + ], + "rank_window_size": 10, + "rank_constant": 1 + } + }, + "highlight": { + "fields": { + "text": { + "fragment_size": 150, + "number_of_fragments": 3 + } + } + }, + "_source": false +} +---- +// TEST[continued] + +This would highlight the `text` field, based on the matches produced by the `standard` retriever. The highlighted snippets +would then be included in the response as usual, i.e. under each search hit. + +.Example response +[%collapsible] +============== +[source,console-result] +---- +{ + "took": 42, + "timed_out": false, + "_shards": { + "total": 1, + "successful": 1, + "skipped": 0, + "failed": 0 + }, + "hits": { + "total": { + "value": 3, + "relation": "eq" + }, + "max_score": 0.8333334, + "hits": [ + { + "_index": "retrievers_example", + "_id": "1", + "_score": 0.8333334, + "highlight": { + "text": [ + "Large language models are revolutionizing information retrieval by boosting search precision, deepening contextual understanding, and reshaping user experiences" + ] + } + }, + { + "_index": "retrievers_example", + "_id": "2", + "_score": 0.8333334, + "highlight": { + "text": [ + "Artificial intelligence is transforming medicine, from advancing diagnostics and tailoring treatment plans to empowering predictive patient care for improved" + ] + } + }, + { + "_index": "retrievers_example", + "_id": "3", + "_score": 0.25 + } + ] + } +} +---- +// TESTRESPONSE[s/"took": 42/"took": $body.took/] +============== + +[discrete] +[[retrievers-examples-inner-hits-retriever-results]] +==== Example: Computing inner hits from nested sub-retrievers + +We can also define `inner_hits` to be computed on any of the sub-retrievers, and propagate those computations to the top +level compound retriever. For example, let's create a new index with a `knn` field, nested under the `nested_field` field, +and index a couple of documents. + +[source,console] +---- +PUT retrievers_example_nested +{ + "mappings": { + "properties": { + "nested_field": { + "type": "nested", + "properties": { + "paragraph_id": { + "type": "keyword" + }, + "nested_vector": { + "type": "dense_vector", + "dims": 3, + "similarity": "l2_norm", + "index": true + } + } + }, + "topic": { + "type": "keyword" + } + } + } +} + +POST /retrievers_example_nested/_doc/1 +{ + "nested_field": [ + { + "paragraph_id": "1a", + "nested_vector": [ + -1.12, + -0.59, + 0.78 + ] + }, + { + "paragraph_id": "1b", + "nested_vector": [ + -0.12, + 1.56, + 0.42 + ] + }, + { + "paragraph_id": "1c", + "nested_vector": [ + 1, + -1, + 0 + ] + } + ], + "topic": [ + "ai" + ] +} + +POST /retrievers_example_nested/_doc/2 +{ + "nested_field": [ + { + "paragraph_id": "2a", + "nested_vector": [ + 0.23, + 1.24, + 0.65 + ] + } + ], + "topic": [ + "information_retrieval" + ] +} + +POST /retrievers_example_nested/_doc/3 +{ + "topic": [ + "ai" + ] +} + +POST /retrievers_example_nested/_refresh +---- +// TEST[continued] + +Now we can run an `rrf` retriever query and also compute <> for the `nested_field.nested_vector` +field, based on the `knn` query specified. + +[source,console] +---- +GET /retrievers_example_nested/_search +{ + "retriever": { + "rrf": { + "retrievers": [ { - "text_similarity_reranker": { - "retriever": { - "standard": { + "standard": { + "query": { + "nested": { + "path": "nested_field", + "inner_hits": { + "name": "nested_vector", + "_source": false, + "fields": [ + "nested_field.paragraph_id" + ] + }, "query": { - "term": { - "topic": "ai" + "knn": { + "field": "nested_field.nested_vector", + "query_vector": [ + 1, + 0, + 0.5 + ], + "k": 10 } } } - }, - "field": "text", - "inference_id": "my-rerank-model", - "inference_text": "Can I use generative AI to identify user intent and improve search relevance?" + } + } + }, + { + "standard": { + "query": { + "term": { + "topic": "ai" + } + } } } ], @@ -310,64 +595,184 @@ GET /retrievers_example/_search } }, "_source": [ - "text", "topic" ] } ---- -//NOTCONSOLE - -[discrete] -[[retrievers-examples-chaining-text-similarity-reranker-retrievers]] -==== Example: Chaining multiple semantic rerankers +// TEST[continued] -Full composability means we can chain together multiple retrievers of the same type. For instance, imagine we have a computationally expensive reranker that's specialized for AI content. We can rerank the results of a `text_similarity_reranker` using another `text_similarity_reranker` retriever. Each reranker can operate on different fields and/or use different inference services. +This would propagate the `inner_hits` defined for the `knn` query to the `rrf` retriever, and compute inner hits for `rrf`'s top results. -[source,js] +.Example response +[%collapsible] +============== +[source,console-result] ---- -GET retrievers_example/_search { - "retriever": { - "text_similarity_reranker": { - "retriever": { - "text_similarity_reranker": { - "retriever": { - "knn": { - "field": "vector", - "query_vector": [ - 0.23, - 0.67, - 0.89 - ], - "k": 3, - "num_candidates": 5 - } - }, - "rank_window_size": 100, - "field": "text", - "inference_id": "my-rerank-model", - "inference_text": "What are the state of the art applications of AI in information retrieval?" - } - }, - "rank_window_size": 10, - "field": "text", - "inference_id": "my-other-more-expensive-rerank-model", - "inference_text": "Applications of Large Language Models in technology and their impact on user satisfaction" - } - }, - "_source": [ - "text", - "topic" - ] + "took": 42, + "timed_out": false, + "_shards": { + "total": 1, + "successful": 1, + "skipped": 0, + "failed": 0 + }, + "hits": { + "total": { + "value": 3, + "relation": "eq" + }, + "max_score": 1.0, + "hits": [ + { + "_index": "retrievers_example_nested", + "_id": "1", + "_score": 1.0, + "_source": { + "topic": [ + "ai" + ] + }, + "inner_hits": { + "nested_vector": { + "hits": { + "total": { + "value": 3, + "relation": "eq" + }, + "max_score": 0.44353113, + "hits": [ + { + "_index": "retrievers_example_nested", + "_id": "1", + "_nested": { + "field": "nested_field", + "offset": 2 + }, + "_score": 0.44353113, + "fields": { + "nested_field": [ + { + "paragraph_id": [ + "1c" + ] + } + ] + } + }, + { + "_index": "retrievers_example_nested", + "_id": "1", + "_nested": { + "field": "nested_field", + "offset": 1 + }, + "_score": 0.26567122, + "fields": { + "nested_field": [ + { + "paragraph_id": [ + "1b" + ] + } + ] + } + }, + { + "_index": "retrievers_example_nested", + "_id": "1", + "_nested": { + "field": "nested_field", + "offset": 0 + }, + "_score": 0.18478848, + "fields": { + "nested_field": [ + { + "paragraph_id": [ + "1a" + ] + } + ] + } + } + ] + } + } + } + }, + { + "_index": "retrievers_example_nested", + "_id": "2", + "_score": 0.33333334, + "_source": { + "topic": [ + "information_retrieval" + ] + }, + "inner_hits": { + "nested_vector": { + "hits": { + "total": { + "value": 1, + "relation": "eq" + }, + "max_score": 0.32002488, + "hits": [ + { + "_index": "retrievers_example_nested", + "_id": "2", + "_nested": { + "field": "nested_field", + "offset": 0 + }, + "_score": 0.32002488, + "fields": { + "nested_field": [ + { + "paragraph_id": [ + "2a" + ] + } + ] + } + } + ] + } + } + } + }, + { + "_index": "retrievers_example_nested", + "_id": "3", + "_score": 0.33333334, + "_source": { + "topic": [ + "ai" + ] + }, + "inner_hits": { + "nested_vector": { + "hits": { + "total": { + "value": 0, + "relation": "eq" + }, + "max_score": null, + "hits": [] + } + } + } + } + ] + } } ---- -//NOTCONSOLE +// TESTRESPONSE[s/"took": 42/"took": $body.took/] +============== - -Note that our example applies two reranking steps. First, we rerank the top 100 -documents from the `knn` search using the `my-rerank-model` reranker. Then we -pick the top 10 results and rerank them using the more fine-grained -`my-other-more-expensive-rerank-model`. +Note: if using more than one `inner_hits` we need to provide custom names for each `inner_hits` so that they +are unique across all retrievers within the request. [discrete] [[retrievers-examples-rrf-and-aggregations]] @@ -380,7 +785,7 @@ the `terms` aggregation for the `topic` field will include all results, not just from the 2 nested retrievers, i.e. all documents whose `year` field is greater than 2023, and whose `topic` field matches the term `elastic`. -[source,js] +[source,console] ---- GET retrievers_example/_search { @@ -412,10 +817,7 @@ GET retrievers_example/_search "rank_constant": 1 } }, - "_source": [ - "text", - "topic" - ], + "_source": false, "aggs": { "topics": { "terms": { @@ -425,4 +827,436 @@ GET retrievers_example/_search } } ---- -//NOTCONSOLE +// TEST[continued] + +.Example response +[%collapsible] +============== +[source, console-result] +---- +{ + "took": 42, + "timed_out": false, + "_shards": { + "total": 1, + "successful": 1, + "skipped": 0, + "failed": 0 + }, + "hits": { + "total": { + "value": 4, + "relation": "eq" + }, + "max_score": 0.5833334, + "hits": [ + { + "_index": "retrievers_example", + "_id": "5", + "_score": 0.5833334 + }, + { + "_index": "retrievers_example", + "_id": "1", + "_score": 0.5 + }, + { + "_index": "retrievers_example", + "_id": "4", + "_score": 0.5 + }, + { + "_index": "retrievers_example", + "_id": "3", + "_score": 0.33333334 + } + ] + }, + "aggregations": { + "topics": { + "doc_count_error_upper_bound": 0, + "sum_other_doc_count": 0, + "buckets": [ + { + "key": "ai", + "doc_count": 3 + }, + { + "key": "elastic", + "doc_count": 2 + }, + { + "key": "assistant", + "doc_count": 1 + }, + { + "key": "documentation", + "doc_count": 1 + }, + { + "key": "information_retrieval", + "doc_count": 1 + }, + { + "key": "llm", + "doc_count": 1 + }, + { + "key": "observability", + "doc_count": 1 + }, + { + "key": "security", + "doc_count": 1 + } + ] + } + } +} +---- +// TESTRESPONSE[s/"took": 42/"took": $body.took/] +============== + +[discrete] +[[retrievers-examples-explain-multiple-rrf]] +==== Example: Explainability with multiple retrievers + +By adding `explain: true` to the request, each retriever will now provide a detailed explanation of all the steps +and calculations required to compute the final score. Composability is fully supported in the context of `explain`, and +each retriever will provide its own explanation, as shown in the example below. + +[source,console] +---- +GET /retrievers_example/_search +{ + "retriever": { + "rrf": { + "retrievers": [ + { + "standard": { + "query": { + "term": { + "topic": "elastic" + } + } + } + }, + { + "rrf": { + "retrievers": [ + { + "standard": { + "query": { + "query_string": { + "query": "(information retrieval) OR (artificial intelligence)", + "default_field": "text" + } + } + } + }, + { + "knn": { + "field": "vector", + "query_vector": [ + 0.23, + 0.67, + 0.89 + ], + "k": 3, + "num_candidates": 5 + } + } + ], + "rank_window_size": 10, + "rank_constant": 1 + } + } + ], + "rank_window_size": 10, + "rank_constant": 1 + } + }, + "_source": false, + "size": 1, + "explain": true +} +---- +// TEST[continued] + +The output of which, albeit a bit verbose, will provide all the necessary info to assist in debugging and reason with ranking. + +.Example response +[%collapsible] +============== +[source, console-result] +---- +{ + "took": 42, + "timed_out": false, + "_shards": { + "total": 1, + "successful": 1, + "skipped": 0, + "failed": 0 + }, + "hits": { + "total": { + "value": 5, + "relation": "eq" + }, + "max_score": 0.5, + "hits": [ + { + "_shard": "[retrievers_example][0]", + "_node": "jnrdZFKS3abUgWVsVdj2Vg", + "_index": "retrievers_example", + "_id": "1", + "_score": 0.5, + "_explanation": { + "value": 0.5, + "description": "rrf score: [0.5] computed for initial ranks [0, 1] with rankConstant: [1] as sum of [1 / (rank + rankConstant)] for each query", + "details": [ + { + "value": 0.0, + "description": "rrf score: [0], result not found in query at index [0]", + "details": [] + }, + { + "value": 1, + "description": "rrf score: [0.5], for rank [1] in query at index [1] computed as [1 / (1 + 1)], for matching query with score", + "details": [ + { + "value": 0.8333334, + "description": "rrf score: [0.8333334] computed for initial ranks [2, 1] with rankConstant: [1] as sum of [1 / (rank + rankConstant)] for each query", + "details": [ + { + "value": 2, + "description": "rrf score: [0.33333334], for rank [2] in query at index [0] computed as [1 / (2 + 1)], for matching query with score", + "details": [ + { + "value": 2.8129659, + "description": "sum of:", + "details": [ + { + "value": 1.4064829, + "description": "weight(text:information in 0) [PerFieldSimilarity], result of:", + "details": [ + *** + ] + }, + { + "value": 1.4064829, + "description": "weight(text:retrieval in 0) [PerFieldSimilarity], result of:", + "details": [ + *** + ] + } + ] + } + ] + }, + { + "value": 1, + "description": "rrf score: [0.5], for rank [1] in query at index [1] computed as [1 / (1 + 1)], for matching query with score", + "details": [ + { + "value": 1, + "description": "doc [0] with an original score of [1.0] is at rank [1] from the following source queries.", + "details": [ + { + "value": 1.0, + "description": "found vector with calculated similarity: 1.0", + "details": [] + } + ] + } + ] + } + ] + } + ] + } + ] + } + } + ] + } +} +---- +// TESTRESPONSE[s/"took": 42/"took": $body.took/] +// TESTRESPONSE[s/\.\.\./$body.hits.hits.0._explanation.details.1.details.0.details.0.details.0.details.0.details.0/] +// TESTRESPONSE[s/\*\*\*/$body.hits.hits.0._explanation.details.1.details.0.details.0.details.0.details.1.details.0/] +// TESTRESPONSE[s/jnrdZFKS3abUgWVsVdj2Vg/$body.hits.hits.0._node/] +============== + +[discrete] +[[retrievers-examples-text-similarity-reranker-on-top-of-rrf]] +==== Example: Rerank results of an RRF retriever + +To demonstrate the full functionality of retrievers, the following examples also require access to a <> set up using the <>. + +In this example we'll set up a reranking service and use it with the `text_similarity_reranker` retriever to rerank our top results. + +[source,console] +---- +PUT _inference/rerank/my-rerank-model +{ + "service": "cohere", + "service_settings": { + "model_id": "rerank-english-v3.0", + "api_key": "{{COHERE_API_KEY}}" + } +} +---- +// TEST[skip: no_access_to_ml] + +Let's start by reranking the results of the `rrf` retriever in our previous example. + +[source,console] +---- +GET retrievers_example/_search +{ + "retriever": { + "text_similarity_reranker": { + "retriever": { + "rrf": { + "retrievers": [ + { + "standard": { + "query": { + "query_string": { + "query": "(information retrieval) OR (artificial intelligence)", + "default_field": "text" + } + } + } + }, + { + "knn": { + "field": "vector", + "query_vector": [ + 0.23, + 0.67, + 0.89 + ], + "k": 3, + "num_candidates": 5 + } + } + ], + "rank_window_size": 10, + "rank_constant": 1 + } + }, + "field": "text", + "inference_id": "my-rerank-model", + "inference_text": "What are the state of the art applications of AI in information retrieval?" + } + }, + "_source": false +} + +---- +// TEST[skip: no_access_to_ml] + +[discrete] +[[retrievers-examples-rrf-ranking-on-text-similarity-reranker-results]] +==== Example: RRF with semantic reranker + +For this example, we'll replace the rrf's `standard` retriever with the `text_similarity_reranker` retriever, using the +`my-rerank-model` reranker we previously configured. Since this is a reranker, it needs an initial pool of +documents to work with. In this case, we'll rerank the top `rank_window_size` documents matching the `ai` topic. + +[source,console] +---- +GET /retrievers_example/_search +{ + "retriever": { + "rrf": { + "retrievers": [ + { + "knn": { + "field": "vector", + "query_vector": [ + 0.23, + 0.67, + 0.89 + ], + "k": 3, + "num_candidates": 5 + } + }, + { + "text_similarity_reranker": { + "retriever": { + "standard": { + "query": { + "term": { + "topic": "ai" + } + } + } + }, + "field": "text", + "inference_id": "my-rerank-model", + "inference_text": "Can I use generative AI to identify user intent and improve search relevance?" + } + } + ], + "rank_window_size": 10, + "rank_constant": 1 + } + }, + "_source": false +} +---- +// TEST[skip: no_access_to_ml] + +[discrete] +[[retrievers-examples-chaining-text-similarity-reranker-retrievers]] +==== Example: Chaining multiple semantic rerankers + +Full composability means we can chain together multiple retrievers of the same type. For instance, +imagine we have a computationally expensive reranker that's specialized for AI content. We can rerank the results of a `text_similarity_reranker` using another `text_similarity_reranker` retriever. Each reranker can operate on different fields and/or use different inference services. + +[source,console] +---- +GET retrievers_example/_search +{ + "retriever": { + "text_similarity_reranker": { + "retriever": { + "text_similarity_reranker": { + "retriever": { + "knn": { + "field": "vector", + "query_vector": [ + 0.23, + 0.67, + 0.89 + ], + "k": 3, + "num_candidates": 5 + } + }, + "rank_window_size": 100, + "field": "text", + "inference_id": "my-rerank-model", + "inference_text": "What are the state of the art applications of AI in information retrieval?" + } + }, + "rank_window_size": 10, + "field": "text", + "inference_id": "my-other-more-expensive-rerank-model", + "inference_text": "Applications of Large Language Models in technology and their impact on user satisfaction" + } + }, + "_source": false +} +---- +// TEST[skip: no_access_to_ml] + +Note that our example applies two reranking steps. First, we rerank the top 100 +documents from the `knn` search using the `my-rerank-model` reranker. Then we +pick the top 10 results and rerank them using the more fine-grained +`my-other-more-expensive-rerank-model`. From 6417e0912f2876c00f4e3b970af84875f23cd943 Mon Sep 17 00:00:00 2001 From: Dimitris Rempapis Date: Fri, 29 Nov 2024 14:53:20 +0200 Subject: [PATCH 21/27] CrossClusterIT testCancel failure (#117750) Investigate and fix test failure --- docs/changelog/117750.yaml | 6 ++++++ .../java/org/elasticsearch/search/ccs/CrossClusterIT.java | 4 ++-- 2 files changed, 8 insertions(+), 2 deletions(-) create mode 100644 docs/changelog/117750.yaml diff --git a/docs/changelog/117750.yaml b/docs/changelog/117750.yaml new file mode 100644 index 0000000000000..3ba3f1693f4df --- /dev/null +++ b/docs/changelog/117750.yaml @@ -0,0 +1,6 @@ +pr: 117750 +summary: '`CrossClusterIT` `testCancel` failure' +area: Search +type: bug +issues: + - 108061 diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java index 5d2d5c917415a..cb4d0681cdb23 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java @@ -63,6 +63,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -189,7 +190,6 @@ public void testProxyConnectionDisconnect() throws Exception { } } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/108061") public void testCancel() throws Exception { assertAcked(client(LOCAL_CLUSTER).admin().indices().prepareCreate("demo")); indexDocs(client(LOCAL_CLUSTER), "demo"); @@ -307,7 +307,7 @@ public void testCancel() throws Exception { } }); - RuntimeException e = expectThrows(RuntimeException.class, () -> queryFuture.result()); + ExecutionException e = expectThrows(ExecutionException.class, () -> queryFuture.result()); assertNotNull(e); assertNotNull(e.getCause()); Throwable t = ExceptionsHelper.unwrap(e, TaskCancelledException.class); From e19f2b7fbb908228a9b53821e275b8ccb58e7029 Mon Sep 17 00:00:00 2001 From: Quentin Pradet Date: Fri, 29 Nov 2024 17:22:37 +0400 Subject: [PATCH 22/27] Remove unsupported async_search parameters from rest-api-spec (#117626) --- .../rest-api-spec/api/async_search.submit.json | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/async_search.submit.json b/rest-api-spec/src/main/resources/rest-api-spec/api/async_search.submit.json index a7a7ebe838eab..3de0dec85f547 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/async_search.submit.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/async_search.submit.json @@ -43,11 +43,6 @@ "description":"Control whether the response should be stored in the cluster if it completed within the provided [wait_for_completion] time (default: false)", "default":false }, - "keep_alive": { - "type": "time", - "description": "Update the time interval in which the results (partial or final) for this search will be available", - "default": "5d" - }, "batched_reduce_size":{ "type":"number", "description":"The number of shard results that should be reduced at once on the coordinating node. This value should be used as the granularity at which progress results will be made available.", @@ -131,11 +126,6 @@ "type":"string", "description":"Specify the node or shard the operation should be performed on (default: random)" }, - "pre_filter_shard_size":{ - "type":"number", - "default": 1, - "description":"Cannot be changed: this is to enforce the execution of a pre-filter roundtrip to retrieve statistics from each shard so that the ones that surely don’t hold any document matching the query get skipped." - }, "rest_total_hits_as_int":{ "type":"boolean", "description":"Indicates whether hits.total should be rendered as an integer or an object in the rest search response", From 60ce74a7870a9e050ddac64900c3b35682e8e355 Mon Sep 17 00:00:00 2001 From: Tommaso Teofili Date: Fri, 29 Nov 2024 15:38:12 +0100 Subject: [PATCH 23/27] mute csv test for scoring in esql for mixed cluster (#117767) --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 96631d15f374f..f5f6b84ab8639 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -222,6 +222,9 @@ tests: - class: "org.elasticsearch.xpack.esql.qa.single_node.EsqlSpecIT" method: "test {scoring.*}" issue: https://github.com/elastic/elasticsearch/issues/117641 +- class: "org.elasticsearch.xpack.esql.qa.mixed.MixedClusterEsqlSpecIT" + method: "test {scoring.*}" + issue: https://github.com/elastic/elasticsearch/issues/117641 - class: org.elasticsearch.xpack.inference.InferenceCrudIT method: testSupportedStream issue: https://github.com/elastic/elasticsearch/issues/117745 From 5f045c05811ffd30f480d08403e3139c9686d97b Mon Sep 17 00:00:00 2001 From: Jan Kuipers <148754765+jan-elastic@users.noreply.github.com> Date: Fri, 29 Nov 2024 16:20:39 +0100 Subject: [PATCH 24/27] One Categorize BlockHash (#117723) * Move all categorize blockhash code to one "CategorizeBlockHash". * close resources in case of failure --- .../AbstractCategorizeBlockHash.java | 132 -------- .../aggregation/blockhash/BlockHash.java | 4 +- .../blockhash/CategorizeBlockHash.java | 309 ++++++++++++++++++ .../blockhash/CategorizeRawBlockHash.java | 147 --------- .../CategorizedIntermediateBlockHash.java | 92 ------ .../blockhash/CategorizeBlockHashTests.java | 8 +- .../function/grouping/Categorize.java | 6 +- 7 files changed, 315 insertions(+), 383 deletions(-) delete mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractCategorizeBlockHash.java create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHash.java delete mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java delete mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizedIntermediateBlockHash.java diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractCategorizeBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractCategorizeBlockHash.java deleted file mode 100644 index 0e89d77820883..0000000000000 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/AbstractCategorizeBlockHash.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.compute.aggregation.blockhash; - -import org.apache.lucene.util.BytesRefBuilder; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.BigArrays; -import org.elasticsearch.common.util.BitArray; -import org.elasticsearch.common.util.BytesRefHash; -import org.elasticsearch.compute.aggregation.SeenGroupIds; -import org.elasticsearch.compute.data.Block; -import org.elasticsearch.compute.data.BlockFactory; -import org.elasticsearch.compute.data.BytesRefBlock; -import org.elasticsearch.compute.data.BytesRefVector; -import org.elasticsearch.compute.data.IntBlock; -import org.elasticsearch.compute.data.IntVector; -import org.elasticsearch.compute.data.Page; -import org.elasticsearch.core.ReleasableIterator; -import org.elasticsearch.xpack.ml.aggs.categorization.CategorizationBytesRefHash; -import org.elasticsearch.xpack.ml.aggs.categorization.CategorizationPartOfSpeechDictionary; -import org.elasticsearch.xpack.ml.aggs.categorization.SerializableTokenListCategory; -import org.elasticsearch.xpack.ml.aggs.categorization.TokenListCategorizer; - -import java.io.IOException; - -/** - * Base BlockHash implementation for {@code Categorize} grouping function. - */ -public abstract class AbstractCategorizeBlockHash extends BlockHash { - protected static final int NULL_ORD = 0; - - // TODO: this should probably also take an emitBatchSize - private final int channel; - private final boolean outputPartial; - protected final TokenListCategorizer.CloseableTokenListCategorizer categorizer; - - /** - * Store whether we've seen any {@code null} values. - *

- * Null gets the {@link #NULL_ORD} ord. - *

- */ - protected boolean seenNull = false; - - AbstractCategorizeBlockHash(BlockFactory blockFactory, int channel, boolean outputPartial) { - super(blockFactory); - this.channel = channel; - this.outputPartial = outputPartial; - this.categorizer = new TokenListCategorizer.CloseableTokenListCategorizer( - new CategorizationBytesRefHash(new BytesRefHash(2048, blockFactory.bigArrays())), - CategorizationPartOfSpeechDictionary.getInstance(), - 0.70f - ); - } - - protected int channel() { - return channel; - } - - @Override - public Block[] getKeys() { - return new Block[] { outputPartial ? buildIntermediateBlock() : buildFinalBlock() }; - } - - @Override - public IntVector nonEmpty() { - return IntVector.range(seenNull ? 0 : 1, categorizer.getCategoryCount() + 1, blockFactory); - } - - @Override - public BitArray seenGroupIds(BigArrays bigArrays) { - return new SeenGroupIds.Range(seenNull ? 0 : 1, Math.toIntExact(categorizer.getCategoryCount() + 1)).seenGroupIds(bigArrays); - } - - @Override - public final ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { - throw new UnsupportedOperationException(); - } - - /** - * Serializes the intermediate state into a single BytesRef block, or an empty Null block if there are no categories. - */ - private Block buildIntermediateBlock() { - if (categorizer.getCategoryCount() == 0) { - return blockFactory.newConstantNullBlock(seenNull ? 1 : 0); - } - try (BytesStreamOutput out = new BytesStreamOutput()) { - // TODO be more careful here. - out.writeBoolean(seenNull); - out.writeVInt(categorizer.getCategoryCount()); - for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { - category.writeTo(out); - } - // We're returning a block with N positions just because the Page must have all blocks with the same position count! - int positionCount = categorizer.getCategoryCount() + (seenNull ? 1 : 0); - return blockFactory.newConstantBytesRefBlockWith(out.bytes().toBytesRef(), positionCount); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - private Block buildFinalBlock() { - BytesRefBuilder scratch = new BytesRefBuilder(); - - if (seenNull) { - try (BytesRefBlock.Builder result = blockFactory.newBytesRefBlockBuilder(categorizer.getCategoryCount())) { - result.appendNull(); - for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { - scratch.copyChars(category.getRegex()); - result.appendBytesRef(scratch.get()); - scratch.clear(); - } - return result.build(); - } - } - - try (BytesRefVector.Builder result = blockFactory.newBytesRefVectorBuilder(categorizer.getCategoryCount())) { - for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { - scratch.copyChars(category.getRegex()); - result.appendBytesRef(scratch.get()); - scratch.clear(); - } - return result.build().asBlock(); - } - } -} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java index ea76c3bd0a0aa..30afa7ae3128d 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/BlockHash.java @@ -180,9 +180,7 @@ public static BlockHash buildCategorizeBlockHash( throw new IllegalArgumentException("only a single CATEGORIZE group can used"); } - return aggregatorMode.isInputPartial() - ? new CategorizedIntermediateBlockHash(groups.get(0).channel, blockFactory, aggregatorMode.isOutputPartial()) - : new CategorizeRawBlockHash(groups.get(0).channel, blockFactory, aggregatorMode.isOutputPartial(), analysisRegistry); + return new CategorizeBlockHash(blockFactory, groups.get(0).channel, aggregatorMode, analysisRegistry); } /** diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHash.java new file mode 100644 index 0000000000000..35c6faf84e623 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHash.java @@ -0,0 +1,309 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute.aggregation.blockhash; + +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.BitArray; +import org.elasticsearch.common.util.BytesRefHash; +import org.elasticsearch.compute.aggregation.AggregatorMode; +import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; +import org.elasticsearch.compute.aggregation.SeenGroupIds; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.BytesRefBlock; +import org.elasticsearch.compute.data.BytesRefVector; +import org.elasticsearch.compute.data.IntBlock; +import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.ReleasableIterator; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.analysis.AnalysisRegistry; +import org.elasticsearch.xpack.core.ml.job.config.CategorizationAnalyzerConfig; +import org.elasticsearch.xpack.ml.aggs.categorization.CategorizationBytesRefHash; +import org.elasticsearch.xpack.ml.aggs.categorization.CategorizationPartOfSpeechDictionary; +import org.elasticsearch.xpack.ml.aggs.categorization.SerializableTokenListCategory; +import org.elasticsearch.xpack.ml.aggs.categorization.TokenListCategorizer; +import org.elasticsearch.xpack.ml.job.categorization.CategorizationAnalyzer; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * Base BlockHash implementation for {@code Categorize} grouping function. + */ +public class CategorizeBlockHash extends BlockHash { + + private static final CategorizationAnalyzerConfig ANALYZER_CONFIG = CategorizationAnalyzerConfig.buildStandardCategorizationAnalyzer( + List.of() + ); + private static final int NULL_ORD = 0; + + // TODO: this should probably also take an emitBatchSize + private final int channel; + private final AggregatorMode aggregatorMode; + private final TokenListCategorizer.CloseableTokenListCategorizer categorizer; + + private final CategorizeEvaluator evaluator; + + /** + * Store whether we've seen any {@code null} values. + *

+ * Null gets the {@link #NULL_ORD} ord. + *

+ */ + private boolean seenNull = false; + + CategorizeBlockHash(BlockFactory blockFactory, int channel, AggregatorMode aggregatorMode, AnalysisRegistry analysisRegistry) { + super(blockFactory); + + this.channel = channel; + this.aggregatorMode = aggregatorMode; + + this.categorizer = new TokenListCategorizer.CloseableTokenListCategorizer( + new CategorizationBytesRefHash(new BytesRefHash(2048, blockFactory.bigArrays())), + CategorizationPartOfSpeechDictionary.getInstance(), + 0.70f + ); + + if (aggregatorMode.isInputPartial() == false) { + CategorizationAnalyzer analyzer; + try { + Objects.requireNonNull(analysisRegistry); + analyzer = new CategorizationAnalyzer(analysisRegistry, ANALYZER_CONFIG); + } catch (Exception e) { + categorizer.close(); + throw new RuntimeException(e); + } + this.evaluator = new CategorizeEvaluator(analyzer); + } else { + this.evaluator = null; + } + } + + @Override + public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { + if (aggregatorMode.isInputPartial() == false) { + addInitial(page, addInput); + } else { + addIntermediate(page, addInput); + } + } + + @Override + public Block[] getKeys() { + return new Block[] { aggregatorMode.isOutputPartial() ? buildIntermediateBlock() : buildFinalBlock() }; + } + + @Override + public IntVector nonEmpty() { + return IntVector.range(seenNull ? 0 : 1, categorizer.getCategoryCount() + 1, blockFactory); + } + + @Override + public BitArray seenGroupIds(BigArrays bigArrays) { + return new SeenGroupIds.Range(seenNull ? 0 : 1, Math.toIntExact(categorizer.getCategoryCount() + 1)).seenGroupIds(bigArrays); + } + + @Override + public final ReleasableIterator lookup(Page page, ByteSizeValue targetBlockSize) { + throw new UnsupportedOperationException(); + } + + @Override + public void close() { + Releasables.close(evaluator, categorizer); + } + + /** + * Adds initial (raw) input to the state. + */ + private void addInitial(Page page, GroupingAggregatorFunction.AddInput addInput) { + try (IntBlock result = (IntBlock) evaluator.eval(page.getBlock(channel))) { + addInput.add(0, result); + } + } + + /** + * Adds intermediate state to the state. + */ + private void addIntermediate(Page page, GroupingAggregatorFunction.AddInput addInput) { + if (page.getPositionCount() == 0) { + return; + } + BytesRefBlock categorizerState = page.getBlock(channel); + if (categorizerState.areAllValuesNull()) { + seenNull = true; + try (var newIds = blockFactory.newConstantIntVector(NULL_ORD, 1)) { + addInput.add(0, newIds); + } + return; + } + + Map idMap = readIntermediate(categorizerState.getBytesRef(0, new BytesRef())); + try (IntBlock.Builder newIdsBuilder = blockFactory.newIntBlockBuilder(idMap.size())) { + int fromId = idMap.containsKey(0) ? 0 : 1; + int toId = fromId + idMap.size(); + for (int i = fromId; i < toId; i++) { + newIdsBuilder.appendInt(idMap.get(i)); + } + try (IntBlock newIds = newIdsBuilder.build()) { + addInput.add(0, newIds); + } + } + } + + /** + * Read intermediate state from a block. + * + * @return a map from the old category id to the new one. The old ids go from 0 to {@code size - 1}. + */ + private Map readIntermediate(BytesRef bytes) { + Map idMap = new HashMap<>(); + try (StreamInput in = new BytesArray(bytes).streamInput()) { + if (in.readBoolean()) { + seenNull = true; + idMap.put(NULL_ORD, NULL_ORD); + } + int count = in.readVInt(); + for (int oldCategoryId = 0; oldCategoryId < count; oldCategoryId++) { + int newCategoryId = categorizer.mergeWireCategory(new SerializableTokenListCategory(in)).getId(); + // +1 because the 0 ordinal is reserved for null + idMap.put(oldCategoryId + 1, newCategoryId + 1); + } + return idMap; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Serializes the intermediate state into a single BytesRef block, or an empty Null block if there are no categories. + */ + private Block buildIntermediateBlock() { + if (categorizer.getCategoryCount() == 0) { + return blockFactory.newConstantNullBlock(seenNull ? 1 : 0); + } + try (BytesStreamOutput out = new BytesStreamOutput()) { + out.writeBoolean(seenNull); + out.writeVInt(categorizer.getCategoryCount()); + for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { + category.writeTo(out); + } + // We're returning a block with N positions just because the Page must have all blocks with the same position count! + int positionCount = categorizer.getCategoryCount() + (seenNull ? 1 : 0); + return blockFactory.newConstantBytesRefBlockWith(out.bytes().toBytesRef(), positionCount); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private Block buildFinalBlock() { + BytesRefBuilder scratch = new BytesRefBuilder(); + + if (seenNull) { + try (BytesRefBlock.Builder result = blockFactory.newBytesRefBlockBuilder(categorizer.getCategoryCount())) { + result.appendNull(); + for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { + scratch.copyChars(category.getRegex()); + result.appendBytesRef(scratch.get()); + scratch.clear(); + } + return result.build(); + } + } + + try (BytesRefVector.Builder result = blockFactory.newBytesRefVectorBuilder(categorizer.getCategoryCount())) { + for (SerializableTokenListCategory category : categorizer.toCategoriesById()) { + scratch.copyChars(category.getRegex()); + result.appendBytesRef(scratch.get()); + scratch.clear(); + } + return result.build().asBlock(); + } + } + + /** + * Similar implementation to an Evaluator. + */ + private final class CategorizeEvaluator implements Releasable { + private final CategorizationAnalyzer analyzer; + + CategorizeEvaluator(CategorizationAnalyzer analyzer) { + this.analyzer = analyzer; + } + + Block eval(BytesRefBlock vBlock) { + BytesRefVector vVector = vBlock.asVector(); + if (vVector == null) { + return eval(vBlock.getPositionCount(), vBlock); + } + IntVector vector = eval(vBlock.getPositionCount(), vVector); + return vector.asBlock(); + } + + IntBlock eval(int positionCount, BytesRefBlock vBlock) { + try (IntBlock.Builder result = blockFactory.newIntBlockBuilder(positionCount)) { + BytesRef vScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + if (vBlock.isNull(p)) { + seenNull = true; + result.appendInt(NULL_ORD); + continue; + } + int first = vBlock.getFirstValueIndex(p); + int count = vBlock.getValueCount(p); + if (count == 1) { + result.appendInt(process(vBlock.getBytesRef(first, vScratch))); + continue; + } + int end = first + count; + result.beginPositionEntry(); + for (int i = first; i < end; i++) { + result.appendInt(process(vBlock.getBytesRef(i, vScratch))); + } + result.endPositionEntry(); + } + return result.build(); + } + } + + IntVector eval(int positionCount, BytesRefVector vVector) { + try (IntVector.FixedBuilder result = blockFactory.newIntVectorFixedBuilder(positionCount)) { + BytesRef vScratch = new BytesRef(); + for (int p = 0; p < positionCount; p++) { + result.appendInt(p, process(vVector.getBytesRef(p, vScratch))); + } + return result.build(); + } + } + + int process(BytesRef v) { + var category = categorizer.computeCategory(v.utf8ToString(), analyzer); + if (category == null) { + seenNull = true; + return NULL_ORD; + } + return category.getId() + 1; + } + + @Override + public void close() { + analyzer.close(); + } + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java deleted file mode 100644 index 47dd7f650dffa..0000000000000 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeRawBlockHash.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.compute.aggregation.blockhash; - -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; -import org.elasticsearch.compute.data.Block; -import org.elasticsearch.compute.data.BlockFactory; -import org.elasticsearch.compute.data.BytesRefBlock; -import org.elasticsearch.compute.data.BytesRefVector; -import org.elasticsearch.compute.data.IntBlock; -import org.elasticsearch.compute.data.IntVector; -import org.elasticsearch.compute.data.Page; -import org.elasticsearch.core.Releasable; -import org.elasticsearch.core.Releasables; -import org.elasticsearch.index.analysis.AnalysisRegistry; -import org.elasticsearch.xpack.core.ml.job.config.CategorizationAnalyzerConfig; -import org.elasticsearch.xpack.ml.aggs.categorization.TokenListCategorizer; -import org.elasticsearch.xpack.ml.job.categorization.CategorizationAnalyzer; - -import java.io.IOException; -import java.util.List; - -/** - * BlockHash implementation for {@code Categorize} grouping function. - *

- * This implementation expects rows, and can't deserialize intermediate states coming from other nodes. - *

- */ -public class CategorizeRawBlockHash extends AbstractCategorizeBlockHash { - private static final CategorizationAnalyzerConfig ANALYZER_CONFIG = CategorizationAnalyzerConfig.buildStandardCategorizationAnalyzer( - List.of() - ); - - private final CategorizeEvaluator evaluator; - - CategorizeRawBlockHash(int channel, BlockFactory blockFactory, boolean outputPartial, AnalysisRegistry analysisRegistry) { - super(blockFactory, channel, outputPartial); - - CategorizationAnalyzer analyzer; - try { - analyzer = new CategorizationAnalyzer(analysisRegistry, ANALYZER_CONFIG); - } catch (IOException e) { - categorizer.close(); - throw new RuntimeException(e); - } - - this.evaluator = new CategorizeEvaluator(analyzer, categorizer, blockFactory); - } - - @Override - public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { - try (IntBlock result = (IntBlock) evaluator.eval(page.getBlock(channel()))) { - addInput.add(0, result); - } - } - - @Override - public void close() { - evaluator.close(); - } - - /** - * Similar implementation to an Evaluator. - */ - public final class CategorizeEvaluator implements Releasable { - private final CategorizationAnalyzer analyzer; - - private final TokenListCategorizer.CloseableTokenListCategorizer categorizer; - - private final BlockFactory blockFactory; - - public CategorizeEvaluator( - CategorizationAnalyzer analyzer, - TokenListCategorizer.CloseableTokenListCategorizer categorizer, - BlockFactory blockFactory - ) { - this.analyzer = analyzer; - this.categorizer = categorizer; - this.blockFactory = blockFactory; - } - - public Block eval(BytesRefBlock vBlock) { - BytesRefVector vVector = vBlock.asVector(); - if (vVector == null) { - return eval(vBlock.getPositionCount(), vBlock); - } - IntVector vector = eval(vBlock.getPositionCount(), vVector); - return vector.asBlock(); - } - - public IntBlock eval(int positionCount, BytesRefBlock vBlock) { - try (IntBlock.Builder result = blockFactory.newIntBlockBuilder(positionCount)) { - BytesRef vScratch = new BytesRef(); - for (int p = 0; p < positionCount; p++) { - if (vBlock.isNull(p)) { - seenNull = true; - result.appendInt(NULL_ORD); - continue; - } - int first = vBlock.getFirstValueIndex(p); - int count = vBlock.getValueCount(p); - if (count == 1) { - result.appendInt(process(vBlock.getBytesRef(first, vScratch))); - continue; - } - int end = first + count; - result.beginPositionEntry(); - for (int i = first; i < end; i++) { - result.appendInt(process(vBlock.getBytesRef(i, vScratch))); - } - result.endPositionEntry(); - } - return result.build(); - } - } - - public IntVector eval(int positionCount, BytesRefVector vVector) { - try (IntVector.FixedBuilder result = blockFactory.newIntVectorFixedBuilder(positionCount)) { - BytesRef vScratch = new BytesRef(); - for (int p = 0; p < positionCount; p++) { - result.appendInt(p, process(vVector.getBytesRef(p, vScratch))); - } - return result.build(); - } - } - - private int process(BytesRef v) { - var category = categorizer.computeCategory(v.utf8ToString(), analyzer); - if (category == null) { - seenNull = true; - return NULL_ORD; - } - return category.getId() + 1; - } - - @Override - public void close() { - Releasables.closeExpectNoException(analyzer, categorizer); - } - } -} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizedIntermediateBlockHash.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizedIntermediateBlockHash.java deleted file mode 100644 index c774d3b26049d..0000000000000 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/aggregation/blockhash/CategorizedIntermediateBlockHash.java +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.compute.aggregation.blockhash; - -import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.compute.aggregation.GroupingAggregatorFunction; -import org.elasticsearch.compute.data.BlockFactory; -import org.elasticsearch.compute.data.BytesRefBlock; -import org.elasticsearch.compute.data.IntBlock; -import org.elasticsearch.compute.data.Page; -import org.elasticsearch.xpack.ml.aggs.categorization.SerializableTokenListCategory; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -/** - * BlockHash implementation for {@code Categorize} grouping function. - *

- * This implementation expects a single intermediate state in a block, as generated by {@link AbstractCategorizeBlockHash}. - *

- */ -public class CategorizedIntermediateBlockHash extends AbstractCategorizeBlockHash { - - CategorizedIntermediateBlockHash(int channel, BlockFactory blockFactory, boolean outputPartial) { - super(blockFactory, channel, outputPartial); - } - - @Override - public void add(Page page, GroupingAggregatorFunction.AddInput addInput) { - if (page.getPositionCount() == 0) { - // No categories - return; - } - BytesRefBlock categorizerState = page.getBlock(channel()); - if (categorizerState.areAllValuesNull()) { - seenNull = true; - try (var newIds = blockFactory.newConstantIntVector(NULL_ORD, 1)) { - addInput.add(0, newIds); - } - return; - } - - Map idMap = readIntermediate(categorizerState.getBytesRef(0, new BytesRef())); - try (IntBlock.Builder newIdsBuilder = blockFactory.newIntBlockBuilder(idMap.size())) { - int fromId = idMap.containsKey(0) ? 0 : 1; - int toId = fromId + idMap.size(); - for (int i = fromId; i < toId; i++) { - newIdsBuilder.appendInt(idMap.get(i)); - } - try (IntBlock newIds = newIdsBuilder.build()) { - addInput.add(0, newIds); - } - } - } - - /** - * Read intermediate state from a block. - * - * @return a map from the old category id to the new one. The old ids go from 0 to {@code size - 1}. - */ - private Map readIntermediate(BytesRef bytes) { - Map idMap = new HashMap<>(); - try (StreamInput in = new BytesArray(bytes).streamInput()) { - if (in.readBoolean()) { - seenNull = true; - idMap.put(NULL_ORD, NULL_ORD); - } - int count = in.readVInt(); - for (int oldCategoryId = 0; oldCategoryId < count; oldCategoryId++) { - int newCategoryId = categorizer.mergeWireCategory(new SerializableTokenListCategory(in)).getId(); - // +1 because the 0 ordinal is reserved for null - idMap.put(oldCategoryId + 1, newCategoryId + 1); - } - return idMap; - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void close() { - categorizer.close(); - } -} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java index 8a3c723557151..3c47e85a4a9c8 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/blockhash/CategorizeBlockHashTests.java @@ -95,7 +95,7 @@ public void testCategorizeRaw() { page = new Page(builder.build()); } - try (BlockHash hash = new CategorizeRawBlockHash(0, blockFactory, true, analysisRegistry)) { + try (BlockHash hash = new CategorizeBlockHash(blockFactory, 0, AggregatorMode.INITIAL, analysisRegistry)) { hash.add(page, new GroupingAggregatorFunction.AddInput() { @Override public void add(int positionOffset, IntBlock groupIds) { @@ -168,8 +168,8 @@ public void testCategorizeIntermediate() { // Fill intermediatePages with the intermediate state from the raw hashes try ( - BlockHash rawHash1 = new CategorizeRawBlockHash(0, blockFactory, true, analysisRegistry); - BlockHash rawHash2 = new CategorizeRawBlockHash(0, blockFactory, true, analysisRegistry); + BlockHash rawHash1 = new CategorizeBlockHash(blockFactory, 0, AggregatorMode.INITIAL, analysisRegistry); + BlockHash rawHash2 = new CategorizeBlockHash(blockFactory, 0, AggregatorMode.INITIAL, analysisRegistry); ) { rawHash1.add(page1, new GroupingAggregatorFunction.AddInput() { @Override @@ -226,7 +226,7 @@ public void close() { page2.releaseBlocks(); } - try (BlockHash intermediateHash = new CategorizedIntermediateBlockHash(0, blockFactory, true)) { + try (BlockHash intermediateHash = new CategorizeBlockHash(blockFactory, 0, AggregatorMode.INTERMEDIATE, null)) { intermediateHash.add(intermediatePage1, new GroupingAggregatorFunction.AddInput() { @Override public void add(int positionOffset, IntBlock groupIds) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/grouping/Categorize.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/grouping/Categorize.java index 31b603ecef889..63b5073c2217a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/grouping/Categorize.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/grouping/Categorize.java @@ -32,12 +32,8 @@ * This function has no evaluators, as it works like an aggregation (Accumulates values, stores intermediate states, etc). *

*

- * For the implementation, see: + * For the implementation, see {@link org.elasticsearch.compute.aggregation.blockhash.CategorizeBlockHash} *

- *
    - *
  • {@link org.elasticsearch.compute.aggregation.blockhash.CategorizedIntermediateBlockHash}
  • - *
  • {@link org.elasticsearch.compute.aggregation.blockhash.CategorizeRawBlockHash}
  • - *
*/ public class Categorize extends GroupingFunction implements Validatable { public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( From 64107e0a0b032c0ee1ed319f0d6bfefce23def9a Mon Sep 17 00:00:00 2001 From: Alexander Spies Date: Fri, 29 Nov 2024 17:34:05 +0100 Subject: [PATCH 25/27] Compute output of LookupJoinExec dynamically (#117763) LookupJoinExec should not assume its output but instead compute it from - Its input fields from the left - The fields added from the lookup index Currently, LookupJoinExec's output is determined when the logical plan is mapped to a physical one, and thereafter the output cannot be changed anymore. This makes it impossible to have late materialization of fields from the left hand side via field extractions, because we are forced to extract *all* fields before the LookupJoinExec, otherwise we do not achieve the prescribed output. Avoid that by tracking only which fields the LookupJoinExec will add from the lookup index instead of tracking the whole output (that was only correct for the logical plan). **Note:** While this PR is a refactoring for the current functionality, it should unblock @craigtaverner 's ongoing work related to field extractions and getting multiple LOOKUP JOIN queries to work correctly without adding hacks. --- .../xpack/esql/ccq/MultiClusterSpecIT.java | 4 +- .../src/main/resources/lookup-join.csv-spec | 10 +-- .../xpack/esql/action/EsqlCapabilities.java | 2 +- .../optimizer/PhysicalOptimizerRules.java | 32 --------- .../physical/local/InsertFieldExtraction.java | 4 +- .../xpack/esql/plan/logical/join/Join.java | 31 +++++---- .../esql/plan/physical/LookupJoinExec.java | 65 ++++++++----------- .../esql/planner/LocalExecutionPlanner.java | 6 +- .../esql/planner/mapper/LocalMapper.java | 10 +-- .../xpack/esql/planner/mapper/Mapper.java | 10 +-- .../elasticsearch/xpack/esql/CsvTests.java | 2 +- 11 files changed, 60 insertions(+), 116 deletions(-) diff --git a/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java b/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java index 8f4522573f880..af5eadc7358a2 100644 --- a/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java +++ b/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java @@ -47,7 +47,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.classpathResources; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.INLINESTATS; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.INLINESTATS_V2; -import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V2; +import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V3; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_PLANNING_V1; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.METADATA_FIELDS_REMOTE_TEST; import static org.elasticsearch.xpack.esql.qa.rest.EsqlSpecTestCase.Mode.SYNC; @@ -125,7 +125,7 @@ protected void shouldSkipTest(String testName) throws IOException { assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(INLINESTATS.capabilityName())); assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(INLINESTATS_V2.capabilityName())); assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_PLANNING_V1.capabilityName())); - assumeFalse("LOOKUP JOIN not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_LOOKUP_V2.capabilityName())); + assumeFalse("LOOKUP JOIN not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_LOOKUP_V3.capabilityName())); } private TestFeatureService remoteFeaturesService() throws IOException { diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec index 11786fb905c60..5de353978b307 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec @@ -5,7 +5,7 @@ //TODO: this sometimes returns null instead of the looked up value (likely related to the execution order) basicOnTheDataNode-Ignore -required_capability: join_lookup_v2 +required_capability: join_lookup_v3 FROM employees | EVAL language_code = languages @@ -22,7 +22,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; basicRow-Ignore -required_capability: join_lookup +required_capability: join_lookup_v3 ROW language_code = 1 | LOOKUP JOIN languages_lookup ON language_code @@ -33,7 +33,7 @@ language_code:keyword | language_name:keyword ; basicOnTheCoordinator -required_capability: join_lookup_v2 +required_capability: join_lookup_v3 FROM employees | SORT emp_no @@ -51,7 +51,7 @@ emp_no:integer | language_code:integer | language_name:keyword //TODO: this sometimes returns null instead of the looked up value (likely related to the execution order) subsequentEvalOnTheDataNode-Ignore -required_capability: join_lookup_v2 +required_capability: join_lookup_v3 FROM employees | EVAL language_code = languages @@ -69,7 +69,7 @@ emp_no:integer | language_code:integer | language_name:keyword | language_code_x ; subsequentEvalOnTheCoordinator -required_capability: join_lookup_v2 +required_capability: join_lookup_v3 FROM employees | SORT emp_no diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 373be23cdf847..dc3329a906741 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -521,7 +521,7 @@ public enum Cap { /** * LOOKUP JOIN */ - JOIN_LOOKUP_V2(Build.current().isSnapshot()), + JOIN_LOOKUP_V3(Build.current().isSnapshot()), /** * Fix for https://github.com/elastic/elasticsearch/issues/117054 diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java index 482a89b50c865..ee192c2420da8 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/PhysicalOptimizerRules.java @@ -7,8 +7,6 @@ package org.elasticsearch.xpack.esql.optimizer; -import org.elasticsearch.xpack.esql.core.expression.Expression; -import org.elasticsearch.xpack.esql.core.util.ReflectionUtils; import org.elasticsearch.xpack.esql.optimizer.rules.logical.OptimizerRules.TransformDirection; import org.elasticsearch.xpack.esql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.esql.rule.ParameterizedRule; @@ -62,34 +60,4 @@ public final PhysicalPlan apply(PhysicalPlan plan) { protected abstract PhysicalPlan rule(SubPlan plan); } - - public abstract static class OptimizerExpressionRule extends Rule { - - private final TransformDirection direction; - // overriding type token which returns the correct class but does an uncheck cast to LogicalPlan due to its generic bound - // a proper solution is to wrap the Expression rule into a Plan rule but that would affect the rule declaration - // so instead this is hacked here - private final Class expressionTypeToken = ReflectionUtils.detectSuperTypeForRuleLike(getClass()); - - public OptimizerExpressionRule(TransformDirection direction) { - this.direction = direction; - } - - @Override - public final PhysicalPlan apply(PhysicalPlan plan) { - return direction == TransformDirection.DOWN - ? plan.transformExpressionsDown(expressionTypeToken, this::rule) - : plan.transformExpressionsUp(expressionTypeToken, this::rule); - } - - protected PhysicalPlan rule(PhysicalPlan plan) { - return plan; - } - - protected abstract Expression rule(E e); - - public Class expressionToken() { - return expressionTypeToken; - } - } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java index 72573821dfeb8..cafe3726f92ac 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/InsertFieldExtraction.java @@ -104,15 +104,15 @@ private static Set missingAttributes(PhysicalPlan p) { var missing = new LinkedHashSet(); var inputSet = p.inputSet(); - // FIXME: the extractors should work on the right side as well + // TODO: We need to extract whatever fields are missing from the left hand side. // skip the lookup join since the right side is always materialized and a projection if (p instanceof LookupJoinExec join) { - // collect fields used in the join condition return Collections.emptySet(); } var input = inputSet; // collect field attributes used inside expressions + // TODO: Rather than going over all expressions manually, this should just call .references() p.forEachExpression(TypedAttribute.class, f -> { if (f instanceof FieldAttribute || f instanceof MetadataAttribute) { if (input.contains(f) == false) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/Join.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/Join.java index dd6b3ea3455f7..6af29fb23b3bb 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/Join.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/join/Join.java @@ -11,7 +11,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.xpack.esql.core.expression.Attribute; -import org.elasticsearch.xpack.esql.core.expression.NamedExpression; +import org.elasticsearch.xpack.esql.core.expression.AttributeSet; import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute; import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; @@ -23,12 +23,9 @@ import java.util.ArrayList; import java.util.List; import java.util.Objects; -import java.util.Set; -import java.util.stream.Collectors; import static org.elasticsearch.xpack.esql.expression.NamedExpressions.mergeOutputAttributes; import static org.elasticsearch.xpack.esql.plan.logical.join.JoinTypes.LEFT; -import static org.elasticsearch.xpack.esql.plan.logical.join.JoinTypes.RIGHT; public class Join extends BinaryPlan { public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(LogicalPlan.class, "Join", Join::new); @@ -100,6 +97,19 @@ public List output() { return lazyOutput; } + public List rightOutputFields() { + AttributeSet leftInputs = left().outputSet(); + + List rightOutputFields = new ArrayList<>(); + for (Attribute attr : output()) { + if (leftInputs.contains(attr) == false) { + rightOutputFields.add(attr); + } + } + + return rightOutputFields; + } + /** * Combine the two lists of attributes into one. * In case of (name) conflicts, specify which sides wins, that is overrides the other column - the left or the right. @@ -108,18 +118,11 @@ public static List computeOutput(List leftOutput, List output; // TODO: make the other side nullable - Set matchFieldNames = config.matchFields().stream().map(NamedExpression::name).collect(Collectors.toSet()); if (LEFT.equals(joinType)) { - // right side becomes nullable and overrides left except for match fields, which we preserve from the left - List rightOutputWithoutMatchFields = rightOutput.stream() - .filter(attr -> matchFieldNames.contains(attr.name()) == false) - .toList(); + // right side becomes nullable and overrides left except for join keys, which we preserve from the left + AttributeSet rightKeys = new AttributeSet(config.rightFields()); + List rightOutputWithoutMatchFields = rightOutput.stream().filter(attr -> rightKeys.contains(attr) == false).toList(); output = mergeOutputAttributes(rightOutputWithoutMatchFields, leftOutput); - } else if (RIGHT.equals(joinType)) { - List leftOutputWithoutMatchFields = leftOutput.stream() - .filter(attr -> matchFieldNames.contains(attr.name()) == false) - .toList(); - output = mergeOutputAttributes(leftOutputWithoutMatchFields, rightOutput); } else { throw new IllegalArgumentException(joinType.joinName() + " unsupported"); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExec.java index e01451ceaecac..2d3caa27da4cd 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExec.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/LookupJoinExec.java @@ -19,7 +19,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Comparator; import java.util.List; import java.util.Objects; @@ -30,43 +29,43 @@ public class LookupJoinExec extends BinaryExec implements EstimatesRowSize { LookupJoinExec::new ); - private final List matchFields; private final List leftFields; private final List rightFields; - private final List output; - private List lazyAddedFields; + /** + * These cannot be computed from the left + right outputs, because + * {@link org.elasticsearch.xpack.esql.optimizer.rules.physical.local.ReplaceSourceAttributes} will replace the {@link EsSourceExec} on + * the right hand side by a {@link EsQueryExec}, and thus lose the information of which fields we'll get from the lookup index. + */ + private final List addedFields; + private List lazyOutput; public LookupJoinExec( Source source, PhysicalPlan left, PhysicalPlan lookup, - List matchFields, List leftFields, List rightFields, - List output + List addedFields ) { super(source, left, lookup); - this.matchFields = matchFields; this.leftFields = leftFields; this.rightFields = rightFields; - this.output = output; + this.addedFields = addedFields; } private LookupJoinExec(StreamInput in) throws IOException { super(Source.readFrom((PlanStreamInput) in), in.readNamedWriteable(PhysicalPlan.class), in.readNamedWriteable(PhysicalPlan.class)); - this.matchFields = in.readNamedWriteableCollectionAsList(Attribute.class); this.leftFields = in.readNamedWriteableCollectionAsList(Attribute.class); this.rightFields = in.readNamedWriteableCollectionAsList(Attribute.class); - this.output = in.readNamedWriteableCollectionAsList(Attribute.class); + this.addedFields = in.readNamedWriteableCollectionAsList(Attribute.class); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeNamedWriteableCollection(matchFields); out.writeNamedWriteableCollection(leftFields); out.writeNamedWriteableCollection(rightFields); - out.writeNamedWriteableCollection(output); + out.writeNamedWriteableCollection(addedFields); } @Override @@ -78,10 +77,6 @@ public PhysicalPlan lookup() { return right(); } - public List matchFields() { - return matchFields; - } - public List leftFields() { return leftFields; } @@ -91,29 +86,26 @@ public List rightFields() { } public List addedFields() { - if (lazyAddedFields == null) { - AttributeSet set = outputSet(); - set.removeAll(left().output()); - for (Attribute m : matchFields) { - set.removeIf(a -> a.name().equals(m.name())); + return addedFields; + } + + @Override + public List output() { + if (lazyOutput == null) { + lazyOutput = new ArrayList<>(left().output()); + for (Attribute attr : addedFields) { + lazyOutput.add(attr); } - lazyAddedFields = new ArrayList<>(set); - lazyAddedFields.sort(Comparator.comparing(Attribute::name)); } - return lazyAddedFields; + return lazyOutput; } @Override public PhysicalPlan estimateRowSize(State state) { - state.add(false, output); + state.add(false, output()); return this; } - @Override - public List output() { - return output; - } - @Override public AttributeSet inputSet() { // TODO: this is a hack since the right side is always materialized - instead this should @@ -129,12 +121,12 @@ protected AttributeSet computeReferences() { @Override public LookupJoinExec replaceChildren(PhysicalPlan left, PhysicalPlan right) { - return new LookupJoinExec(source(), left, right, matchFields, leftFields, rightFields, output); + return new LookupJoinExec(source(), left, right, leftFields, rightFields, addedFields); } @Override protected NodeInfo info() { - return NodeInfo.create(this, LookupJoinExec::new, left(), right(), matchFields, leftFields, rightFields, output); + return NodeInfo.create(this, LookupJoinExec::new, left(), right(), leftFields, rightFields, addedFields); } @Override @@ -148,15 +140,12 @@ public boolean equals(Object o) { if (super.equals(o) == false) { return false; } - LookupJoinExec hash = (LookupJoinExec) o; - return matchFields.equals(hash.matchFields) - && leftFields.equals(hash.leftFields) - && rightFields.equals(hash.rightFields) - && output.equals(hash.output); + LookupJoinExec other = (LookupJoinExec) o; + return leftFields.equals(other.leftFields) && rightFields.equals(other.rightFields) && addedFields.equals(other.addedFields); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), matchFields, leftFields, rightFields, output); + return Objects.hash(super.hashCode(), leftFields, rightFields, addedFields); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java index 1ffc652e54337..a8afaa4d8119b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlanner.java @@ -583,8 +583,8 @@ private PhysicalOperation planLookupJoin(LookupJoinExec join, LocalExecutionPlan if (localSourceExec.indexMode() != IndexMode.LOOKUP) { throw new IllegalArgumentException("can't plan [" + join + "]"); } - List matchFields = new ArrayList<>(join.matchFields().size()); - for (Attribute m : join.matchFields()) { + List matchFields = new ArrayList<>(join.leftFields().size()); + for (Attribute m : join.leftFields()) { Layout.ChannelAndType t = source.layout.get(m.id()); if (t == null) { throw new IllegalArgumentException("can't plan [" + join + "][" + m + "]"); @@ -604,7 +604,7 @@ private PhysicalOperation planLookupJoin(LookupJoinExec join, LocalExecutionPlan lookupFromIndexService, matchFields.getFirst().type(), localSourceExec.index().name(), - join.matchFields().getFirst().name(), + join.leftFields().getFirst().name(), join.addedFields().stream().map(f -> (NamedExpression) f).toList(), join.source() ), diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/LocalMapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/LocalMapper.java index fc52f2d5a9d23..f95ae0e0783e5 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/LocalMapper.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/LocalMapper.java @@ -120,15 +120,7 @@ private PhysicalPlan mapBinary(BinaryPlan binary) { ); } if (right instanceof EsSourceExec source && source.indexMode() == IndexMode.LOOKUP) { - return new LookupJoinExec( - join.source(), - left, - right, - config.matchFields(), - config.leftFields(), - config.rightFields(), - join.output() - ); + return new LookupJoinExec(join.source(), left, right, config.leftFields(), config.rightFields(), join.rightOutputFields()); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/Mapper.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/Mapper.java index 23e6f4fb91d18..8a4325ed84b2a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/Mapper.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/mapper/Mapper.java @@ -207,15 +207,7 @@ private PhysicalPlan mapBinary(BinaryPlan bp) { if (right instanceof FragmentExec fragment && fragment.fragment() instanceof EsRelation relation && relation.indexMode() == IndexMode.LOOKUP) { - return new LookupJoinExec( - join.source(), - left, - right, - config.matchFields(), - config.leftFields(), - config.rightFields(), - join.output() - ); + return new LookupJoinExec(join.source(), left, right, config.leftFields(), config.rightFields(), join.rightOutputFields()); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java index 6763988eac638..df974a88a4c57 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java @@ -263,7 +263,7 @@ public final void test() throws Throwable { ); assumeFalse( "lookup join disabled for csv tests", - testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.JOIN_LOOKUP_V2.capabilityName()) + testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.JOIN_LOOKUP_V3.capabilityName()) ); if (Build.current().isSnapshot()) { assertThat( From 39481e912f10f9ce4ca85176b1bee9a9b97c43f6 Mon Sep 17 00:00:00 2001 From: Mikhail Berezovskiy Date: Fri, 29 Nov 2024 09:40:31 -0800 Subject: [PATCH 26/27] trash derived buffers (#117744) --- .../transport/netty4/NettyAllocator.java | 43 -- .../transport/netty4/TrashingByteBuf.java | 536 ++++++++++++++++++ .../transport/netty4/NettyAllocatorTests.java | 1 - 3 files changed, 536 insertions(+), 44 deletions(-) create mode 100644 modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/TrashingByteBuf.java diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyAllocator.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyAllocator.java index 1eb7e13889338..e8bd5514947d6 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyAllocator.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyAllocator.java @@ -362,49 +362,6 @@ public ByteBufAllocator getDelegate() { } } - static class TrashingByteBuf extends WrappedByteBuf { - - private boolean trashed = false; - - protected TrashingByteBuf(ByteBuf buf) { - super(buf); - } - - @Override - public boolean release() { - if (refCnt() == 1) { - // see [NOTE on racy trashContent() calls] - trashContent(); - } - return super.release(); - } - - @Override - public boolean release(int decrement) { - if (refCnt() == decrement && refCnt() > 0) { - // see [NOTE on racy trashContent() calls] - trashContent(); - } - return super.release(decrement); - } - - // [NOTE on racy trashContent() calls]: We trash the buffer content _before_ reducing the ref - // count to zero, which looks racy because in principle a concurrent caller could come along - // and successfully retain() this buffer to keep it alive after it's been trashed. Such a - // caller would sometimes get an IllegalReferenceCountException ofc but that's something it - // could handle - see for instance org.elasticsearch.transport.netty4.Netty4Utils.ByteBufRefCounted.tryIncRef. - // Yet in practice this should never happen, we only ever retain() these buffers while we - // know them to be alive (i.e. via RefCounted#mustIncRef or its moral equivalents) so it'd - // be a bug for a caller to retain() a buffer whose ref count is heading to zero and whose - // contents we've already decided to trash. - private void trashContent() { - if (trashed == false) { - trashed = true; - TrashingByteBufAllocator.trashBuffer(buf); - } - } - } - static class TrashingCompositeByteBuf extends CompositeByteBuf { TrashingCompositeByteBuf(ByteBufAllocator alloc, boolean direct, int maxNumComponents) { diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/TrashingByteBuf.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/TrashingByteBuf.java new file mode 100644 index 0000000000000..ead0d595f0105 --- /dev/null +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/TrashingByteBuf.java @@ -0,0 +1,536 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.transport.netty4; + +import io.netty.buffer.ByteBuf; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +class TrashingByteBuf extends WrappedByteBuf { + + private boolean trashed = false; + + protected TrashingByteBuf(ByteBuf buf) { + super(buf); + } + + static TrashingByteBuf newBuf(ByteBuf buf) { + return new TrashingByteBuf(buf); + } + + @Override + public boolean release() { + if (refCnt() == 1) { + // see [NOTE on racy trashContent() calls] + trashContent(); + } + return super.release(); + } + + @Override + public boolean release(int decrement) { + if (refCnt() == decrement && refCnt() > 0) { + // see [NOTE on racy trashContent() calls] + trashContent(); + } + return super.release(decrement); + } + + // [NOTE on racy trashContent() calls]: We trash the buffer content _before_ reducing the ref + // count to zero, which looks racy because in principle a concurrent caller could come along + // and successfully retain() this buffer to keep it alive after it's been trashed. Such a + // caller would sometimes get an IllegalReferenceCountException ofc but that's something it + // could handle - see for instance org.elasticsearch.transport.netty4.Netty4Utils.ByteBufRefCounted.tryIncRef. + // Yet in practice this should never happen, we only ever retain() these buffers while we + // know them to be alive (i.e. via RefCounted#mustIncRef or its moral equivalents) so it'd + // be a bug for a caller to retain() a buffer whose ref count is heading to zero and whose + // contents we've already decided to trash. + private void trashContent() { + if (trashed == false) { + trashed = true; + NettyAllocator.TrashingByteBufAllocator.trashBuffer(buf); + } + } + + @Override + public ByteBuf capacity(int newCapacity) { + super.capacity(newCapacity); + return this; + } + + @Override + public ByteBuf order(ByteOrder endianness) { + return newBuf(super.order(endianness)); + } + + @Override + public ByteBuf asReadOnly() { + return newBuf(super.asReadOnly()); + } + + @Override + public ByteBuf setIndex(int readerIndex, int writerIndex) { + super.setIndex(readerIndex, writerIndex); + return this; + } + + @Override + public ByteBuf discardReadBytes() { + super.discardReadBytes(); + return this; + } + + @Override + public ByteBuf discardSomeReadBytes() { + super.discardSomeReadBytes(); + return this; + } + + @Override + public ByteBuf ensureWritable(int minWritableBytes) { + super.ensureWritable(minWritableBytes); + return this; + } + + @Override + public ByteBuf getBytes(int index, ByteBuf dst) { + super.getBytes(index, dst); + return this; + } + + @Override + public ByteBuf getBytes(int index, ByteBuf dst, int length) { + super.getBytes(index, dst, length); + return this; + } + + @Override + public ByteBuf getBytes(int index, ByteBuf dst, int dstIndex, int length) { + super.getBytes(index, dst, dstIndex, length); + return this; + } + + @Override + public ByteBuf getBytes(int index, byte[] dst) { + super.getBytes(index, dst); + return this; + } + + @Override + public ByteBuf getBytes(int index, byte[] dst, int dstIndex, int length) { + super.getBytes(index, dst, dstIndex, length); + return this; + } + + @Override + public ByteBuf getBytes(int index, ByteBuffer dst) { + super.getBytes(index, dst); + return this; + } + + @Override + public ByteBuf getBytes(int index, OutputStream out, int length) throws IOException { + super.getBytes(index, out, length); + return this; + } + + @Override + public ByteBuf setBoolean(int index, boolean value) { + super.setBoolean(index, value); + return this; + } + + @Override + public ByteBuf setByte(int index, int value) { + super.setByte(index, value); + return this; + } + + @Override + public ByteBuf setShort(int index, int value) { + super.setShort(index, value); + return this; + } + + @Override + public ByteBuf setShortLE(int index, int value) { + super.setShortLE(index, value); + return this; + } + + @Override + public ByteBuf setMedium(int index, int value) { + super.setMedium(index, value); + return this; + } + + @Override + public ByteBuf setMediumLE(int index, int value) { + super.setMediumLE(index, value); + return this; + } + + @Override + public ByteBuf setInt(int index, int value) { + super.setInt(index, value); + return this; + } + + @Override + public ByteBuf setIntLE(int index, int value) { + super.setIntLE(index, value); + return this; + } + + @Override + public ByteBuf setLong(int index, long value) { + super.setLong(index, value); + return this; + } + + @Override + public ByteBuf setLongLE(int index, long value) { + super.setLongLE(index, value); + return this; + } + + @Override + public ByteBuf setChar(int index, int value) { + super.setChar(index, value); + return this; + } + + @Override + public ByteBuf setFloat(int index, float value) { + super.setFloat(index, value); + return this; + } + + @Override + public ByteBuf setDouble(int index, double value) { + super.setDouble(index, value); + return this; + } + + @Override + public ByteBuf setBytes(int index, ByteBuf src) { + super.setBytes(index, src); + return this; + } + + @Override + public ByteBuf setBytes(int index, ByteBuf src, int length) { + super.setBytes(index, src, length); + return this; + } + + @Override + public ByteBuf setBytes(int index, ByteBuf src, int srcIndex, int length) { + super.setBytes(index, src, srcIndex, length); + return this; + } + + @Override + public ByteBuf setBytes(int index, byte[] src) { + super.setBytes(index, src); + return this; + } + + @Override + public ByteBuf setBytes(int index, byte[] src, int srcIndex, int length) { + super.setBytes(index, src, srcIndex, length); + return this; + } + + @Override + public ByteBuf setBytes(int index, ByteBuffer src) { + super.setBytes(index, src); + return this; + } + + @Override + public ByteBuf readBytes(int length) { + return newBuf(super.readBytes(length)); + } + + @Override + public ByteBuf readSlice(int length) { + return newBuf(super.readSlice(length)); + } + + @Override + public ByteBuf readRetainedSlice(int length) { + return newBuf(super.readRetainedSlice(length)); + } + + @Override + public ByteBuf readBytes(ByteBuf dst) { + super.readBytes(dst); + return this; + } + + @Override + public ByteBuf readBytes(ByteBuf dst, int length) { + super.readBytes(dst, length); + return this; + } + + @Override + public ByteBuf readBytes(ByteBuf dst, int dstIndex, int length) { + super.readBytes(dst, dstIndex, length); + return this; + } + + @Override + public ByteBuf readBytes(byte[] dst) { + super.readBytes(dst); + return this; + } + + @Override + public ByteBuf readBytes(ByteBuffer dst) { + super.readBytes(dst); + return this; + } + + @Override + public ByteBuf readBytes(byte[] dst, int dstIndex, int length) { + super.readBytes(dst, dstIndex, length); + return this; + } + + @Override + public ByteBuf readBytes(OutputStream out, int length) throws IOException { + super.readBytes(out, length); + return this; + } + + @Override + public ByteBuf skipBytes(int length) { + super.skipBytes(length); + return this; + } + + @Override + public ByteBuf writeBoolean(boolean value) { + super.writeBoolean(value); + return this; + } + + @Override + public ByteBuf writeByte(int value) { + super.writeByte(value); + return this; + } + + @Override + public ByteBuf writeShort(int value) { + super.writeShort(value); + return this; + } + + @Override + public ByteBuf writeShortLE(int value) { + super.writeShortLE(value); + return this; + } + + @Override + public ByteBuf writeMedium(int value) { + super.writeMedium(value); + return this; + } + + @Override + public ByteBuf writeMediumLE(int value) { + super.writeMediumLE(value); + return this; + } + + @Override + public ByteBuf writeInt(int value) { + super.writeInt(value); + return this; + + } + + @Override + public ByteBuf writeIntLE(int value) { + super.writeIntLE(value); + return this; + } + + @Override + public ByteBuf writeLong(long value) { + super.writeLong(value); + return this; + } + + @Override + public ByteBuf writeLongLE(long value) { + super.writeLongLE(value); + return this; + } + + @Override + public ByteBuf writeChar(int value) { + super.writeChar(value); + return this; + } + + @Override + public ByteBuf writeFloat(float value) { + super.writeFloat(value); + return this; + } + + @Override + public ByteBuf writeDouble(double value) { + super.writeDouble(value); + return this; + } + + @Override + public ByteBuf writeBytes(ByteBuf src) { + super.writeBytes(src); + return this; + } + + @Override + public ByteBuf writeBytes(ByteBuf src, int length) { + super.writeBytes(src, length); + return this; + } + + @Override + public ByteBuf writeBytes(ByteBuf src, int srcIndex, int length) { + super.writeBytes(src, srcIndex, length); + return this; + } + + @Override + public ByteBuf writeBytes(byte[] src) { + super.writeBytes(src); + return this; + } + + @Override + public ByteBuf writeBytes(byte[] src, int srcIndex, int length) { + super.writeBytes(src, srcIndex, length); + return this; + } + + @Override + public ByteBuf writeBytes(ByteBuffer src) { + super.writeBytes(src); + return this; + } + + @Override + public ByteBuf writeZero(int length) { + super.writeZero(length); + return this; + } + + @Override + public ByteBuf copy() { + return newBuf(super.copy()); + } + + @Override + public ByteBuf copy(int index, int length) { + return newBuf(super.copy(index, length)); + } + + @Override + public ByteBuf slice() { + return newBuf(super.slice()); + } + + @Override + public ByteBuf retainedSlice() { + return newBuf(super.retainedSlice()); + } + + @Override + public ByteBuf slice(int index, int length) { + return newBuf(super.slice(index, length)); + } + + @Override + public ByteBuf retainedSlice(int index, int length) { + return newBuf(super.retainedSlice(index, length)); + } + + @Override + public ByteBuf duplicate() { + return newBuf(super.duplicate()); + } + + @Override + public ByteBuf retainedDuplicate() { + return newBuf(super.retainedDuplicate()); + } + + @Override + public ByteBuf retain(int increment) { + super.retain(increment); + return this; + } + + @Override + public ByteBuf touch(Object hint) { + super.touch(hint); + return this; + } + + @Override + public ByteBuf retain() { + super.retain(); + return this; + } + + @Override + public ByteBuf touch() { + super.touch(); + return this; + } + + @Override + public ByteBuf setFloatLE(int index, float value) { + return super.setFloatLE(index, value); + } + + @Override + public ByteBuf setDoubleLE(int index, double value) { + super.setDoubleLE(index, value); + return this; + } + + @Override + public ByteBuf writeFloatLE(float value) { + super.writeFloatLE(value); + return this; + } + + @Override + public ByteBuf writeDoubleLE(double value) { + super.writeDoubleLE(value); + return this; + } + + @Override + public ByteBuf asByteBuf() { + return this; + } +} diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyAllocatorTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyAllocatorTests.java index a76eb9fa4875b..b9e9b667e72fe 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyAllocatorTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/NettyAllocatorTests.java @@ -20,7 +20,6 @@ import java.nio.ByteBuffer; import java.util.List; -import static org.elasticsearch.transport.netty4.NettyAllocator.TrashingByteBuf; import static org.elasticsearch.transport.netty4.NettyAllocator.TrashingByteBufAllocator; public class NettyAllocatorTests extends ESTestCase { From 0b764adbc19a99ee14d88b96f5f99002fabc19cb Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sat, 30 Nov 2024 08:29:46 +1100 Subject: [PATCH 27/27] Mute org.elasticsearch.search.ccs.CrossClusterIT testCancel #108061 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index f5f6b84ab8639..b82e95ea26890 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -231,6 +231,9 @@ tests: - class: org.elasticsearch.xpack.esql.ccq.MultiClusterSpecIT method: test {scoring.QstrWithFieldAndScoringSortedEval} issue: https://github.com/elastic/elasticsearch/issues/117751 +- class: org.elasticsearch.search.ccs.CrossClusterIT + method: testCancel + issue: https://github.com/elastic/elasticsearch/issues/108061 # Examples: #