From cff81d9c45537cf90e6442c84f5b1e89a7dadf0f Mon Sep 17 00:00:00 2001 From: Przemyslaw Witek Date: Mon, 19 May 2025 16:06:54 +0200 Subject: [PATCH 001/231] Introduce grouping in TopN operator --- .../compute/operator/TopNBenchmark.java | 1 + .../compute/operator/topn/TopNOperator.java | 96 +++++++++++++--- .../operator/topn/TopNOperatorTests.java | 20 +++- .../esql/expression/ExpressionWritables.java | 1 + .../xpack/esql/expression/Partition.java | 105 ++++++++++++++++++ .../logical/ReplaceLimitAndSortAsTopN.java | 4 +- .../xpack/esql/plan/logical/TopN.java | 21 +++- .../xpack/esql/plan/physical/TopNExec.java | 29 ++++- .../esql/planner/LocalExecutionPlanner.java | 12 +- .../esql/planner/mapper/LocalMapper.java | 2 +- .../xpack/esql/planner/mapper/Mapper.java | 2 +- .../PartitionSerializationTests.java | 37 ++++++ .../physical/local/PushTopNToSourceTests.java | 11 +- .../plan/logical/TopNSerializationTests.java | 19 +++- .../physical/TopNExecSerializationTests.java | 17 ++- 15 files changed, 328 insertions(+), 49 deletions(-) create mode 100644 x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/Partition.java create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/PartitionSerializationTests.java diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/TopNBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/TopNBenchmark.java index 945e7b6c14fca..24006463ce4c4 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/TopNBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/TopNBenchmark.java @@ -125,6 +125,7 @@ private static Operator operator(String data, int topCount) { topCount, elementTypes, encoders, + List.of(), IntStream.range(0, count).mapToObj(c -> new TopNOperator.SortOrder(c, false, false)).toList(), 16 * 1024 ); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java index 0489be58fade7..fedfb9af50263 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.BytesRefBlock; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.BreakingBytesRefBuilder; @@ -23,11 +24,14 @@ import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; +import java.util.TreeMap; /** * An operator that sorts "rows" of values by encoding the values to sort on, as bytes (using BytesRef). Each data type is encoded @@ -194,6 +198,16 @@ private void writeValues(int position, BreakingBytesRefBuilder values) { } } + public record Partition(int channel) { + + private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(Partition.class); + + @Override + public String toString() { + return "Partition[channel=" + this.channel + "]"; + } + } + public record SortOrder(int channel, boolean asc, boolean nullsFirst) { private static final long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(SortOrder.class); @@ -224,6 +238,7 @@ public record TopNOperatorFactory( int topCount, List elementTypes, List encoders, + List partitions, List sortOrders, int maxPageSize ) implements OperatorFactory { @@ -243,6 +258,7 @@ public TopNOperator get(DriverContext driverContext) { topCount, elementTypes, encoders, + partitions, sortOrders, maxPageSize ); @@ -256,6 +272,8 @@ public String describe() { + elementTypes + ", encoders=" + encoders + + ", partitions=" + + partitions + ", sortOrders=" + sortOrders + "]"; @@ -264,12 +282,14 @@ public String describe() { private final BlockFactory blockFactory; private final CircuitBreaker breaker; - private final Queue inputQueue; + private final Map inputQueues; + private final int topCount; private final int maxPageSize; private final List elementTypes; private final List encoders; + private final List partitions; private final List sortOrders; private Row spare; @@ -304,16 +324,19 @@ public TopNOperator( int topCount, List elementTypes, List encoders, + List partitions, List sortOrders, int maxPageSize ) { this.blockFactory = blockFactory; this.breaker = breaker; + this.topCount = topCount; this.maxPageSize = maxPageSize; this.elementTypes = elementTypes; this.encoders = encoders; + this.partitions = partitions; this.sortOrders = sortOrders; - this.inputQueue = new Queue(topCount); + this.inputQueues = new TreeMap<>(); } static int compareRows(Row r1, Row r2) { @@ -385,6 +408,8 @@ public void addInput(Page page) { spareKeysPreAllocSize = Math.max(spare.keys.length(), spareKeysPreAllocSize / 2); spareValuesPreAllocSize = Math.max(spare.values.length(), spareValuesPreAllocSize / 2); + String partitionKey = getPartitionKey(page, i); + Queue inputQueue = inputQueues.computeIfAbsent(partitionKey, key -> new Queue(topCount)); spare = inputQueue.insertWithOverflow(spare); } } finally { @@ -394,6 +419,28 @@ public void addInput(Page page) { } } + /** + * Calculates the partition key of the i-th row of the given page. + * + * @param page page for which the partition key should be calculated + * @param i row index + * @return partition key of the i-th row of the given page + */ + private String getPartitionKey(Page page, int i) { + if (partitions.isEmpty()) { + return ""; + } + assert page.getPositionCount() > 0; + StringBuilder builder = new StringBuilder(); + for (Partition partition : partitions) { + try (var block = page.getBlock(partition.channel).filter(i)) { + BytesRef partitionFieldValue = ((BytesRefBlock) block).getBytesRef(i, new BytesRef()); + builder.append(partitionFieldValue.utf8ToString()); + } + } + return builder.toString(); + } + @Override public void finish() { if (output == null) { @@ -407,14 +454,17 @@ private Iterator toPages() { spare.close(); spare = null; } - if (inputQueue.size() == 0) { - return Collections.emptyIterator(); - } - List list = new ArrayList<>(inputQueue.size()); - List result = new ArrayList<>(); - ResultBuilder[] builders = null; boolean success = false; + List list = null; + ResultBuilder[] builders = null; + List result = new ArrayList<>(); + // TODO: optimize case where all the queues are empty try { + for (var entry : inputQueues.entrySet()) { + Queue inputQueue = entry.getValue(); + + list = new ArrayList<>(inputQueue.size()); + builders = null; while (inputQueue.size() > 0) { list.add(inputQueue.pop()); } @@ -483,6 +533,7 @@ private Iterator toPages() { } } assert builders == null; + } success = true; return result.iterator(); } finally { @@ -524,20 +575,20 @@ public Page getOutput() { @Override public void close() { + List releasables = new ArrayList<>(); + releasables.addAll(inputQueues.values().stream().map(Releasables::wrap).toList()); + releasables.add(output == null ? null : Releasables.wrap(() -> Iterators.map(output, p -> p::releaseBlocks))); /* * If we close before calling finish then spare and inputQueue will be live rows * that need closing. If we close after calling finish then the output iterator * will contain pages of results that have yet to be returned. */ - Releasables.closeExpectNoException( - spare, - inputQueue == null ? null : Releasables.wrap(inputQueue), - output == null ? null : Releasables.wrap(() -> Iterators.map(output, p -> p::releaseBlocks)) - ); + Releasables.closeExpectNoException(spare, Releasables.wrap(releasables)); } - private static long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(TopNOperator.class) + RamUsageEstimator - .shallowSizeOfInstance(List.class) * 3; + private static long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(TopNOperator.class) + + RamUsageEstimator.shallowSizeOfInstance(List.class) * 4 + + RamUsageEstimator.shallowSizeOfInstance(Map.class); @Override public long ramBytesUsed() { @@ -548,25 +599,34 @@ public long ramBytesUsed() { // These lists may slightly under-count, but it's not likely to be by much. size += RamUsageEstimator.alignObjectSize(arrHeader + ref * elementTypes.size()); size += RamUsageEstimator.alignObjectSize(arrHeader + ref * encoders.size()); + size += RamUsageEstimator.alignObjectSize(arrHeader + ref * partitions.size()); + size += partitions.size() * Partition.SHALLOW_SIZE; size += RamUsageEstimator.alignObjectSize(arrHeader + ref * sortOrders.size()); size += sortOrders.size() * SortOrder.SHALLOW_SIZE; - size += inputQueue.ramBytesUsed(); + long ramBytesUsedSum = inputQueues.entrySet().stream() + .mapToLong(e -> e.getKey().getBytes(Charset.defaultCharset()).length + e.getValue().ramBytesUsed()) + .sum(); + size += ramBytesUsedSum; return size; } @Override public Status status() { - return new TopNOperatorStatus(inputQueue.size(), ramBytesUsed(), pagesReceived, pagesEmitted, rowsReceived, rowsEmitted); + int queueSizeSum = inputQueues.values().stream().mapToInt(Queue::size).sum(); + return new TopNOperatorStatus(queueSizeSum, ramBytesUsed(), pagesReceived, pagesEmitted, rowsReceived, rowsEmitted); } @Override public String toString() { + int queueSizeSum = inputQueues.values().stream().mapToInt(Queue::size).sum(); return "TopNOperator[count=" - + inputQueue + + queueSizeSum + "/" + topCount + ", elementTypes=" + elementTypes + ", encoders=" + encoders + + ", partitions=" + + partitions + ", sortOrders=" + sortOrders + "]"; diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java index 8561ce84744aa..2534d1285571d 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java @@ -135,6 +135,7 @@ protected TopNOperator.TopNOperatorFactory simple(SimpleOptions options) { 4, List.of(LONG), List.of(DEFAULT_UNSORTABLE), + List.of(), List.of(new TopNOperator.SortOrder(0, true, false)), pageSize ); @@ -143,7 +144,7 @@ protected TopNOperator.TopNOperatorFactory simple(SimpleOptions options) { @Override protected Matcher expectedDescriptionOfSimple() { return equalTo( - "TopNOperator[count=4, elementTypes=[LONG], encoders=[DefaultUnsortable], " + "TopNOperator[count=4, elementTypes=[LONG], encoders=[DefaultUnsortable], partitions=[], " + "sortOrders=[SortOrder[channel=0, asc=true, nullsFirst=false]]]" ); } @@ -151,7 +152,7 @@ protected Matcher expectedDescriptionOfSimple() { @Override protected Matcher expectedToStringOfSimple() { return equalTo( - "TopNOperator[count=0/4, elementTypes=[LONG], encoders=[DefaultUnsortable], " + "TopNOperator[count=0/4, elementTypes=[LONG], encoders=[DefaultUnsortable], partitions=[], " + "sortOrders=[SortOrder[channel=0, asc=true, nullsFirst=false]]]" ); } @@ -216,6 +217,7 @@ public long accumulateObject(Object o, long shallowSize, Map fiel topCount, List.of(LONG), List.of(DEFAULT_UNSORTABLE), + List.of(), List.of(new TopNOperator.SortOrder(0, true, false)), pageSize ).get(context) @@ -554,6 +556,7 @@ public void testCollectAllValues() { topCount, elementTypes, encoders, + List.of(), List.of(new TopNOperator.SortOrder(0, false, false)), randomPageSize() ) @@ -643,6 +646,7 @@ public void testCollectAllValues_RandomMultiValues() { topCount, elementTypes, encoders, + List.of(), List.of(new TopNOperator.SortOrder(0, false, false)), randomPageSize() ) @@ -677,6 +681,7 @@ private List> topNTwoColumns( limit, elementTypes, encoder, + List.of(), sortOrders, randomPageSize() ) @@ -704,6 +709,7 @@ public void testTopNManyDescriptionAndToString() { 10, List.of(BYTES_REF, BYTES_REF), List.of(UTF8, new FixedLengthTopNEncoder(fixedLength)), + List.of(), List.of(new TopNOperator.SortOrder(1, false, false), new TopNOperator.SortOrder(3, false, true)), randomPageSize() ); @@ -712,7 +718,7 @@ public void testTopNManyDescriptionAndToString() { .collect(Collectors.joining(", ")); String tail = ", elementTypes=[BYTES_REF, BYTES_REF], encoders=[UTF8TopNEncoder, FixedLengthTopNEncoder[" + fixedLength - + "]], sortOrders=[" + + "]], partitions=[], sortOrders=[" + sorts + "]]"; assertThat(factory.describe(), equalTo("TopNOperator[count=10" + tail)); @@ -946,6 +952,7 @@ private void assertSortingOnMV( topCount, List.of(blockType), List.of(encoder), + List.of(), List.of(sortOrders), randomPageSize() ) @@ -1076,6 +1083,7 @@ public void testRandomMultiValuesTopN() { topCount, elementTypes, encoders, + List.of(), uniqueOrders.stream().toList(), rows ), @@ -1119,6 +1127,7 @@ public void testIPSortingSingleValue() throws UnknownHostException { ips.size(), List.of(BYTES_REF), List.of(TopNEncoder.IP), + List.of(), List.of(new TopNOperator.SortOrder(0, asc, randomBoolean())), randomPageSize() ) @@ -1245,6 +1254,7 @@ private void assertIPSortingOnMultiValues( ips.size(), List.of(BYTES_REF), List.of(TopNEncoder.IP), + List.of(), List.of(new TopNOperator.SortOrder(0, asc, nullsFirst)), randomPageSize() ) @@ -1332,6 +1342,7 @@ public void testZeroByte() { 2, List.of(BYTES_REF, INT), List.of(TopNEncoder.UTF8, DEFAULT_UNSORTABLE), + List.of(), List.of( new TopNOperator.SortOrder(0, true, randomBoolean()), new TopNOperator.SortOrder(1, randomBoolean(), randomBoolean()) @@ -1371,6 +1382,7 @@ public void testErrorBeforeFullyDraining() { topCount, List.of(LONG), List.of(DEFAULT_UNSORTABLE), + List.of(), List.of(new TopNOperator.SortOrder(0, true, randomBoolean())), maxPageSize ) @@ -1406,6 +1418,7 @@ public void testCloseWithoutCompleting() { 2, List.of(INT), List.of(DEFAULT_UNSORTABLE), + List.of(), List.of(new TopNOperator.SortOrder(0, randomBoolean(), randomBoolean())), randomPageSize() ) @@ -1429,6 +1442,7 @@ public void testRowResizes() { 10, types, encoders, + List.of(), List.of(new TopNOperator.SortOrder(0, randomBoolean(), randomBoolean())), randomPageSize() ) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/ExpressionWritables.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/ExpressionWritables.java index 4ffa778b8287a..158d9e17affc0 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/ExpressionWritables.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/ExpressionWritables.java @@ -126,6 +126,7 @@ public static List expressions() { List entries = new ArrayList<>(); entries.addAll(ExpressionCoreWritables.expressions()); entries.add(UnsupportedAttribute.EXPRESSION_ENTRY); + entries.add(Partition.ENTRY); entries.add(Order.ENTRY); return entries; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/Partition.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/Partition.java new file mode 100644 index 0000000000000..72a9a36f8aef3 --- /dev/null +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/Partition.java @@ -0,0 +1,105 @@ +/* + * 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.expression; + +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.expression.Nullability; +import org.elasticsearch.xpack.esql.core.tree.NodeInfo; +import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; + +import java.io.IOException; +import java.util.List; +import java.util.Objects; + +import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isExact; + +public class Partition extends Expression { + public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry( + Expression.class, + "Partition", + Partition::new + ); + + private final Expression child; + + public Partition(Source source, Expression child) { + super(source, List.of(child)); + this.child = child; + } + + public Partition(StreamInput in) throws IOException { + this(Source.readFrom((PlanStreamInput) in), in.readNamedWriteable(Expression.class)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + Source.EMPTY.writeTo(out); + out.writeNamedWriteable(child); + } + + @Override + public String getWriteableName() { + return ENTRY.name; + } + + @Override + protected TypeResolution resolveType() { + if (DataType.isString(child.dataType())) { + return TypeResolution.TYPE_RESOLVED; + } + return isExact(child, "BY cannot be applied to field of data type [{}]: {}"); + } + + @Override + public DataType dataType() { + return child.dataType(); + } + + @Override + public Partition replaceChildren(List newChildren) { + return new Partition(source(), newChildren.get(0)); + } + + @Override + protected NodeInfo info() { + return NodeInfo.create(this, Partition::new, child); + } + + @Override + public Nullability nullable() { + return Nullability.FALSE; + } + + public Expression child() { + return child; + } + + @Override + public int hashCode() { + return Objects.hash(child); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + Partition other = (Partition) obj; + return Objects.equals(child, other.child); + } +} diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java index 7d44fa1fda5a2..37ee7b7ec65ad 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceLimitAndSortAsTopN.java @@ -12,13 +12,15 @@ import org.elasticsearch.xpack.esql.plan.logical.OrderBy; import org.elasticsearch.xpack.esql.plan.logical.TopN; +import java.util.List; + public final class ReplaceLimitAndSortAsTopN extends OptimizerRules.OptimizerRule { @Override protected LogicalPlan rule(Limit plan) { LogicalPlan p = plan; if (plan.child() instanceof OrderBy o) { - p = new TopN(plan.source(), o.child(), o.order(), plan.limit()); + p = new TopN(plan.source(), o.child(), List.of(), o.order(), plan.limit()); } return p; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/TopN.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/TopN.java index a9a5dbddc544f..88579cdd3efbd 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/TopN.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/logical/TopN.java @@ -15,6 +15,7 @@ import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.expression.Order; +import org.elasticsearch.xpack.esql.expression.Partition; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; import java.io.IOException; @@ -24,11 +25,13 @@ public class TopN extends UnaryPlan { public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(LogicalPlan.class, "TopN", TopN::new); + private final List partition; private final List order; private final Expression limit; - public TopN(Source source, LogicalPlan child, List order, Expression limit) { + public TopN(Source source, LogicalPlan child, List partition, List order, Expression limit) { super(source, child); + this.partition = partition; this.order = order; this.limit = limit; } @@ -37,6 +40,7 @@ private TopN(StreamInput in) throws IOException { this( Source.readFrom((PlanStreamInput) in), in.readNamedWriteable(LogicalPlan.class), + in.readCollectionAsList(Partition::new), in.readCollectionAsList(Order::new), in.readNamedWriteable(Expression.class) ); @@ -46,6 +50,7 @@ private TopN(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { Source.EMPTY.writeTo(out); out.writeNamedWriteable(child()); + out.writeCollection(partition); out.writeCollection(order); out.writeNamedWriteable(limit); } @@ -57,37 +62,41 @@ public String getWriteableName() { @Override public boolean expressionsResolved() { - return limit.resolved() && Resolvables.resolved(order); + return limit.resolved() && Resolvables.resolved(partition) && Resolvables.resolved(order); } @Override protected NodeInfo info() { - return NodeInfo.create(this, TopN::new, child(), order, limit); + return NodeInfo.create(this, TopN::new, child(), partition, order, limit); } @Override public TopN replaceChild(LogicalPlan newChild) { - return new TopN(source(), newChild, order, limit); + return new TopN(source(), newChild, partition, order, limit); } public Expression limit() { return limit; } + public List partition() { + return partition; + } + public List order() { return order; } @Override public int hashCode() { - return Objects.hash(super.hashCode(), order, limit); + return Objects.hash(super.hashCode(), partition, order, limit); } @Override public boolean equals(Object obj) { if (super.equals(obj)) { var other = (TopN) obj; - return Objects.equals(order, other.order) && Objects.equals(limit, other.limit); + return Objects.equals(partition, other.partition) && Objects.equals(order, other.order) && Objects.equals(limit, other.limit); } return false; } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/TopNExec.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/TopNExec.java index c88e77e4972ca..71632b035b8f7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/TopNExec.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plan/physical/TopNExec.java @@ -16,6 +16,7 @@ import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.expression.Order; +import org.elasticsearch.xpack.esql.expression.Partition; import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput; import java.io.IOException; @@ -30,6 +31,7 @@ public class TopNExec extends UnaryExec implements EstimatesRowSize { ); private final Expression limit; + private final List partition; private final List order; /** @@ -38,8 +40,16 @@ public class TopNExec extends UnaryExec implements EstimatesRowSize { */ private final Integer estimatedRowSize; - public TopNExec(Source source, PhysicalPlan child, List order, Expression limit, Integer estimatedRowSize) { + public TopNExec( + Source source, + PhysicalPlan child, + List partition, + List order, + Expression limit, + Integer estimatedRowSize + ) { super(source, child); + this.partition = partition; this.order = order; this.limit = limit; this.estimatedRowSize = estimatedRowSize; @@ -49,6 +59,7 @@ private TopNExec(StreamInput in) throws IOException { this( Source.readFrom((PlanStreamInput) in), in.readNamedWriteable(PhysicalPlan.class), + in.readCollectionAsList(org.elasticsearch.xpack.esql.expression.Partition::new), in.readCollectionAsList(org.elasticsearch.xpack.esql.expression.Order::new), in.readNamedWriteable(Expression.class), in.readOptionalVInt() @@ -59,6 +70,7 @@ private TopNExec(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { Source.EMPTY.writeTo(out); out.writeNamedWriteable(child()); + out.writeCollection(partition()); out.writeCollection(order()); out.writeNamedWriteable(limit()); out.writeOptionalVInt(estimatedRowSize()); @@ -71,18 +83,22 @@ public String getWriteableName() { @Override protected NodeInfo info() { - return NodeInfo.create(this, TopNExec::new, child(), order, limit, estimatedRowSize); + return NodeInfo.create(this, TopNExec::new, child(), partition, order, limit, estimatedRowSize); } @Override public TopNExec replaceChild(PhysicalPlan newChild) { - return new TopNExec(source(), newChild, order, limit, estimatedRowSize); + return new TopNExec(source(), newChild, partition, order, limit, estimatedRowSize); } public Expression limit() { return limit; } + public List partition() { + return partition; + } + public List order() { return order; } @@ -102,12 +118,12 @@ public PhysicalPlan estimateRowSize(State state) { state.add(needsSortedDocIds, output); int size = state.consumeAllFields(true); size = Math.max(size, 1); - return Objects.equals(this.estimatedRowSize, size) ? this : new TopNExec(source(), child(), order, limit, size); + return Objects.equals(this.estimatedRowSize, size) ? this : new TopNExec(source(), child(), partition, order, limit, size); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), order, limit, estimatedRowSize); + return Objects.hash(super.hashCode(), partition, order, limit, estimatedRowSize); } @Override @@ -115,7 +131,8 @@ public boolean equals(Object obj) { boolean equals = super.equals(obj); if (equals) { var other = (TopNExec) obj; - equals = Objects.equals(order, other.order) + equals = Objects.equals(partition, other.partition) + && Objects.equals(order, other.order) && Objects.equals(limit, other.limit) && Objects.equals(estimatedRowSize, other.estimatedRowSize); } 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 c89327e6a6aa4..f6d5c2ef8ec2c 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 @@ -444,6 +444,16 @@ private PhysicalOperation planTopN(TopNExec topNExec, LocalExecutionPlannerConte case PARTIAL_AGG, UNSUPPORTED -> TopNEncoder.UNSUPPORTED; }; } + List partitions = topNExec.partition().stream().map(partition -> { + int partitionByChannel; + if (partition.child() instanceof Attribute a) { + partitionByChannel = source.layout.get(a.id()).channel(); + } else { + throw new EsqlIllegalArgumentException("partition by expression must be an attribute"); + } + return new TopNOperator.Partition(partitionByChannel); + }).toList(); + List orders = topNExec.order().stream().map(order -> { int sortByChannel; if (order.child() instanceof Attribute a) { @@ -466,7 +476,7 @@ private PhysicalOperation planTopN(TopNExec topNExec, LocalExecutionPlannerConte throw new EsqlIllegalArgumentException("limit only supported with literal values"); } return source.with( - new TopNOperatorFactory(limit, asList(elementTypes), asList(encoders), orders, context.pageSize(rowSize)), + new TopNOperatorFactory(limit, asList(elementTypes), asList(encoders), partitions, orders, context.pageSize(rowSize)), source.layout ); } 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 4eba58edbe762..36320f7133409 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 @@ -82,7 +82,7 @@ private PhysicalPlan mapUnary(UnaryPlan unary) { } if (unary instanceof TopN topN) { - return new TopNExec(topN.source(), mappedChild, topN.order(), topN.limit(), null); + return new TopNExec(topN.source(), mappedChild, topN.partition(), topN.order(), topN.limit(), null); } if (unary instanceof Sample sample) { 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 1af000a7a36bd..cb23002e24957 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 @@ -173,7 +173,7 @@ private PhysicalPlan mapUnary(UnaryPlan unary) { if (unary instanceof TopN topN) { mappedChild = addExchangeForFragment(topN, mappedChild); - return new TopNExec(topN.source(), mappedChild, topN.order(), topN.limit(), null); + return new TopNExec(topN.source(), mappedChild, topN.partition(), topN.order(), topN.limit(), null); } if (unary instanceof Rerank rerank) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/PartitionSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/PartitionSerializationTests.java new file mode 100644 index 0000000000000..8b9e38cb939f1 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/PartitionSerializationTests.java @@ -0,0 +1,37 @@ +/* + * 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.expression; + +import org.elasticsearch.xpack.esql.core.expression.Expression; +import org.elasticsearch.xpack.esql.core.tree.Source; + +import java.io.IOException; + +public class PartitionSerializationTests extends AbstractExpressionSerializationTests { + public static Partition randomPartition() { + return new Partition(randomSource(), randomChild()); + } + + @Override + protected Partition createTestInstance() { + return randomPartition(); + } + + @Override + protected Partition mutateInstance(Partition instance) throws IOException { + Source source = instance.source(); + Expression child = instance.child(); + child = randomValueOtherThan(child, AbstractExpressionSerializationTests::randomChild); + return new Partition(source, child); + } + + @Override + protected boolean alwaysEmptySource() { + return true; + } +} 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 b7eadc243d977..7efee8c62ad14 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 @@ -28,6 +28,7 @@ import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.expression.Order; +import org.elasticsearch.xpack.esql.expression.Partition; import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.StDistance; import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add; import org.elasticsearch.xpack.esql.optimizer.LocalPhysicalOptimizerContext; @@ -484,6 +485,7 @@ static class TestPhysicalPlanBuilder { private final LinkedHashMap metadata; private IndexMode indexMode; private final List aliases = new ArrayList<>(); + private final List partitions = new ArrayList<>(); private final List orders = new ArrayList<>(); private int limit = Integer.MAX_VALUE; @@ -586,7 +588,14 @@ public TopNExec build() { if (aliases.isEmpty() == false) { child = new EvalExec(Source.EMPTY, child, aliases); } - return new TopNExec(Source.EMPTY, child, orders, new Literal(Source.EMPTY, limit, INTEGER), randomEstimatedRowSize()); + return new TopNExec( + Source.EMPTY, + child, + partitions, + orders, + new Literal(Source.EMPTY, limit, INTEGER), + randomEstimatedRowSize() + ); } public TestPhysicalPlanBuilder asTimeSeries() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/TopNSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/TopNSerializationTests.java index 6c3863582f215..6096acf819070 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/TopNSerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/logical/TopNSerializationTests.java @@ -12,6 +12,8 @@ import org.elasticsearch.xpack.esql.expression.AbstractExpressionSerializationTests; import org.elasticsearch.xpack.esql.expression.Order; import org.elasticsearch.xpack.esql.expression.OrderSerializationTests; +import org.elasticsearch.xpack.esql.expression.Partition; +import org.elasticsearch.xpack.esql.expression.PartitionSerializationTests; import java.io.IOException; import java.util.List; @@ -20,9 +22,14 @@ public class TopNSerializationTests extends AbstractLogicalPlanSerializationTest public static TopN randomTopN(int depth) { Source source = randomSource(); LogicalPlan child = randomChild(depth); + List partition = randomPartitions(); List order = randomOrders(); Expression limit = AbstractExpressionSerializationTests.randomChild(); - return new TopN(source, child, order, limit); + return new TopN(source, child, partition, order, limit); + } + + private static List randomPartitions() { + return randomList(1, 10, PartitionSerializationTests::randomPartition); } private static List randomOrders() { @@ -38,14 +45,16 @@ protected TopN createTestInstance() { protected TopN mutateInstance(TopN instance) throws IOException { Source source = instance.source(); LogicalPlan child = instance.child(); + List partition = instance.partition(); List order = instance.order(); Expression limit = instance.limit(); - switch (between(0, 2)) { + switch (between(0, 3)) { case 0 -> child = randomValueOtherThan(child, () -> randomChild(0)); - case 1 -> order = randomValueOtherThan(order, TopNSerializationTests::randomOrders); - case 2 -> limit = randomValueOtherThan(limit, AbstractExpressionSerializationTests::randomChild); + case 1 -> partition = randomValueOtherThan(partition, TopNSerializationTests::randomPartitions); + case 2 -> order = randomValueOtherThan(order, TopNSerializationTests::randomOrders); + case 3 -> limit = randomValueOtherThan(limit, AbstractExpressionSerializationTests::randomChild); } - return new TopN(source, child, order, limit); + return new TopN(source, child, partition, order, limit); } @Override diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/TopNExecSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/TopNExecSerializationTests.java index 9606079e2f698..a9544eb4842fc 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/TopNExecSerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/plan/physical/TopNExecSerializationTests.java @@ -13,6 +13,8 @@ import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.expression.Order; import org.elasticsearch.xpack.esql.expression.OrderSerializationTests; +import org.elasticsearch.xpack.esql.expression.Partition; +import org.elasticsearch.xpack.esql.expression.PartitionSerializationTests; import java.io.IOException; import java.util.List; @@ -21,10 +23,11 @@ public class TopNExecSerializationTests extends AbstractPhysicalPlanSerializatio public static TopNExec randomTopNExec(int depth) { Source source = randomSource(); PhysicalPlan child = randomChild(depth); + List partition = randomList(1, 10, PartitionSerializationTests::randomPartition); List order = randomList(1, 10, OrderSerializationTests::randomOrder); Expression limit = new Literal(randomSource(), randomNonNegativeInt(), DataType.INTEGER); Integer estimatedRowSize = randomEstimatedRowSize(); - return new TopNExec(source, child, order, limit, estimatedRowSize); + return new TopNExec(source, child, partition, order, limit, estimatedRowSize); } @Override @@ -35,20 +38,22 @@ protected TopNExec createTestInstance() { @Override protected TopNExec mutateInstance(TopNExec instance) throws IOException { PhysicalPlan child = instance.child(); + List partition = instance.partition(); List order = instance.order(); Expression limit = instance.limit(); Integer estimatedRowSize = instance.estimatedRowSize(); - switch (between(0, 3)) { + switch (between(0, 4)) { case 0 -> child = randomValueOtherThan(child, () -> randomChild(0)); - case 1 -> order = randomValueOtherThan(order, () -> randomList(1, 10, OrderSerializationTests::randomOrder)); - case 2 -> limit = randomValueOtherThan(limit, () -> new Literal(randomSource(), randomNonNegativeInt(), DataType.INTEGER)); - case 3 -> estimatedRowSize = randomValueOtherThan( + case 1 -> partition = randomValueOtherThan(partition, () -> randomList(1, 10, PartitionSerializationTests::randomPartition)); + case 2 -> order = randomValueOtherThan(order, () -> randomList(1, 10, OrderSerializationTests::randomOrder)); + case 3 -> limit = randomValueOtherThan(limit, () -> new Literal(randomSource(), randomNonNegativeInt(), DataType.INTEGER)); + case 4 -> estimatedRowSize = randomValueOtherThan( estimatedRowSize, AbstractPhysicalPlanSerializationTests::randomEstimatedRowSize ); default -> throw new UnsupportedOperationException(); } - return new TopNExec(instance.source(), child, order, limit, estimatedRowSize); + return new TopNExec(instance.source(), child, partition, order, limit, estimatedRowSize); } @Override From 8ac8814ad3a341a76e4af9d7fc46e807f34c0c3c Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Tue, 20 May 2025 09:04:38 +0000 Subject: [PATCH 002/231] [CI] Auto commit changes from spotless --- .../compute/operator/topn/TopNOperator.java | 134 +++++++++--------- 1 file changed, 68 insertions(+), 66 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java index fedfb9af50263..17db1b6673710 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java @@ -461,78 +461,78 @@ private Iterator toPages() { // TODO: optimize case where all the queues are empty try { for (var entry : inputQueues.entrySet()) { - Queue inputQueue = entry.getValue(); + Queue inputQueue = entry.getValue(); - list = new ArrayList<>(inputQueue.size()); - builders = null; - while (inputQueue.size() > 0) { - list.add(inputQueue.pop()); - } - Collections.reverse(list); - - int p = 0; - int size = 0; - for (int i = 0; i < list.size(); i++) { - if (builders == null) { - size = Math.min(maxPageSize, list.size() - i); - builders = new ResultBuilder[elementTypes.size()]; - for (int b = 0; b < builders.length; b++) { - builders[b] = ResultBuilder.resultBuilderFor( - blockFactory, - elementTypes.get(b), - encoders.get(b).toUnsortable(), - channelInKey(sortOrders, b), - size - ); - } - p = 0; + list = new ArrayList<>(inputQueue.size()); + builders = null; + while (inputQueue.size() > 0) { + list.add(inputQueue.pop()); } + Collections.reverse(list); + + int p = 0; + int size = 0; + for (int i = 0; i < list.size(); i++) { + if (builders == null) { + size = Math.min(maxPageSize, list.size() - i); + builders = new ResultBuilder[elementTypes.size()]; + for (int b = 0; b < builders.length; b++) { + builders[b] = ResultBuilder.resultBuilderFor( + blockFactory, + elementTypes.get(b), + encoders.get(b).toUnsortable(), + channelInKey(sortOrders, b), + size + ); + } + p = 0; + } - Row row = list.get(i); - BytesRef keys = row.keys.bytesRefView(); - for (SortOrder so : sortOrders) { - if (keys.bytes[keys.offset] == so.nul()) { + Row row = list.get(i); + BytesRef keys = row.keys.bytesRefView(); + for (SortOrder so : sortOrders) { + if (keys.bytes[keys.offset] == so.nul()) { + keys.offset++; + keys.length--; + continue; + } keys.offset++; keys.length--; - continue; + builders[so.channel].decodeKey(keys); + } + if (keys.length != 0) { + throw new IllegalArgumentException("didn't read all keys"); } - keys.offset++; - keys.length--; - builders[so.channel].decodeKey(keys); - } - if (keys.length != 0) { - throw new IllegalArgumentException("didn't read all keys"); - } - - BytesRef values = row.values.bytesRefView(); - for (ResultBuilder builder : builders) { - builder.decodeValue(values); - } - if (values.length != 0) { - throw new IllegalArgumentException("didn't read all values"); - } - list.set(i, null); - row.close(); + BytesRef values = row.values.bytesRefView(); + for (ResultBuilder builder : builders) { + builder.decodeValue(values); + } + if (values.length != 0) { + throw new IllegalArgumentException("didn't read all values"); + } - p++; - if (p == size) { - Block[] blocks = new Block[builders.length]; - try { - for (int b = 0; b < blocks.length; b++) { - blocks[b] = builders[b].build(); - } - } finally { - if (blocks[blocks.length - 1] == null) { - Releasables.closeExpectNoException(blocks); + list.set(i, null); + row.close(); + + p++; + if (p == size) { + Block[] blocks = new Block[builders.length]; + try { + for (int b = 0; b < blocks.length; b++) { + blocks[b] = builders[b].build(); + } + } finally { + if (blocks[blocks.length - 1] == null) { + Releasables.closeExpectNoException(blocks); + } } + result.add(new Page(blocks)); + Releasables.closeExpectNoException(builders); + builders = null; } - result.add(new Page(blocks)); - Releasables.closeExpectNoException(builders); - builders = null; } - } - assert builders == null; + assert builders == null; } success = true; return result.iterator(); @@ -586,9 +586,8 @@ public void close() { Releasables.closeExpectNoException(spare, Releasables.wrap(releasables)); } - private static long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(TopNOperator.class) - + RamUsageEstimator.shallowSizeOfInstance(List.class) * 4 - + RamUsageEstimator.shallowSizeOfInstance(Map.class); + private static long SHALLOW_SIZE = RamUsageEstimator.shallowSizeOfInstance(TopNOperator.class) + RamUsageEstimator + .shallowSizeOfInstance(List.class) * 4 + RamUsageEstimator.shallowSizeOfInstance(Map.class); @Override public long ramBytesUsed() { @@ -603,7 +602,8 @@ public long ramBytesUsed() { size += partitions.size() * Partition.SHALLOW_SIZE; size += RamUsageEstimator.alignObjectSize(arrHeader + ref * sortOrders.size()); size += sortOrders.size() * SortOrder.SHALLOW_SIZE; - long ramBytesUsedSum = inputQueues.entrySet().stream() + long ramBytesUsedSum = inputQueues.entrySet() + .stream() .mapToLong(e -> e.getKey().getBytes(Charset.defaultCharset()).length + e.getValue().ramBytesUsed()) .sum(); size += ramBytesUsedSum; @@ -620,7 +620,9 @@ public Status status() { public String toString() { int queueSizeSum = inputQueues.values().stream().mapToInt(Queue::size).sum(); return "TopNOperator[count=" - + queueSizeSum + "/" + topCount + + queueSizeSum + + "/" + + topCount + ", elementTypes=" + elementTypes + ", encoders=" From 2d09714260a05abbd2152b23922b5c3fd58c2ce3 Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Tue, 20 May 2025 11:15:55 +0200 Subject: [PATCH 003/231] chore: deps(ironbank): Bump ubi version to 9.6 (#128171) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Made with ❤️️ by updatecli Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- distribution/docker/src/docker/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/docker/src/docker/Dockerfile b/distribution/docker/src/docker/Dockerfile index 26ebe19022246..4cb4e9fe008f3 100644 --- a/distribution/docker/src/docker/Dockerfile +++ b/distribution/docker/src/docker/Dockerfile @@ -22,7 +22,7 @@ <% if (docker_base == 'iron_bank') { %> ARG BASE_REGISTRY=registry1.dso.mil ARG BASE_IMAGE=ironbank/redhat/ubi/ubi9 -ARG BASE_TAG=9.5 +ARG BASE_TAG=9.6 <% } %> ################################################################################ From 4c0c9b65b66b86965701a30a7057c22917433243 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 20 May 2025 11:35:06 +0200 Subject: [PATCH 004/231] Avoid some uses of ConcurrentLinkedQueue.size() in SharedBlobCacheService (#128119) SharedBlobCacheService keeps track of the free regions in a ConcurrentLinkedQueue. We use its "size()" method in three places outside of tests but unfortunately this is not a constant time operation because of the asynchronous nature of this queue. This change removes two of the uses where we only check if the queue is empty by calling the "isEmpty()" method instead. --- .../blobcache/shared/SharedBlobCacheService.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java index ed7965b85a36a..05b39f32bfc7a 100644 --- a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java +++ b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java @@ -500,6 +500,7 @@ public boolean maybeFetchFullEntry( ActionListener listener ) { int finalRegion = getEndingRegion(length); + // TODO freeRegionCount uses freeRegions.size() which is is NOT a constant-time operation. Can we do better? if (freeRegionCount() < finalRegion) { // Not enough room to download a full file without evicting existing data, so abort listener.onResponse(null); @@ -571,7 +572,7 @@ public void maybeFetchRegion( final Executor fetchExecutor, final ActionListener listener ) { - if (freeRegionCount() < 1 && maybeEvictLeastUsed() == false) { + if (freeRegions.isEmpty() && maybeEvictLeastUsed() == false) { // no free page available and no old enough unused region to be evicted logger.info("No free regions, skipping loading region [{}]", region); listener.onResponse(false); @@ -619,7 +620,7 @@ public void maybeFetchRange( final Executor fetchExecutor, final ActionListener listener ) { - if (freeRegionCount() < 1 && maybeEvictLeastUsed() == false) { + if (freeRegions.isEmpty() && maybeEvictLeastUsed() == false) { // no free page available and no old enough unused region to be evicted logger.info("No free regions, skipping loading region [{}]", region); listener.onResponse(false); @@ -671,7 +672,11 @@ private static void throwAlreadyClosed(String message) { throw new AlreadyClosedException(message); } - // used by tests + /** + * NOTE: Method is package private mostly to allow checking the number of fee regions in tests. + * However, it is also used by {@link SharedBlobCacheService#maybeFetchFullEntry} but we should try + * to move away from that because calling "size" on a ConcurrentLinkedQueue is not a constant time operation. + */ int freeRegionCount() { return freeRegions.size(); } From 4e7b99cc739d97bb627d4dc64ec5e45469f82e53 Mon Sep 17 00:00:00 2001 From: Jeremy Dahlgren Date: Tue, 20 May 2025 07:25:19 -0400 Subject: [PATCH 005/231] Add cancellation support in TransportGetAllocationStatsAction (#127371) Replaces the use of a SingleResultDeduplicator by refactoring the cache as a subclass of CancellableSingleObjectCache. Refactored the AllocationStatsService and NodeAllocationStatsAndWeightsCalculator to accept the Runnable used to test for cancellation. Closes #123248 --- docs/changelog/127371.yaml | 6 + .../TransportGetAllocationStatsAction.java | 90 +++++++----- .../allocation/AllocationStatsService.java | 9 ++ ...deAllocationStatsAndWeightsCalculator.java | 2 + .../DesiredBalanceShardsAllocator.java | 8 ++ .../util/CancellableSingleObjectCache.java | 16 ++- ...ransportGetAllocationStatsActionTests.java | 128 +++++++++++++----- .../AllocationStatsServiceTests.java | 6 +- .../CancellableSingleObjectCacheTests.java | 51 ++++++- .../cluster/ESAllocationTestCase.java | 2 + 10 files changed, 241 insertions(+), 77 deletions(-) create mode 100644 docs/changelog/127371.yaml diff --git a/docs/changelog/127371.yaml b/docs/changelog/127371.yaml new file mode 100644 index 0000000000000..10f5f17243193 --- /dev/null +++ b/docs/changelog/127371.yaml @@ -0,0 +1,6 @@ +pr: 127371 +summary: Add cancellation support in `TransportGetAllocationStatsAction` +area: Allocation +type: feature +issues: + - 123248 diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportGetAllocationStatsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportGetAllocationStatsAction.java index eecbb3525bda9..e46762c9e97ab 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportGetAllocationStatsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/allocation/TransportGetAllocationStatsAction.java @@ -15,7 +15,6 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.ActionType; -import org.elasticsearch.action.SingleResultDeduplicator; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequestParameters.Metric; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.SubscribableListener; @@ -31,10 +30,12 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.util.CancellableSingleObjectCache; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.injection.guice.Inject; +import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; @@ -43,7 +44,8 @@ import java.io.IOException; import java.util.EnumSet; import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.Executor; +import java.util.function.BooleanSupplier; public class TransportGetAllocationStatsAction extends TransportMasterNodeReadAction< TransportGetAllocationStatsAction.Request, @@ -62,7 +64,6 @@ public class TransportGetAllocationStatsAction extends TransportMasterNodeReadAc ); private final AllocationStatsCache allocationStatsCache; - private final SingleResultDeduplicator> allocationStatsSupplier; private final DiskThresholdSettings diskThresholdSettings; @Inject @@ -85,21 +86,7 @@ public TransportGetAllocationStatsAction( // very cheaply. EsExecutors.DIRECT_EXECUTOR_SERVICE ); - final var managementExecutor = threadPool.executor(ThreadPool.Names.MANAGEMENT); - this.allocationStatsCache = new AllocationStatsCache(threadPool, DEFAULT_CACHE_TTL); - this.allocationStatsSupplier = new SingleResultDeduplicator<>(threadPool.getThreadContext(), l -> { - final var cachedStats = allocationStatsCache.get(); - if (cachedStats != null) { - l.onResponse(cachedStats); - return; - } - - managementExecutor.execute(ActionRunnable.supply(l, () -> { - final var stats = allocationStatsService.stats(); - allocationStatsCache.put(stats); - return stats; - })); - }); + this.allocationStatsCache = new AllocationStatsCache(threadPool, allocationStatsService, DEFAULT_CACHE_TTL); this.diskThresholdSettings = new DiskThresholdSettings(clusterService.getSettings(), clusterService.getClusterSettings()); clusterService.getClusterSettings().initializeAndWatch(CACHE_TTL_SETTING, this.allocationStatsCache::setTTL); } @@ -118,8 +105,11 @@ protected void doExecute(Task task, Request request, ActionListener li protected void masterOperation(Task task, Request request, ClusterState state, ActionListener listener) throws Exception { // NB we are still on a transport thread here - if adding more functionality here make sure to fork to a different pool + assert task instanceof CancellableTask; + final var cancellableTask = (CancellableTask) task; + final SubscribableListener> allocationStatsStep = request.metrics().contains(Metric.ALLOCATIONS) - ? SubscribableListener.newForked(allocationStatsSupplier::execute) + ? SubscribableListener.newForked(l -> allocationStatsCache.get(cancellableTask::isCancelled, l)) : SubscribableListener.newSucceeded(Map.of()); allocationStatsStep.andThenApply( @@ -167,6 +157,11 @@ public EnumSet metrics() { public ActionRequestValidationException validate() { return null; } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new CancellableTask(id, type, action, "", parentTaskId, headers); + } } public static class Response extends ActionResponse { @@ -209,39 +204,60 @@ public DiskThresholdSettings getDiskThresholdSettings() { } } - private record CachedAllocationStats(Map stats, long timestampMillis) {} - - private static class AllocationStatsCache { + private static class AllocationStatsCache extends CancellableSingleObjectCache> { private volatile long ttlMillis; private final ThreadPool threadPool; - private final AtomicReference cachedStats; + private final Executor executor; + private final AllocationStatsService allocationStatsService; - AllocationStatsCache(ThreadPool threadPool, TimeValue ttl) { + AllocationStatsCache(ThreadPool threadPool, AllocationStatsService allocationStatsService, TimeValue ttl) { + super(threadPool.getThreadContext()); this.threadPool = threadPool; - this.cachedStats = new AtomicReference<>(); + this.executor = threadPool.executor(ThreadPool.Names.MANAGEMENT); + this.allocationStatsService = allocationStatsService; setTTL(ttl); } void setTTL(TimeValue ttl) { ttlMillis = ttl.millis(); - if (ttlMillis == 0L) { - cachedStats.set(null); - } + clearCacheIfDisabled(); } - Map get() { - if (ttlMillis == 0L) { - return null; + void get(BooleanSupplier isCancelled, ActionListener> listener) { + get(threadPool.relativeTimeInMillis(), isCancelled, listener); + } + + @Override + protected void refresh( + Long aLong, + Runnable ensureNotCancelled, + BooleanSupplier supersedeIfStale, + ActionListener> listener + ) { + if (supersedeIfStale.getAsBoolean() == false) { + executor.execute( + ActionRunnable.supply( + // If caching is disabled the item is only cached long enough to prevent duplicate concurrent requests. + ActionListener.runBefore(listener, this::clearCacheIfDisabled), + () -> allocationStatsService.stats(ensureNotCancelled) + ) + ); } + } - // We don't set the atomic ref to null here upon expiration since we know it is about to be replaced with a fresh instance. - final var stats = cachedStats.get(); - return stats == null || threadPool.relativeTimeInMillis() - stats.timestampMillis > ttlMillis ? null : stats.stats; + @Override + protected Long getKey(Long timestampMillis) { + return timestampMillis; + } + + @Override + protected boolean isFresh(Long currentKey, Long newKey) { + return ttlMillis == 0 || newKey - currentKey <= ttlMillis; } - void put(Map stats) { - if (ttlMillis > 0L) { - cachedStats.set(new CachedAllocationStats(stats, threadPool.relativeTimeInMillis())); + private void clearCacheIfDisabled() { + if (ttlMillis == 0) { + clearCurrentCachedItem(); } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationStatsService.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationStatsService.java index 926a6926c9aea..f31ddd36a2e31 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationStatsService.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationStatsService.java @@ -47,6 +47,14 @@ public AllocationStatsService( * Returns a map of node IDs to node allocation stats. */ public Map stats() { + return stats(() -> {}); + } + + /** + * Returns a map of node IDs to node allocation stats, promising to execute the provided {@link Runnable} during the computation to + * test for cancellation. + */ + public Map stats(Runnable ensureNotCancelled) { assert Transports.assertNotTransportThread("too expensive for a transport worker"); var clusterState = clusterService.state(); @@ -54,6 +62,7 @@ public Map stats() { clusterState.metadata(), clusterState.getRoutingNodes(), clusterInfoService.getClusterInfo(), + ensureNotCancelled, desiredBalanceSupplier.get() ); return nodesStatsAndWeights.entrySet() diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/NodeAllocationStatsAndWeightsCalculator.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/NodeAllocationStatsAndWeightsCalculator.java index 21e006f76b1d1..c92a65543d0ff 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/NodeAllocationStatsAndWeightsCalculator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/NodeAllocationStatsAndWeightsCalculator.java @@ -58,6 +58,7 @@ public Map nodesAllocationStatsAndWeights( Metadata metadata, RoutingNodes routingNodes, ClusterInfo clusterInfo, + Runnable ensureNotCancelled, @Nullable DesiredBalance desiredBalance ) { if (metadata.hasAnyIndices()) { @@ -78,6 +79,7 @@ public Map nodesAllocationStatsAndWeights( long forecastedDiskUsage = 0; long currentDiskUsage = 0; for (ShardRouting shardRouting : node) { + ensureNotCancelled.run(); if (shardRouting.relocating()) { // Skip the shard if it is moving off this node. The node running recovery will count it. continue; diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java index e8d8d509282ab..515da761d8696 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocator.java @@ -391,6 +391,13 @@ public void resetDesiredBalance() { resetCurrentDesiredBalance = true; } + /** + * Used as the argument for the {@code ensureNotCancelled} {@code Runnable} when calling the + * {@code nodeAllocationStatsAndWeightsCalculator} since there is no cancellation mechanism when called from + * {@code updateDesireBalanceMetrics()}. + */ + private static final Runnable NEVER_CANCELLED = () -> {}; + private void updateDesireBalanceMetrics( DesiredBalance desiredBalance, RoutingAllocation routingAllocation, @@ -400,6 +407,7 @@ private void updateDesireBalanceMetrics( routingAllocation.metadata(), routingAllocation.routingNodes(), routingAllocation.clusterInfo(), + NEVER_CANCELLED, desiredBalance ); Map filteredNodeAllocationStatsAndWeights = diff --git a/server/src/main/java/org/elasticsearch/common/util/CancellableSingleObjectCache.java b/server/src/main/java/org/elasticsearch/common/util/CancellableSingleObjectCache.java index fa8ec26bbad2c..2e33dc47f0d60 100644 --- a/server/src/main/java/org/elasticsearch/common/util/CancellableSingleObjectCache.java +++ b/server/src/main/java/org/elasticsearch/common/util/CancellableSingleObjectCache.java @@ -100,6 +100,13 @@ protected boolean isFresh(Key currentKey, Key newKey) { return currentKey.equals(newKey); } + /** + * Sets the currently cached item reference to {@code null}, which will result in a {@code refresh()} on the next {@code get()} call. + */ + protected final void clearCurrentCachedItem() { + this.currentCachedItemRef.set(null); + } + /** * Start a retrieval for the value associated with the given {@code input}, and pass it to the given {@code listener}. *

@@ -110,7 +117,8 @@ protected boolean isFresh(Key currentKey, Key newKey) { * * @param input The input to compute the desired value, converted to a {@link Key} to determine if the value that's currently * cached or pending is fresh enough. - * @param isCancelled Returns {@code true} if the listener no longer requires the value being computed. + * @param isCancelled Returns {@code true} if the listener no longer requires the value being computed. The listener is expected to be + * completed as soon as possible when cancellation is detected. * @param listener The listener to notify when the desired value becomes available. */ public final void get(Input input, BooleanSupplier isCancelled, ActionListener listener) { @@ -230,11 +238,15 @@ boolean addListener(ActionListener listener, BooleanSupplier isCancelled) ActionListener.completeWith(listener, future::actionResult); } else { // Refresh is still pending; it's not cancelled because there are still references. - future.addListener(ContextPreservingActionListener.wrapPreservingContext(listener, threadContext)); + final var cancellableListener = ActionListener.notifyOnce( + ContextPreservingActionListener.wrapPreservingContext(listener, threadContext) + ); + future.addListener(cancellableListener); final AtomicBoolean released = new AtomicBoolean(); cancellationChecks.add(() -> { if (released.get() == false && isCancelled.getAsBoolean() && released.compareAndSet(false, true)) { decRef(); + cancellableListener.onFailure(new TaskCancelledException("task cancelled")); } }); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/TransportGetAllocationStatsActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/TransportGetAllocationStatsActionTests.java index d60ac5ca47f6d..133cf5d648611 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/TransportGetAllocationStatsActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/allocation/TransportGetAllocationStatsActionTests.java @@ -23,7 +23,9 @@ import org.elasticsearch.core.CheckedConsumer; import org.elasticsearch.core.TimeValue; import org.elasticsearch.node.Node; -import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.TaskCancelHelper; +import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ClusterServiceUtils; @@ -34,6 +36,7 @@ import org.elasticsearch.transport.TransportService; import org.junit.After; import org.junit.Before; +import org.mockito.ArgumentCaptor; import java.util.EnumSet; import java.util.List; @@ -48,6 +51,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.not; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -126,26 +130,22 @@ public void testReturnsOnlyRequestedStats() throws Exception { EnumSet.allOf(Metric.class), EnumSet.copyOf(randomSubsetOf(between(1, Metric.values().length), EnumSet.allOf(Metric.class))) )) { - var request = new TransportGetAllocationStatsAction.Request( - TimeValue.ONE_MINUTE, - new TaskId(randomIdentifier(), randomNonNegativeLong()), - metrics - ); + var request = new TransportGetAllocationStatsAction.Request(TimeValue.ONE_MINUTE, TaskId.EMPTY_TASK_ID, metrics); - when(allocationStatsService.stats()).thenReturn( + when(allocationStatsService.stats(any())).thenReturn( Map.of(randomIdentifier(), NodeAllocationStatsTests.randomNodeAllocationStats()) ); var future = new PlainActionFuture(); - action.masterOperation(mock(Task.class), request, ClusterState.EMPTY_STATE, future); + action.masterOperation(getTask(), request, ClusterState.EMPTY_STATE, future); var response = future.get(); if (metrics.contains(Metric.ALLOCATIONS)) { assertThat(response.getNodeAllocationStats(), not(anEmptyMap())); - verify(allocationStatsService, times(++expectedNumberOfStatsServiceCalls)).stats(); + verifyAllocationStatsServiceNumCallsEqualTo(++expectedNumberOfStatsServiceCalls); } else { assertThat(response.getNodeAllocationStats(), anEmptyMap()); - verify(allocationStatsService, times(expectedNumberOfStatsServiceCalls)).stats(); + verifyAllocationStatsServiceNumCallsEqualTo(expectedNumberOfStatsServiceCalls); } if (metrics.contains(Metric.FS)) { @@ -160,7 +160,7 @@ public void testDeduplicatesStatsComputations() throws InterruptedException { disableAllocationStatsCache(); final var requestCounter = new AtomicInteger(); final var isExecuting = new AtomicBoolean(); - when(allocationStatsService.stats()).thenAnswer(invocation -> { + when(allocationStatsService.stats(any())).thenAnswer(invocation -> { try { assertTrue(isExecuting.compareAndSet(false, true)); assertThat(Thread.currentThread().getName(), containsString("[management]")); @@ -180,16 +180,7 @@ public void testDeduplicatesStatsComputations() throws InterruptedException { final var minRequestIndex = requestCounter.get(); final TransportGetAllocationStatsAction.Response response = safeAwait( - l -> action.masterOperation( - mock(Task.class), - new TransportGetAllocationStatsAction.Request( - TEST_REQUEST_TIMEOUT, - TaskId.EMPTY_TASK_ID, - EnumSet.of(Metric.ALLOCATIONS) - ), - ClusterState.EMPTY_STATE, - l - ) + l -> action.masterOperation(getTask(), getRequest(), ClusterState.EMPTY_STATE, l) ); final var requestIndex = Integer.valueOf(response.getNodeAllocationStats().keySet().iterator().next()); @@ -203,6 +194,65 @@ public void testDeduplicatesStatsComputations() throws InterruptedException { } } + public void testAllTasksCancelledCacheEnabled() throws InterruptedException { + runTestWithCancelledTasks(between(2, 10), false, true); + } + + public void testAllTasksCancelledCacheDisabled() throws InterruptedException { + runTestWithCancelledTasks(between(2, 10), true, true); + } + + public void testSomeTasksCancelledCacheEnabled() throws InterruptedException { + runTestWithCancelledTasks(between(2, 10), false, false); + } + + public void testSomeTasksCancelledCacheDisabled() throws InterruptedException { + runTestWithCancelledTasks(between(2, 10), true, false); + } + + private void runTestWithCancelledTasks(final int numThreads, final boolean cacheDisabled, final boolean cancelAllTasks) + throws InterruptedException { + if (cacheDisabled) { + disableAllocationStatsCache(); + } + final var isExecuting = new AtomicBoolean(); + final var ensureNotCancelledCaptor = ArgumentCaptor.forClass(Runnable.class); + final var tasks = new CancellableTask[numThreads]; + final var cancellations = new boolean[numThreads]; + final var stats = Map.of(randomIdentifier(), NodeAllocationStatsTests.randomNodeAllocationStats()); + + when(allocationStatsService.stats(ensureNotCancelledCaptor.capture())).thenAnswer(invocation -> { + try { + assertTrue(isExecuting.compareAndSet(false, true)); + for (int i = 0; i < numThreads; ++i) { + if (cancellations[i]) { + TaskCancelHelper.cancel(tasks[i], "cancelled"); + } + } + ensureNotCancelledCaptor.getValue().run(); + return stats; + } finally { + Thread.yield(); + assertTrue(isExecuting.compareAndSet(true, false)); + } + }); + + ESTestCase.startInParallel(numThreads, threadNumber -> { + tasks[threadNumber] = getTask(); + cancellations[threadNumber] = cancelAllTasks || randomBoolean(); + final ActionListener listener = ActionListener.wrap(response -> { + assertSame(stats, response.getNodeAllocationStats()); + }, e -> { + if (e instanceof TaskCancelledException) { + assertTrue("got an unexpected cancellation exception for thread " + threadNumber, cancellations[threadNumber]); + } else { + fail(e); + } + }); + ActionListener.run(listener, l -> action.masterOperation(tasks[threadNumber], getRequest(), ClusterState.EMPTY_STATE, l)); + }); + } + public void testGetStatsWithCachingEnabled() throws Exception { final AtomicReference> allocationStats = new AtomicReference<>(); @@ -211,17 +261,11 @@ public void testGetStatsWithCachingEnabled() throws Exception { final Runnable resetExpectedAllocationStats = () -> { final var stats = Map.of(randomIdentifier(), NodeAllocationStatsTests.randomNodeAllocationStats()); allocationStats.set(stats); - when(allocationStatsService.stats()).thenReturn(stats); + when(allocationStatsService.stats(any())).thenReturn(stats); }; final CheckedConsumer, Exception> threadTask = l -> { - final var request = new TransportGetAllocationStatsAction.Request( - TEST_REQUEST_TIMEOUT, - new TaskId(randomIdentifier(), randomNonNegativeLong()), - EnumSet.of(Metric.ALLOCATIONS) - ); - - action.masterOperation(mock(Task.class), request, ClusterState.EMPTY_STATE, l.map(response -> { + action.masterOperation(getTask(), getRequest(), ClusterState.EMPTY_STATE, l.map(response -> { assertSame("Expected the cached allocation stats to be returned", response.getNodeAllocationStats(), allocationStats.get()); return null; })); @@ -230,12 +274,12 @@ public void testGetStatsWithCachingEnabled() throws Exception { // Initial cache miss, all threads should get the same value. resetExpectedAllocationStats.run(); ESTestCase.startInParallel(between(1, 5), threadNumber -> safeAwait(threadTask)); - verify(allocationStatsService, times(++numExpectedAllocationStatsServiceCalls)).stats(); + verifyAllocationStatsServiceNumCallsEqualTo(++numExpectedAllocationStatsServiceCalls); // Advance the clock to a time less than or equal to the TTL and verify we still get the cached stats. threadPool.setCurrentTimeInMillis(startTimeMillis + between(0, (int) allocationStatsCacheTTL.millis())); ESTestCase.startInParallel(between(1, 5), threadNumber -> safeAwait(threadTask)); - verify(allocationStatsService, times(numExpectedAllocationStatsServiceCalls)).stats(); + verifyAllocationStatsServiceNumCallsEqualTo(numExpectedAllocationStatsServiceCalls); // Force the cached stats to expire. threadPool.setCurrentTimeInMillis(startTimeMillis + allocationStatsCacheTTL.getMillis() + 1); @@ -243,20 +287,34 @@ public void testGetStatsWithCachingEnabled() throws Exception { // Expect a single call to the stats service on the cache miss. resetExpectedAllocationStats.run(); ESTestCase.startInParallel(between(1, 5), threadNumber -> safeAwait(threadTask)); - verify(allocationStatsService, times(++numExpectedAllocationStatsServiceCalls)).stats(); + verifyAllocationStatsServiceNumCallsEqualTo(++numExpectedAllocationStatsServiceCalls); // Update the TTL setting to disable the cache, we expect a service call each time. setAllocationStatsCacheTTL(TimeValue.ZERO); safeAwait(threadTask); safeAwait(threadTask); numExpectedAllocationStatsServiceCalls += 2; - verify(allocationStatsService, times(numExpectedAllocationStatsServiceCalls)).stats(); + verifyAllocationStatsServiceNumCallsEqualTo(numExpectedAllocationStatsServiceCalls); // Re-enable the cache, only one thread should call the stats service. - setAllocationStatsCacheTTL(TimeValue.timeValueMinutes(5)); + final var newTTL = TimeValue.timeValueMinutes(5); + setAllocationStatsCacheTTL(newTTL); + threadPool.setCurrentTimeInMillis(threadPool.relativeTimeInMillis() + newTTL.getMillis() + 1); resetExpectedAllocationStats.run(); ESTestCase.startInParallel(between(1, 5), threadNumber -> safeAwait(threadTask)); - verify(allocationStatsService, times(++numExpectedAllocationStatsServiceCalls)).stats(); + verifyAllocationStatsServiceNumCallsEqualTo(++numExpectedAllocationStatsServiceCalls); + } + + private void verifyAllocationStatsServiceNumCallsEqualTo(int numCalls) { + verify(allocationStatsService, times(numCalls)).stats(any()); + } + + private static TransportGetAllocationStatsAction.Request getRequest() { + return new TransportGetAllocationStatsAction.Request(TEST_REQUEST_TIMEOUT, TaskId.EMPTY_TASK_ID, EnumSet.of(Metric.ALLOCATIONS)); + } + + private static CancellableTask getTask() { + return new CancellableTask(randomLong(), "type", "action", "desc", null, Map.of()); } private static class ControlledRelativeTimeThreadPool extends ThreadPool { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationStatsServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationStatsServiceTests.java index cf2653bc6c559..4a07b837b08af 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationStatsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationStatsServiceTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.telemetry.TelemetryProvider; import org.elasticsearch.test.ClusterServiceUtils; @@ -92,7 +93,7 @@ public void testShardStats() { ) ); assertThat( - service.stats(), + service.stats(() -> {}), allOf( aMapWithSize(1), hasEntry( @@ -101,6 +102,9 @@ public void testShardStats() { ) ) ); + + // Verify that the ensureNotCancelled Runnable is tested during execution. + assertThrows(TaskCancelledException.class, () -> service.stats(() -> { throw new TaskCancelledException("cancelled"); })); } } diff --git a/server/src/test/java/org/elasticsearch/common/util/CancellableSingleObjectCacheTests.java b/server/src/test/java/org/elasticsearch/common/util/CancellableSingleObjectCacheTests.java index ecb74009682d3..8760b0d9faa08 100644 --- a/server/src/test/java/org/elasticsearch/common/util/CancellableSingleObjectCacheTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/CancellableSingleObjectCacheTests.java @@ -101,8 +101,35 @@ public void testListenerCompletedByRefreshEvenIfDiscarded() throws ExecutionExce testCache.completeNextRefresh("foo", 1); assertThat(future2.result(), equalTo(1)); - // ... and the original listener is also completed successfully - assertThat(future1.result(), sameInstance(future2.result())); + // We expect the first listener to have been completed with a cancellation exception when detected in the ensureNotCancelled() call. + assertTrue(future1.isDone()); + expectThrows(ExecutionException.class, TaskCancelledException.class, future1::result); + } + + public void testBothListenersReceiveTaskCancelledExceptionWhenBothSupersededAndNewTasksAreCancelled() { + final TestCache testCache = new TestCache(); + + // This computation is superseded and then cancelled. + final AtomicBoolean isCancelled = new AtomicBoolean(); + final TestFuture future1 = new TestFuture(); + testCache.get("foo", isCancelled::get, future1); + testCache.assertPendingRefreshes(1); + + // A second get() call that supersedes the original refresh and starts another one, but will be cancelled as well. + final TestFuture future2 = new TestFuture(); + testCache.get("bar", isCancelled::get, future2); + testCache.assertPendingRefreshes(2); + + testCache.assertNextRefreshCancelled(); + assertFalse(future1.isDone()); + testCache.assertPendingRefreshes(1); + assertFalse(future2.isDone()); + + isCancelled.set(true); + // This next refresh should also fail with a cancellation exception. + testCache.completeNextRefresh("bar", 1); + expectThrows(ExecutionException.class, TaskCancelledException.class, future1::result); + expectThrows(ExecutionException.class, TaskCancelledException.class, future2::result); } public void testListenerCompletedWithCancellationExceptionIfRefreshCancelled() throws ExecutionException { @@ -421,6 +448,26 @@ protected String getKey(String s) { expectThrows(ExecutionException.class, TaskCancelledException.class, cancelledFuture::result); } + public void testClearCurrentCachedItem() throws ExecutionException { + final TestCache testCache = new TestCache(); + + // The first get() calls the refresh function. + final TestFuture future0 = new TestFuture(); + testCache.get("foo", () -> false, future0); + testCache.assertPendingRefreshes(1); + testCache.completeNextRefresh("foo", 1); + assertThat(future0.result(), equalTo(1)); + + testCache.clearCurrentCachedItem(); + + // The second get() with a matching key will execute a refresh since the cached item was cleared. + final TestFuture future1 = new TestFuture(); + testCache.get("foo", () -> false, future1); + testCache.assertPendingRefreshes(1); + testCache.completeNextRefresh("foo", 2); + assertThat(future1.result(), equalTo(2)); + } + private static final ThreadContext testThreadContext = new ThreadContext(Settings.EMPTY); private static class TestCache extends CancellableSingleObjectCache { diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java index e6a3f7664bd28..8a49db652374e 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java @@ -453,8 +453,10 @@ public Map nodesAllocationStatsAndWeights( Metadata metadata, RoutingNodes routingNodes, ClusterInfo clusterInfo, + Runnable ensureNotCancelled, @Nullable DesiredBalance desiredBalance ) { + ensureNotCancelled.run(); return Map.of(); } }; From 265848e5ababcc6f0f3913265eab87973a5b235e Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Tue, 20 May 2025 21:52:58 +1000 Subject: [PATCH 006/231] [Test] Fix testContentRangeValidation (#128188) Ensure sufficent bytes for the start position. --- .../repositories/s3/S3RetryingInputStreamTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RetryingInputStreamTests.java b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RetryingInputStreamTests.java index e735bae25f662..c4cd31f444144 100644 --- a/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RetryingInputStreamTests.java +++ b/modules/repository-s3/src/test/java/org/elasticsearch/repositories/s3/S3RetryingInputStreamTests.java @@ -120,7 +120,7 @@ public void testReadAfterBlobLengthThrowsRequestedRangeNotSatisfiedException() t } public void testContentRangeValidation() throws IOException { - final byte[] bytes = randomByteArrayOfLength(between(100, 200)); + final byte[] bytes = randomByteArrayOfLength(between(101, 200)); final int position = between(0, 100); final int length = between(1, 100); try (var stream = createInputStream(bytes, position, length)) { From 557f1f12b353950d315f5dda4d391e8f8a06b933 Mon Sep 17 00:00:00 2001 From: kanoshiou <73424326+kanoshiou@users.noreply.github.com> Date: Tue, 20 May 2025 20:06:24 +0800 Subject: [PATCH 007/231] ESQL: Fix alias removal in regex extraction with `JOIN` (#127687) * Disallow removal of regex extracted fields --------- Co-authored-by: Andrei Stefan Co-authored-by: elasticsearchmachine --- docs/changelog/127687.yaml | 6 ++ .../rest/generative/GenerativeRestTest.java | 1 - .../src/main/resources/lookup-join.csv-spec | 69 +++++++++++++++++++ .../xpack/esql/action/EsqlCapabilities.java | 8 ++- .../xpack/esql/session/EsqlSession.java | 18 ++--- .../session/IndexResolverFieldNamesTests.java | 47 +++++++++++++ 6 files changed, 139 insertions(+), 10 deletions(-) create mode 100644 docs/changelog/127687.yaml diff --git a/docs/changelog/127687.yaml b/docs/changelog/127687.yaml new file mode 100644 index 0000000000000..e053c4a31ad2e --- /dev/null +++ b/docs/changelog/127687.yaml @@ -0,0 +1,6 @@ +pr: 127687 +summary: "ESQL: Fix alias removal in regex extraction with JOIN" +area: ES|QL +type: bug +issues: + - 127467 diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/generative/GenerativeRestTest.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/generative/GenerativeRestTest.java index e0826ca710eb7..9e07e8c162432 100644 --- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/generative/GenerativeRestTest.java +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/generative/GenerativeRestTest.java @@ -51,7 +51,6 @@ public abstract class GenerativeRestTest extends ESRestTestCase { "Unknown column \\[\\]", // https://github.com/elastic/elasticsearch/issues/121741, "Plan \\[ProjectExec\\[\\[.* optimized incorrectly due to missing references", // https://github.com/elastic/elasticsearch/issues/125866 "optimized incorrectly due to missing references", // https://github.com/elastic/elasticsearch/issues/116781 - "Unknown column", // https://github.com/elastic/elasticsearch/issues/127467 "only supports KEYWORD or TEXT values", // https://github.com/elastic/elasticsearch/issues/127468 "The incoming YAML document exceeds the limit:" // still to investigate, but it seems to be specific to the test framework ); 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 9aaf5552c7aed..f36d42de96c77 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 @@ -1673,3 +1673,72 @@ null | Milky Way | Marunouchi null | null | null null | null | null ; + + +joinMaskingRegex +// https://github.com/elastic/elasticsearch/issues/127467 +required_capability: union_types +required_capability: join_lookup_v12 +required_capability: fix_join_masking_regex_extract +from books,message_*,ul* +| enrich languages_policy on status +| drop `language_name`, `bytes_out`, `id`, id +| dissect book_no "%{type}" +| dissect author.keyword "%{HZicfARaID}" +| mv_expand `status` +| sort HZicfARaID, year DESC NULLS LAST, publisher DESC NULLS FIRST, description DESC, type NULLS LAST, message ASC NULLS LAST, title NULLS FIRST, status NULLS LAST +| enrich languages_policy on book_no +| grok message "%{WORD:DiLNyZKNDu}" +| limit 7972 +| rename year as language_code +| lookup join languages_lookup on language_code +| limit 13966 +| stats rcyIZnSOb = min(language_code), `ratings` = min(@timestamp), dgDxwMeFYrD = count(`@timestamp`), ifyZfXigqVN = count(*), qTXdrzSpY = min(language_code) by author.keyword +| rename author.keyword as message +| lookup join message_types_lookup on message +| stats `ratings` = count(*) by type +| stats `type` = count(type), `ratings` = count(*) +| keep `ratings`, ratings +; + +ratings:long +1 +; + +joinMaskingDissect +// https://github.com/elastic/elasticsearch/issues/127467 +required_capability: join_lookup_v12 +required_capability: fix_join_masking_regex_extract +from sample_data +| dissect message "%{type}" +| drop type +| lookup join message_types_lookup on message +| stats count = count(*) by type +| keep count +| sort count +; +count:long +1 +3 +3 +; + + +joinMaskingGrok +// https://github.com/elastic/elasticsearch/issues/127467 +required_capability: join_lookup_v12 +required_capability: fix_join_masking_regex_extract +from sample_data +| grok message "%{WORD:type}" +| drop type +| lookup join message_types_lookup on message +| stats max = max(event_duration) by type +| keep max +| sort max +; + +max:long +1232382 +3450233 +8268153 +; 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 86649e7d4a9eb..4a45ddeeecba3 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 @@ -1085,7 +1085,13 @@ public enum Cap { /** * Full text functions in STATS */ - FULL_TEXT_FUNCTIONS_IN_STATS_WHERE; + FULL_TEXT_FUNCTIONS_IN_STATS_WHERE, + + /** + * During resolution (pre-analysis) we have to consider that joins can override regex extracted values + * see ES|QL: pruning of JOINs leads to missing fields #127467 + */ + FIX_JOIN_MASKING_REGEX_EXTRACT; private final boolean enabled; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index 91c8547dc1bfb..4f0e06dc8d8f9 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -41,6 +41,8 @@ import org.elasticsearch.xpack.esql.core.expression.Expressions; import org.elasticsearch.xpack.esql.core.expression.FoldContext; import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute; +import org.elasticsearch.xpack.esql.core.expression.NamedExpression; +import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute; import org.elasticsearch.xpack.esql.core.expression.UnresolvedAttribute; import org.elasticsearch.xpack.esql.core.expression.UnresolvedStar; import org.elasticsearch.xpack.esql.core.util.Holder; @@ -606,11 +608,7 @@ static PreAnalysisResult fieldNames(LogicalPlan parsed, Set enrichPolicy parsed.forEachDown(p -> {// go over each plan top-down if (p instanceof RegexExtract re) { // for Grok and Dissect - // remove other down-the-tree references to the extracted fields - for (Attribute extracted : re.extractedFields()) { - referencesBuilder.removeIf(attr -> matchByName(attr, extracted.name(), false)); - } - // but keep the inputs needed by Grok/Dissect + // keep the inputs needed by Grok/Dissect referencesBuilder.addAll(re.input().references()); } else if (p instanceof Enrich enrich) { AttributeSet enrichFieldRefs = Expressions.references(enrich.enrichFields()); @@ -665,15 +663,19 @@ static PreAnalysisResult fieldNames(LogicalPlan parsed, Set enrichPolicy // remove any already discovered UnresolvedAttributes that are in fact aliases defined later down in the tree // for example "from test | eval x = salary | stats max = max(x) by gender" // remove the UnresolvedAttribute "x", since that is an Alias defined in "eval" + // also remove other down-the-tree references to the extracted fields from "grok" and "dissect" AttributeSet planRefs = p.references(); Set fieldNames = planRefs.names(); - p.forEachExpressionDown(Alias.class, alias -> { + p.forEachExpressionDown(NamedExpression.class, ne -> { + if ((ne instanceof Alias || ne instanceof ReferenceAttribute) == false) { + return; + } // do not remove the UnresolvedAttribute that has the same name as its alias, ie "rename id AS id" // or the UnresolvedAttributes that are used in Functions that have aliases "STATS id = MAX(id)" - if (fieldNames.contains(alias.name())) { + if (fieldNames.contains(ne.name())) { return; } - referencesBuilder.removeIf(attr -> matchByName(attr, alias.name(), shadowingRefsBuilder.contains(attr))); + referencesBuilder.removeIf(attr -> matchByName(attr, ne.name(), shadowingRefsBuilder.contains(attr))); }); } }); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java index 539036d7d1cf0..64f9404ca2a8a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java @@ -1341,6 +1341,53 @@ public void testDissectOverwriteName() { assertThat(fieldNames, equalTo(Set.of("emp_no", "emp_no.*", "first_name", "first_name.*"))); } + /** + * Fix alias removal in regex extraction with JOIN + * @see ES|QL: pruning of JOINs leads to missing fields + */ + public void testAvoidGrokAttributesRemoval() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V12.isEnabled()); + Set fieldNames = fieldNames(""" + from message_types + | eval type = 1 + | lookup join message_types_lookup on message + | drop message + | grok type "%{WORD:b}" + | stats x = max(b) + | keep x""", Set.of()); + assertThat(fieldNames, equalTo(Set.of("message", "x", "x.*", "message.*"))); + } + + public void testAvoidGrokAttributesRemoval2() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V12.isEnabled()); + Set fieldNames = fieldNames(""" + from sample_data + | dissect message "%{type}" + | drop type + | lookup join message_types_lookup on message + | stats count = count(*) by type + | keep count + | sort count""", Set.of()); + assertThat(fieldNames, equalTo(Set.of("type", "message", "count", "message.*", "type.*", "count.*"))); + } + + public void testAvoidGrokAttributesRemoval3() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V12.isEnabled()); + Set fieldNames = fieldNames(""" + from sample_data + | grok message "%{WORD:type}" + | drop type + | lookup join message_types_lookup on message + | stats max = max(event_duration) by type + | keep max + | sort max""", Set.of()); + assertThat( + fieldNames, + equalTo(Set.of("type", "event_duration", "message", "max", "event_duration.*", "message.*", "type.*", "max.*")) + ); + + } + public void testEnrichOnDefaultField() { Set fieldNames = fieldNames(""" from employees From 07aff0a7394579cdb5bfcbd5d4f91c99d295c4a7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Iv=C3=A1n=20Cea=20Fontenla?= Date: Tue, 20 May 2025 14:59:48 +0200 Subject: [PATCH 008/231] ESQL: Limit Replace function memory usage (#127924) The Replace string result limit was fixed to 1MB, same as Repeat --- docs/changelog/127924.yaml | 5 + .../function/scalar/ScalarFunction.java | 9 ++ .../string/ReplaceConstantEvaluator.java | 5 +- .../scalar/string/ReplaceEvaluator.java | 5 +- .../function/scalar/string/Repeat.java | 7 +- .../function/scalar/string/Replace.java | 50 ++++++- .../AbstractScalarFunctionTestCase.java | 1 + .../scalar/string/RepeatStaticTests.java | 5 +- .../scalar/string/ReplaceStaticTests.java | 139 ++++++++++++++++++ .../function/scalar/string/ReplaceTests.java | 18 ++- 10 files changed, 225 insertions(+), 19 deletions(-) create mode 100644 docs/changelog/127924.yaml create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceStaticTests.java diff --git a/docs/changelog/127924.yaml b/docs/changelog/127924.yaml new file mode 100644 index 0000000000000..4aaaa710563ab --- /dev/null +++ b/docs/changelog/127924.yaml @@ -0,0 +1,5 @@ +pr: 127924 +summary: Limit Replace function memory usage +area: ES|QL +type: enhancement +issues: [] diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/function/scalar/ScalarFunction.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/function/scalar/ScalarFunction.java index 09359943684b5..2a59b21a4c022 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/function/scalar/ScalarFunction.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/function/scalar/ScalarFunction.java @@ -13,6 +13,7 @@ import java.util.List; import static java.util.Collections.emptyList; +import static org.elasticsearch.common.unit.ByteSizeUnit.MB; /** * A {@code ScalarFunction} is a {@code Function} that takes values from some @@ -22,6 +23,14 @@ */ public abstract class ScalarFunction extends Function { + /** + * Limit for the BytesRef return of functions. + *

+ * To be used when there's no CircuitBreaking, as an arbitrary measure to limit memory usage. + *

+ */ + public static final long MAX_BYTES_REF_RESULT_SIZE = MB.toBytes(1); + protected ScalarFunction(Source source) { super(source, emptyList()); } diff --git a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceConstantEvaluator.java b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceConstantEvaluator.java index a5aa37a0db56e..f63966810a5fe 100644 --- a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceConstantEvaluator.java +++ b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceConstantEvaluator.java @@ -8,7 +8,6 @@ import java.lang.Override; import java.lang.String; import java.util.regex.Pattern; -import java.util.regex.PatternSyntaxException; import org.apache.lucene.util.BytesRef; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BytesRefBlock; @@ -92,7 +91,7 @@ public BytesRefBlock eval(int positionCount, BytesRefBlock strBlock, BytesRefBlo } try { result.appendBytesRef(Replace.process(strBlock.getBytesRef(strBlock.getFirstValueIndex(p), strScratch), this.regex, newStrBlock.getBytesRef(newStrBlock.getFirstValueIndex(p), newStrScratch))); - } catch (PatternSyntaxException e) { + } catch (IllegalArgumentException e) { warnings().registerException(e); result.appendNull(); } @@ -109,7 +108,7 @@ public BytesRefBlock eval(int positionCount, BytesRefVector strVector, position: for (int p = 0; p < positionCount; p++) { try { result.appendBytesRef(Replace.process(strVector.getBytesRef(p, strScratch), this.regex, newStrVector.getBytesRef(p, newStrScratch))); - } catch (PatternSyntaxException e) { + } catch (IllegalArgumentException e) { warnings().registerException(e); result.appendNull(); } diff --git a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceEvaluator.java b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceEvaluator.java index 7a7a947453d0a..6eb3aa898b79c 100644 --- a/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceEvaluator.java +++ b/x-pack/plugin/esql/src/main/generated/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceEvaluator.java @@ -7,7 +7,6 @@ import java.lang.IllegalArgumentException; import java.lang.Override; import java.lang.String; -import java.util.regex.PatternSyntaxException; import org.apache.lucene.util.BytesRef; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BytesRefBlock; @@ -111,7 +110,7 @@ public BytesRefBlock eval(int positionCount, BytesRefBlock strBlock, BytesRefBlo } try { result.appendBytesRef(Replace.process(strBlock.getBytesRef(strBlock.getFirstValueIndex(p), strScratch), regexBlock.getBytesRef(regexBlock.getFirstValueIndex(p), regexScratch), newStrBlock.getBytesRef(newStrBlock.getFirstValueIndex(p), newStrScratch))); - } catch (PatternSyntaxException e) { + } catch (IllegalArgumentException e) { warnings().registerException(e); result.appendNull(); } @@ -129,7 +128,7 @@ public BytesRefBlock eval(int positionCount, BytesRefVector strVector, BytesRefV position: for (int p = 0; p < positionCount; p++) { try { result.appendBytesRef(Replace.process(strVector.getBytesRef(p, strScratch), regexVector.getBytesRef(p, regexScratch), newStrVector.getBytesRef(p, newStrScratch))); - } catch (PatternSyntaxException e) { + } catch (IllegalArgumentException e) { warnings().registerException(e); result.appendNull(); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Repeat.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Repeat.java index 363991d1556f1..faa7ddbf63266 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Repeat.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Repeat.java @@ -30,7 +30,6 @@ import java.util.Arrays; import java.util.List; -import static org.elasticsearch.common.unit.ByteSizeUnit.MB; import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; @@ -40,8 +39,6 @@ public class Repeat extends EsqlScalarFunction implements OptionalArgument { public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Expression.class, "Repeat", Repeat::new); - static final long MAX_REPEATED_LENGTH = MB.toBytes(1); - private final Expression str; private final Expression number; @@ -123,9 +120,9 @@ static BytesRef process( static BytesRef processInner(BreakingBytesRefBuilder scratch, BytesRef str, int number) { int repeatedLen = str.length * number; - if (repeatedLen > MAX_REPEATED_LENGTH) { + if (repeatedLen > MAX_BYTES_REF_RESULT_SIZE) { throw new IllegalArgumentException( - "Creating repeated strings with more than [" + MAX_REPEATED_LENGTH + "] bytes is not supported" + "Creating repeated strings with more than [" + MAX_BYTES_REF_RESULT_SIZE + "] bytes is not supported" ); } scratch.grow(repeatedLen); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Replace.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Replace.java index 1fbe6bec85121..4c7e9fd524263 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Replace.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Replace.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.regex.PatternSyntaxException; @@ -121,24 +122,63 @@ public boolean foldable() { return str.foldable() && regex.foldable() && newStr.foldable(); } - @Evaluator(extraName = "Constant", warnExceptions = PatternSyntaxException.class) + @Evaluator(extraName = "Constant", warnExceptions = IllegalArgumentException.class) static BytesRef process(BytesRef str, @Fixed Pattern regex, BytesRef newStr) { if (str == null || regex == null || newStr == null) { return null; } - return new BytesRef(regex.matcher(str.utf8ToString()).replaceAll(newStr.utf8ToString())); + return safeReplace(str, regex, newStr); } - @Evaluator(warnExceptions = PatternSyntaxException.class) + @Evaluator(warnExceptions = IllegalArgumentException.class) static BytesRef process(BytesRef str, BytesRef regex, BytesRef newStr) { if (str == null) { return null; } - if (regex == null || newStr == null) { return str; } - return new BytesRef(str.utf8ToString().replaceAll(regex.utf8ToString(), newStr.utf8ToString())); + return safeReplace(str, Pattern.compile(regex.utf8ToString()), newStr); + } + + /** + * Executes a Replace without surpassing the memory limit. + */ + private static BytesRef safeReplace(BytesRef strBytesRef, Pattern regex, BytesRef newStrBytesRef) { + String str = strBytesRef.utf8ToString(); + Matcher m = regex.matcher(str); + if (false == m.find()) { + return strBytesRef; + } + String newStr = newStrBytesRef.utf8ToString(); + + // Count potential groups (E.g. "$1") used in the replacement + int constantReplacementLength = newStr.length(); + int groupsInReplacement = 0; + for (int i = 0; i < newStr.length(); i++) { + if (newStr.charAt(i) == '$') { + groupsInReplacement++; + constantReplacementLength -= 2; + i++; + } + } + + // Initialize the buffer with an approximate size for the first replacement + StringBuilder result = new StringBuilder(str.length() + newStr.length() + 8); + do { + int matchSize = m.end() - m.start(); + int potentialReplacementSize = constantReplacementLength + groupsInReplacement * matchSize; + int remainingStr = str.length() - m.end(); + if (result.length() + potentialReplacementSize + remainingStr > MAX_BYTES_REF_RESULT_SIZE) { + throw new IllegalArgumentException( + "Creating strings with more than [" + MAX_BYTES_REF_RESULT_SIZE + "] bytes is not supported" + ); + } + + m.appendReplacement(result, newStr); + } while (m.find()); + m.appendTail(result); + return new BytesRef(result.toString()); } @Override diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractScalarFunctionTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractScalarFunctionTestCase.java index f056e0c61c8d1..d64d0dcc43dc3 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractScalarFunctionTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractScalarFunctionTestCase.java @@ -304,6 +304,7 @@ public final void testEvaluateInManyThreads() throws ExecutionException, Interru if (testCase.getExpectedBuildEvaluatorWarnings() != null) { assertWarnings(testCase.getExpectedBuildEvaluatorWarnings()); } + ExecutorService exec = Executors.newFixedThreadPool(threads); try { List> futures = new ArrayList<>(); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RepeatStaticTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RepeatStaticTests.java index 33a490fbde3be..95db9daa21283 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RepeatStaticTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RepeatStaticTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.compute.test.TestBlockFactory; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; +import org.elasticsearch.xpack.esql.core.expression.function.scalar.ScalarFunction; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.EsField; @@ -45,14 +46,14 @@ public class RepeatStaticTests extends ESTestCase { public void testAlmostTooBig() { String str = randomAlphaOfLength(1); - int number = (int) Repeat.MAX_REPEATED_LENGTH; + int number = (int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE; String repeated = process(str, number); assertThat(repeated, equalTo(str.repeat(number))); } public void testTooBig() { String str = randomAlphaOfLength(1); - int number = (int) Repeat.MAX_REPEATED_LENGTH + 1; + int number = (int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE + 1; String repeated = process(str, number); assertNull(repeated); assertWarnings( diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceStaticTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceStaticTests.java new file mode 100644 index 0000000000000..cac4b5acfa320 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceStaticTests.java @@ -0,0 +1,139 @@ +/* + * 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.expression.function.scalar.string; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.operator.DriverContext; +import org.elasticsearch.compute.test.TestBlockFactory; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; +import org.elasticsearch.xpack.esql.core.expression.function.scalar.ScalarFunction; +import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.core.type.EsField; +import org.elasticsearch.xpack.esql.expression.function.AbstractScalarFunctionTestCase; +import org.junit.After; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +/** + * These tests create rows that are 1MB in size. Test classes + * which extend AbstractScalarFunctionTestCase rerun test cases with + * many randomized inputs. Unfortunately, tests are run with + * limited memory, and instantiating many copies of these + * tests with large rows causes out of memory. + */ +public class ReplaceStaticTests extends ESTestCase { + + public void testLimit() { + int textLength = (int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE / 10; + String text = randomAlphaOfLength((int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE / 10); + String regex = "^(.+)$"; + + // 10 times the original text + the remainder + String extraString = "a".repeat((int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE % 10); + assert textLength * 10 + extraString.length() == ScalarFunction.MAX_BYTES_REF_RESULT_SIZE; + String newStr = "$0$0$0$0$0$0$0$0$0$0" + extraString; + + String result = process(text, regex, newStr); + assertThat(result, equalTo(newStr.replaceAll("\\$\\d", text))); + } + + public void testTooBig() { + String textAndNewStr = randomAlphaOfLength((int) (ScalarFunction.MAX_BYTES_REF_RESULT_SIZE / 10)); + String regex = "."; + + String result = process(textAndNewStr, regex, textAndNewStr); + assertNull(result); + assertWarnings( + "Line -1:-1: evaluation of [] failed, treating result as null. Only first 20 failures recorded.", + "Line -1:-1: java.lang.IllegalArgumentException: " + + "Creating strings with more than [" + + ScalarFunction.MAX_BYTES_REF_RESULT_SIZE + + "] bytes is not supported" + ); + } + + public void testTooBigWithGroups() { + int textLength = (int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE / 10; + String text = randomAlphaOfLength(textLength); + String regex = "(.+)"; + + // 10 times the original text + the remainder + 1 + String extraString = "a".repeat(1 + (int) ScalarFunction.MAX_BYTES_REF_RESULT_SIZE % 10); + assert textLength * 10 + extraString.length() == ScalarFunction.MAX_BYTES_REF_RESULT_SIZE + 1; + String newStr = "$0$1$0$1$0$1$0$1$0$1" + extraString; + + String result = process(text, regex, newStr); + assertNull(result); + assertWarnings( + "Line -1:-1: evaluation of [] failed, treating result as null. Only first 20 failures recorded.", + "Line -1:-1: java.lang.IllegalArgumentException: " + + "Creating strings with more than [" + + ScalarFunction.MAX_BYTES_REF_RESULT_SIZE + + "] bytes is not supported" + ); + } + + public String process(String text, String regex, String newStr) { + try ( + var eval = AbstractScalarFunctionTestCase.evaluator( + new Replace( + Source.EMPTY, + field("text", DataType.KEYWORD), + field("regex", DataType.KEYWORD), + field("newStr", DataType.KEYWORD) + ) + ).get(driverContext()); + Block block = eval.eval(row(List.of(new BytesRef(text), new BytesRef(regex), new BytesRef(newStr)))); + ) { + return block.isNull(0) ? null : ((BytesRef) BlockUtils.toJavaObject(block, 0)).utf8ToString(); + } + } + + /** + * The following fields and methods were borrowed from AbstractScalarFunctionTestCase + */ + private final List breakers = Collections.synchronizedList(new ArrayList<>()); + + private static Page row(List values) { + return new Page(1, BlockUtils.fromListRow(TestBlockFactory.getNonBreakingInstance(), values)); + } + + private static FieldAttribute field(String name, DataType type) { + return new FieldAttribute(Source.synthetic(name), name, new EsField(name, type, Map.of(), true)); + } + + private DriverContext driverContext() { + BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, ByteSizeValue.ofMb(256)).withCircuitBreaking(); + CircuitBreaker breaker = bigArrays.breakerService().getBreaker(CircuitBreaker.REQUEST); + breakers.add(breaker); + return new DriverContext(bigArrays, new BlockFactory(breaker, bigArrays)); + } + + @After + public void allMemoryReleased() { + for (CircuitBreaker breaker : breakers) { + assertThat(breaker.getUsed(), equalTo(0L)); + } + } +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceTests.java index bb27be3f67d3e..2deb2677e0ce0 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/ReplaceTests.java @@ -78,6 +78,22 @@ public static Iterable parameters() { ) ); + // Groups + suppliers.add(fixedCase("Full group", "Cats are awesome", ".+", "<$0>", "")); + suppliers.add( + fixedCase("Nested groups", "A cat is great, a cat is awesome", "\\b([Aa] (\\w+)) is (\\w+)\\b", "$1$2", "A catcat, a catcat") + ); + suppliers.add( + fixedCase( + "Multiple groups", + "Cats are awesome", + "(\\w+) (.+)", + "$0 -> $1 and dogs $2", + "Cats are awesome -> Cats and dogs are awesome" + ) + ); + + // Errors suppliers.add(new TestCaseSupplier("syntax error", List.of(DataType.KEYWORD, DataType.KEYWORD, DataType.KEYWORD), () -> { String text = randomAlphaOfLength(10); String invalidRegex = "["; @@ -85,7 +101,7 @@ public static Iterable parameters() { return new TestCaseSupplier.TestCase( List.of( new TestCaseSupplier.TypedData(new BytesRef(text), DataType.KEYWORD, "str"), - new TestCaseSupplier.TypedData(new BytesRef(invalidRegex), DataType.KEYWORD, "oldStr"), + new TestCaseSupplier.TypedData(new BytesRef(invalidRegex), DataType.KEYWORD, "regex"), new TestCaseSupplier.TypedData(new BytesRef(newStr), DataType.KEYWORD, "newStr") ), "ReplaceEvaluator[str=Attribute[channel=0], regex=Attribute[channel=1], newStr=Attribute[channel=2]]", From be396aef85ba01f732ab79a3d2b738eebde659dd Mon Sep 17 00:00:00 2001 From: zhouhui Date: Tue, 20 May 2025 21:11:30 +0800 Subject: [PATCH 009/231] Fix comment about CircuitBreaker#IN_FLIGHT_REQUESTS (#128192) --- .../java/org/elasticsearch/common/breaker/CircuitBreaker.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java index 544710635fbae..1384ae0783dbb 100644 --- a/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java +++ b/server/src/main/java/org/elasticsearch/common/breaker/CircuitBreaker.java @@ -39,8 +39,8 @@ public interface CircuitBreaker { */ String REQUEST = "request"; /** - * The in-flight request breaker tracks bytes allocated for reading and - * writing requests on the network layer. + * The in-flight request breaker tracks bytes allocated for reading requests + * on the network layer. */ String IN_FLIGHT_REQUESTS = "inflight_requests"; From c0f5e00378874032929b52656ecff2f2b1fd9b52 Mon Sep 17 00:00:00 2001 From: Pat Whelan Date: Tue, 20 May 2025 09:16:52 -0400 Subject: [PATCH 010/231] [Transform] Check alias during update (#124825) When the Transform System Index has been reindexed and aliased, we should check the Transform Update index against the alias when updating the Transform Config. --- docs/changelog/124825.yaml | 5 ++ .../integration/TransformOldTransformsIT.java | 67 +++++++++++++++++++ .../TransformConfigManagerTests.java | 3 +- .../transform/action/TransformUpdater.java | 5 +- .../IndexBasedTransformConfigManager.java | 19 +++++- .../persistence/TransformConfigManager.java | 5 ++ 6 files changed, 98 insertions(+), 6 deletions(-) create mode 100644 docs/changelog/124825.yaml diff --git a/docs/changelog/124825.yaml b/docs/changelog/124825.yaml new file mode 100644 index 0000000000000..413695a5ae641 --- /dev/null +++ b/docs/changelog/124825.yaml @@ -0,0 +1,5 @@ +pr: 124825 +summary: Check alias during update +area: Transform +type: bug +issues: [] diff --git a/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformOldTransformsIT.java b/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformOldTransformsIT.java index cb8580054f8cc..70b24696d0815 100644 --- a/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformOldTransformsIT.java +++ b/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/integration/TransformOldTransformsIT.java @@ -16,6 +16,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Strings; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.reindex.ReindexAction; +import org.elasticsearch.index.reindex.ReindexRequest; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentFactory; import org.elasticsearch.xcontent.XContentType; @@ -25,12 +27,16 @@ import org.elasticsearch.xpack.core.transform.TransformDeprecations; import org.elasticsearch.xpack.core.transform.TransformField; import org.elasticsearch.xpack.core.transform.action.GetTransformAction; +import org.elasticsearch.xpack.core.transform.action.PutTransformAction; import org.elasticsearch.xpack.core.transform.action.StartTransformAction; import org.elasticsearch.xpack.core.transform.action.StopTransformAction; import org.elasticsearch.xpack.core.transform.action.UpdateTransformAction; +import org.elasticsearch.xpack.core.transform.transforms.DestConfig; +import org.elasticsearch.xpack.core.transform.transforms.SourceConfig; import org.elasticsearch.xpack.core.transform.transforms.TransformConfig; import org.elasticsearch.xpack.core.transform.transforms.TransformConfigUpdate; import org.elasticsearch.xpack.core.transform.transforms.persistence.TransformInternalIndexConstants; +import org.elasticsearch.xpack.core.transform.transforms.pivot.PivotConfigTests; import org.elasticsearch.xpack.core.transform.utils.TransformConfigVersionUtils; import org.elasticsearch.xpack.transform.TransformSingleNodeTestCase; import org.elasticsearch.xpack.transform.persistence.TransformInternalIndex; @@ -253,4 +259,65 @@ public void testStartReplacesDeprecatedTransformSettings() throws Exception { assertMaxPageSearchSizeInSettings(transformId, expectedMaxPageSearchSize); } + public void testMigratedTransformIndex() { + // create transform + var sourceIndex = "source-index"; + createSourceIndex(sourceIndex); + var transformId = "transform-migrated-system-index"; + + var sourceConfig = new SourceConfig(sourceIndex); + var destConfig = new DestConfig("some-dest-index", null, null); + var config = new TransformConfig( + transformId, + sourceConfig, + destConfig, + null, + null, + null, + PivotConfigTests.randomPivotConfig(), + null, + null, + null, + null, + null, + null, + null + ); + var putTransform = new PutTransformAction.Request(config, true, TimeValue.THIRTY_SECONDS); + assertTrue(client().execute(PutTransformAction.INSTANCE, putTransform).actionGet().isAcknowledged()); + + // simulate migration by reindexing and aliasing + var newSystemIndex = TransformInternalIndexConstants.LATEST_INDEX_NAME + "-reindexed"; + var reindexRequest = new ReindexRequest(); + reindexRequest.setSourceIndices(TransformInternalIndexConstants.LATEST_INDEX_NAME); + reindexRequest.setDestIndex(newSystemIndex); + reindexRequest.setRefresh(true); + client().execute(ReindexAction.INSTANCE, reindexRequest).actionGet(); + + var aliasesRequest = admin().indices().prepareAliases(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS); + aliasesRequest.removeIndex(TransformInternalIndexConstants.LATEST_INDEX_NAME); + aliasesRequest.addAlias(newSystemIndex, TransformInternalIndexConstants.LATEST_INDEX_NAME); + aliasesRequest.execute().actionGet(); + + // update should succeed + var updateConfig = new TransformConfigUpdate( + sourceConfig, + new DestConfig("some-new-dest-index", null, null), + null, + null, + null, + null, + null, + null + ); + var updateRequest = new UpdateTransformAction.Request(updateConfig, transformId, true, TimeValue.THIRTY_SECONDS); + client().execute(UpdateTransformAction.INSTANCE, updateRequest).actionGet(); + + // verify update succeeded + var getTransformRequest = new GetTransformAction.Request(transformId); + var getTransformResponse = client().execute(GetTransformAction.INSTANCE, getTransformRequest).actionGet(); + var transformConfig = getTransformResponse.getTransformConfigurations().get(0); + assertThat(transformConfig.getDestination().getIndex(), equalTo("some-new-dest-index")); + } + } diff --git a/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/persistence/TransformConfigManagerTests.java b/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/persistence/TransformConfigManagerTests.java index cf7441282fa67..86b95e67e356e 100644 --- a/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/persistence/TransformConfigManagerTests.java +++ b/x-pack/plugin/transform/src/internalClusterTest/java/org/elasticsearch/xpack/transform/persistence/TransformConfigManagerTests.java @@ -78,7 +78,8 @@ public class TransformConfigManagerTests extends TransformSingleNodeTestCase { @Before public void createComponents() { - clusterService = mock(ClusterService.class); + clusterService = mock(); + when(clusterService.state()).thenReturn(ClusterState.EMPTY_STATE); transformConfigManager = new IndexBasedTransformConfigManager( clusterService, TestIndexNameExpressionResolver.newInstance(), diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransformUpdater.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransformUpdater.java index 935ff04c47d85..f5a6f510765b5 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransformUpdater.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransformUpdater.java @@ -252,7 +252,7 @@ private static void updateTransformStateAndGetLastCheckpoint( long lastCheckpoint = currentState.v1().getTransformState().getCheckpoint(); // if: the state is stored on the latest index, it does not need an update - if (currentState.v2().getIndex().equals(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME)) { + if (transformConfigManager.isLatestTransformIndex(currentState.v2().getIndex())) { listener.onResponse(lastCheckpoint); return; } @@ -283,8 +283,7 @@ private static void updateTransformCheckpoint( ActionListener listener ) { transformConfigManager.getTransformCheckpointForUpdate(transformId, lastCheckpoint, ActionListener.wrap(checkpointAndVersion -> { - if (checkpointAndVersion == null - || checkpointAndVersion.v2().getIndex().equals(TransformInternalIndexConstants.LATEST_INDEX_VERSIONED_NAME)) { + if (checkpointAndVersion == null || transformConfigManager.isLatestTransformIndex(checkpointAndVersion.v2().getIndex())) { listener.onResponse(true); return; } diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/IndexBasedTransformConfigManager.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/IndexBasedTransformConfigManager.java index f78868112e8cf..2d121d95b4da8 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/IndexBasedTransformConfigManager.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/IndexBasedTransformConfigManager.java @@ -170,7 +170,7 @@ public void updateTransformConfiguration( listener.onFailure(conflictStatusException("Cannot update Transform while the Transform feature is upgrading.")); return; } - if (seqNoPrimaryTermAndIndex.getIndex().equals(TransformInternalIndexConstants.LATEST_INDEX_NAME)) { + if (isLatestTransformIndex(seqNoPrimaryTermAndIndex.getIndex())) { // update the config in the same, current index using optimistic concurrency control putTransformConfiguration(transformConfig, DocWriteRequest.OpType.INDEX, seqNoPrimaryTermAndIndex, listener); } else { @@ -180,6 +180,21 @@ public void updateTransformConfiguration( } } + @Override + public boolean isLatestTransformIndex(String indexName) { + if (TransformInternalIndexConstants.LATEST_INDEX_NAME.equals(indexName)) { + return true; + } + + // in some cases, the System Index gets reindexed and LATEST_INDEX_NAME is now an alias pointing to that reindexed index + // this mostly likely happens after the SystemIndexMigrator ran + // we need to check if the LATEST_INDEX_NAME is now an alias and points to the indexName + var metadata = clusterService.state().projectState().metadata(); + var indicesForAlias = metadata.aliasedIndices(TransformInternalIndexConstants.LATEST_INDEX_NAME); + var index = metadata.index(indexName); + return index != null && indicesForAlias.contains(index.getIndex()); + } + @Override public void deleteOldTransformConfigurations(String transformId, ActionListener listener) { if (isUpgrading()) { @@ -697,7 +712,7 @@ public void putOrUpdateTransformStoredDoc( // could have been called, see gh#80073 indexRequest.opType(DocWriteRequest.OpType.INDEX); // if on the latest index use optimistic concurrency control in addition - if (seqNoPrimaryTermAndIndex.getIndex().equals(TransformInternalIndexConstants.LATEST_INDEX_NAME)) { + if (isLatestTransformIndex(seqNoPrimaryTermAndIndex.getIndex())) { indexRequest.setIfSeqNo(seqNoPrimaryTermAndIndex.getSeqNo()) .setIfPrimaryTerm(seqNoPrimaryTermAndIndex.getPrimaryTerm()); } diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/TransformConfigManager.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/TransformConfigManager.java index 5ed1a7254268b..ba8ea989cfabf 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/TransformConfigManager.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/TransformConfigManager.java @@ -15,6 +15,7 @@ import org.elasticsearch.xpack.core.transform.transforms.TransformCheckpoint; import org.elasticsearch.xpack.core.transform.transforms.TransformConfig; import org.elasticsearch.xpack.core.transform.transforms.TransformStoredDoc; +import org.elasticsearch.xpack.core.transform.transforms.persistence.TransformInternalIndexConstants; import java.util.Collection; import java.util.Collections; @@ -206,4 +207,8 @@ void getTransformStoredDoc( void getTransformStoredDocs(Collection transformIds, TimeValue timeout, ActionListener> listener); void refresh(ActionListener listener); + + default boolean isLatestTransformIndex(String indexName) { + return TransformInternalIndexConstants.LATEST_INDEX_NAME.equals(indexName); + } } From a2b4a6f246fd6dc990138c145a629c7cd166c3fd Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Tue, 20 May 2025 07:09:27 -0700 Subject: [PATCH 011/231] Add temporary LegacyActionRequest (#128107) In order to remove ActionType, ActionRequest will become strongly typed, referring to the ActionResponse type. As a precursor to that, this commit adds a LegacyActionRequest which all existing ActionRequest implementations now inherit from. This will allow adding the ActionResponse type to ActionRequest in a future commit without modifying every implementation at once. --- .../ingest/common/GrokProcessorGetAction.java | 6 +-- .../mustache/MultiSearchTemplateRequest.java | 4 +- .../mustache/SearchTemplateRequest.java | 5 ++- .../action/PainlessContextAction.java | 4 +- .../index/rankeval/RankEvalRequest.java | 4 +- .../elasticsearch/rest/root/MainRequest.java | 4 +- .../netty4/Netty4ChunkedContinuationsIT.java | 8 ++-- .../node/tasks/CancellableTasksIT.java | 5 ++- .../admin/cluster/tasks/ListTasksIT.java | 4 +- .../MockedRequestActionBasedRerankerIT.java | 4 +- .../action/LegacyActionRequest.java | 30 +++++++++++++++ .../ClusterFormationInfoAction.java | 20 ++++------ .../CoordinationDiagnosticsAction.java | 10 ++--- .../coordination/MasterHistoryAction.java | 11 +++--- .../node/tasks/get/GetTaskRequest.java | 4 +- .../remote/RemoteClusterNodesAction.java | 11 ++---- .../cluster/remote/RemoteInfoRequest.java | 4 +- .../stats/RemoteClusterStatsRequest.java | 4 +- .../GetScriptContextRequest.java | 4 +- .../GetScriptLanguageRequest.java | 4 +- .../mapping/get/GetFieldMappingsRequest.java | 14 ++++--- .../resolve/ResolveClusterActionRequest.java | 4 +- .../indices/resolve/ResolveIndexAction.java | 16 ++++---- .../action/bulk/BulkRequest.java | 3 +- .../FieldCapabilitiesNodeRequest.java | 4 +- .../fieldcaps/FieldCapabilitiesRequest.java | 4 +- .../action/get/MultiGetRequest.java | 4 +- .../get/TransportGetFromTranslogAction.java | 4 +- ...ansportShardMultiGetFomTranslogAction.java | 4 +- .../ingest/SimulatePipelineRequest.java | 4 +- .../action/search/ClearScrollRequest.java | 4 +- .../search/ClosePointInTimeRequest.java | 4 +- .../action/search/MultiSearchRequest.java | 4 +- .../action/search/OpenPointInTimeRequest.java | 4 +- .../action/search/SearchRequest.java | 23 +++++------ .../action/search/SearchScrollRequest.java | 4 +- .../action/search/SearchShardsRequest.java | 4 +- .../support/broadcast/BroadcastRequest.java | 4 +- .../BroadcastUnpromotableRequest.java | 4 +- .../local/LocalClusterStateRequest.java | 10 +++-- .../support/master/MasterNodeRequest.java | 11 +++--- .../support/nodes/BaseNodesRequest.java | 4 +- .../replication/ReplicationRequest.java | 6 ++- .../InstanceShardOperationRequest.java | 4 +- .../single/shard/SingleShardRequest.java | 12 +++--- .../support/tasks/BaseTasksRequest.java | 6 +-- .../AbstractSynonymsPagedResultAction.java | 4 +- .../synonyms/DeleteSynonymRuleAction.java | 4 +- .../action/synonyms/DeleteSynonymsAction.java | 4 +- .../action/synonyms/GetSynonymRuleAction.java | 4 +- .../action/synonyms/PutSynonymRuleAction.java | 4 +- .../action/synonyms/PutSynonymsAction.java | 4 +- .../termvectors/MultiTermVectorsRequest.java | 4 +- .../elasticsearch/health/GetHealthAction.java | 4 +- .../health/node/action/HealthNodeRequest.java | 4 +- .../reindex/AbstractBulkByScrollRequest.java | 8 ++-- .../index/shard/PrimaryReplicaSyncer.java | 8 ++-- .../StatelessPrimaryRelocationAction.java | 4 +- ...StatelessUnpromotableRelocationAction.java | 4 +- .../VerifyNodeRepositoryAction.java | 4 +- .../action/ActionModuleTests.java | 2 +- .../support/MappedActionFiltersTests.java | 3 +- ...StateAwareHandledTransportActionTests.java | 4 +- ...portActionFilterChainRefCountingTests.java | 3 +- .../TransportActionFilterChainTests.java | 3 +- .../action/support/TransportActionTests.java | 4 +- .../ClearCcrRestoreSessionRequest.java | 4 +- .../DeleteInternalCcrRepositoryRequest.java | 4 +- .../GetCcrRestoreFileChunkRequest.java | 4 +- .../PutInternalCcrRepositoryRequest.java | 4 +- .../license/GetFeatureUsageRequest.java | 4 +- .../protocol/xpack/XPackInfoRequest.java | 4 +- .../xpack/graph/GraphExploreRequest.java | 25 +++++------- .../xpack/watcher/DeleteWatchRequest.java | 4 +- .../xpack/watcher/PutWatchRequest.java | 22 +++++------ .../action/AbstractGetResourcesRequest.java | 4 +- .../xpack/core/async/AsyncStopRequest.java | 4 +- .../core/async/DeleteAsyncResultRequest.java | 4 +- .../core/async/GetAsyncResultRequest.java | 4 +- .../core/async/GetAsyncStatusRequest.java | 5 ++- .../core/esql/action/EsqlQueryRequest.java | 4 +- .../action/BaseInferenceActionRequest.java | 4 +- .../action/InferenceActionProxy.java | 4 +- .../ml/action/AuditMlNotificationAction.java | 4 +- .../CancelJobModelSnapshotUpgradeAction.java | 4 +- .../ml/action/CoordinatedInferenceAction.java | 6 +-- .../ml/action/DeleteExpiredDataAction.java | 5 ++- .../ml/action/DeleteModelSnapshotAction.java | 4 +- .../ml/action/EstimateModelMemoryAction.java | 6 +-- .../ml/action/EvaluateDataFrameAction.java | 4 +- .../core/ml/action/GetBucketsAction.java | 4 +- .../ml/action/GetCalendarEventsAction.java | 4 +- .../core/ml/action/GetCalendarsAction.java | 4 +- .../core/ml/action/GetCategoriesAction.java | 4 +- .../ml/action/GetDatafeedsStatsAction.java | 8 ++-- .../core/ml/action/GetInfluencersAction.java | 4 +- .../ml/action/GetModelSnapshotsAction.java | 4 +- .../ml/action/GetOverallBucketsAction.java | 4 +- .../core/ml/action/GetRecordsAction.java | 4 +- .../core/ml/action/InferModelAction.java | 38 ++++++++----------- .../xpack/core/ml/action/MlInfoAction.java | 4 +- .../ml/action/PostCalendarEventsAction.java | 4 +- .../PreviewDataFrameAnalyticsAction.java | 11 +++--- .../core/ml/action/PreviewDatafeedAction.java | 9 +++-- .../core/ml/action/PutCalendarAction.java | 4 +- .../xpack/core/ml/action/PutFilterAction.java | 4 +- .../ml/action/UpdateCalendarJobAction.java | 4 +- .../core/ml/action/UpdateFilterAction.java | 4 +- .../ml/action/UpdateModelSnapshotAction.java | 4 +- .../ml/action/ValidateDetectorAction.java | 4 +- .../ml/action/ValidateJobConfigAction.java | 4 +- .../action/MonitoringBulkRequest.java | 4 +- .../rollup/action/GetRollupCapsAction.java | 4 +- .../action/GetRollupIndexCapsAction.java | 4 +- .../action/SubmitAsyncSearchRequest.java | 4 +- .../DelegatePkiAuthenticationRequest.java | 4 +- .../core/security/action/GrantRequest.java | 4 +- .../apikey/AbstractCreateApiKeyRequest.java | 4 +- .../apikey/BaseUpdateApiKeyRequest.java | 4 +- .../action/apikey/GetApiKeyRequest.java | 4 +- .../apikey/InvalidateApiKeyRequest.java | 16 ++++---- .../action/apikey/QueryApiKeyRequest.java | 4 +- .../enrollment/KibanaEnrollmentRequest.java | 4 +- .../enrollment/NodeEnrollmentRequest.java | 4 +- .../OpenIdConnectAuthenticateRequest.java | 4 +- .../oidc/OpenIdConnectLogoutRequest.java | 4 +- ...IdConnectPrepareAuthenticationRequest.java | 4 +- .../privilege/DeletePrivilegesRequest.java | 4 +- .../GetBuiltinPrivilegesRequest.java | 4 +- .../privilege/GetPrivilegesRequest.java | 4 +- .../privilege/PutPrivilegesRequest.java | 7 +++- .../action/profile/GetProfilesRequest.java | 4 +- .../profile/SetProfileEnabledRequest.java | 4 +- .../profile/SuggestProfilesRequest.java | 4 +- .../profile/UpdateProfileDataRequest.java | 4 +- .../action/role/BulkDeleteRolesRequest.java | 4 +- .../action/role/BulkPutRolesRequest.java | 4 +- .../action/role/DeleteRoleRequest.java | 4 +- .../security/action/role/GetRolesRequest.java | 4 +- .../security/action/role/PutRoleRequest.java | 4 +- .../action/role/QueryRoleRequest.java | 4 +- .../rolemapping/DeleteRoleMappingRequest.java | 4 +- .../rolemapping/GetRoleMappingsRequest.java | 7 ++-- .../rolemapping/PutRoleMappingRequest.java | 6 +-- .../action/saml/SamlAuthenticateRequest.java | 4 +- .../saml/SamlCompleteLogoutRequest.java | 4 +- .../saml/SamlInvalidateSessionRequest.java | 4 +- .../action/saml/SamlLogoutRequest.java | 4 +- .../SamlPrepareAuthenticationRequest.java | 4 +- .../action/saml/SamlSpMetadataRequest.java | 4 +- .../CreateServiceAccountTokenRequest.java | 4 +- .../DeleteServiceAccountTokenRequest.java | 4 +- .../GetServiceAccountCredentialsRequest.java | 4 +- .../service/GetServiceAccountRequest.java | 4 +- .../action/token/CreateTokenRequest.java | 4 +- .../action/token/InvalidateTokenRequest.java | 10 ++--- .../action/user/AuthenticateRequest.java | 4 +- .../action/user/ChangePasswordRequest.java | 4 +- .../action/user/DeleteUserRequest.java | 4 +- .../action/user/GetUserPrivilegesRequest.java | 4 +- .../security/action/user/GetUsersRequest.java | 4 +- .../action/user/HasPrivilegesRequest.java | 4 +- .../user/ProfileHasPrivilegesRequest.java | 5 ++- .../security/action/user/PutUserRequest.java | 4 +- .../action/user/QueryUserRequest.java | 4 +- .../action/user/SetEnabledRequest.java | 4 +- .../ssl/action/GetCertificateInfoAction.java | 4 +- .../action/AbstractFindStructureRequest.java | 4 +- .../action/TestGrokPatternAction.java | 18 ++++----- .../transform/action/GetCheckpointAction.java | 4 +- .../action/GetCheckpointNodeAction.java | 4 +- .../transport/actions/QueryWatchesAction.java | 7 ++-- .../actions/ack/AckWatchRequest.java | 4 +- .../activate/ActivateWatchRequest.java | 6 +-- .../actions/execute/ExecuteWatchRequest.java | 14 +++---- .../actions/get/GetWatchRequest.java | 4 +- .../action/MockIndicesRequest.java | 2 +- .../ApiFilteringActionFilterTests.java | 4 +- .../apikey/InvalidateApiKeyRequestTests.java | 4 +- ...DownsampleShardPersistentTaskExecutor.java | 4 +- .../action/PostAnalyticsEventAction.java | 6 +-- .../action/ConnectorActionRequest.java | 10 ++--- .../action/DeleteConnectorSecretRequest.java | 4 +- .../action/GetConnectorSecretRequest.java | 4 +- .../action/PostConnectorSecretRequest.java | 4 +- .../action/PutConnectorSecretRequest.java | 7 ++-- .../action/ConnectorSyncJobActionRequest.java | 4 +- .../rules/action/DeleteQueryRuleAction.java | 5 ++- .../action/DeleteQueryRulesetAction.java | 5 ++- .../rules/action/GetQueryRuleAction.java | 7 ++-- .../rules/action/GetQueryRulesetAction.java | 9 +++-- .../rules/action/ListQueryRulesetsAction.java | 6 +-- .../rules/action/PutQueryRuleAction.java | 12 +++--- .../rules/action/PutQueryRulesetAction.java | 10 ++--- .../rules/action/TestQueryRulesetAction.java | 9 +++-- .../action/DeleteSearchApplicationAction.java | 4 +- .../action/GetSearchApplicationAction.java | 6 +-- .../action/ListSearchApplicationAction.java | 7 ++-- .../action/PutSearchApplicationAction.java | 7 ++-- .../SearchApplicationSearchRequest.java | 4 +- .../xpack/eql/action/EqlSearchRequest.java | 10 ++--- .../AsyncTaskManagementServiceTests.java | 4 +- .../compute/operator/DriverTaskRunner.java | 4 +- .../esql/action/EsqlGetQueryRequest.java | 4 +- .../esql/action/EsqlListQueriesRequest.java | 4 +- .../fleet/action/DeleteSecretRequest.java | 4 +- .../action/GetGlobalCheckpointsAction.java | 7 ++-- .../xpack/fleet/action/GetSecretRequest.java | 4 +- .../xpack/fleet/action/PostSecretRequest.java | 4 +- .../DeleteSamlServiceProviderRequest.java | 4 +- .../action/PutSamlServiceProviderRequest.java | 4 +- .../SamlInitiateSingleSignOnRequest.java | 4 +- .../xpack/idp/action/SamlMetadataRequest.java | 4 +- .../SamlValidateAuthnRequestRequest.java | 4 +- .../action/DeletePipelineRequest.java | 4 +- .../logstash/action/GetPipelineRequest.java | 4 +- .../logstash/action/PutPipelineRequest.java | 4 +- .../action/CancelReindexDataStreamAction.java | 4 +- .../action/CreateIndexFromSourceAction.java | 6 +-- .../GetMigrationReindexStatusAction.java | 4 +- .../action/ReindexDataStreamAction.java | 4 +- .../action/ReindexDataStreamIndexAction.java | 4 +- .../action/GetStackTracesRequest.java | 4 +- .../AsyncTaskManagementServiceTests.java | 4 +- .../action/cache/FrozenCacheInfoAction.java | 8 ++-- .../cache/FrozenCacheInfoNodeAction.java | 4 +- ...lusterSecurityFcActionAuthorizationIT.java | 3 +- ...tReloadRemoteClusterCredentialsAction.java | 11 +++--- ...TransportSLMGetExpiredSnapshotsAction.java | 4 +- .../testkit/analyze/BlobAnalyzeAction.java | 13 +++---- .../ContendedRegisterAnalyzeAction.java | 4 +- .../analyze/GetBlobChecksumAction.java | 6 +-- .../analyze/RepositoryAnalyzeAction.java | 8 ++-- .../UncontendedRegisterAnalyzeAction.java | 4 +- ...nsportRepositoryVerifyIntegrityAction.java | 6 +-- ...toryVerifyIntegrityCoordinationAction.java | 4 +- ...oryVerifyIntegrityResponseChunkAction.java | 4 +- .../xpack/sql/action/AbstractSqlRequest.java | 4 +- 238 files changed, 682 insertions(+), 645 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/action/LegacyActionRequest.java diff --git a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/GrokProcessorGetAction.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/GrokProcessorGetAction.java index f149d768bf46b..033a45946fb93 100644 --- a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/GrokProcessorGetAction.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/common/GrokProcessorGetAction.java @@ -10,10 +10,10 @@ import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.internal.node.NodeClient; @@ -43,11 +43,11 @@ public class GrokProcessorGetAction { - static final ActionType INSTANCE = new ActionType<>("cluster:admin/ingest/processor/grok/get"); + static final ActionType INSTANCE = new ActionType<>("cluster:admin/ingest/processor/grok/get"); private GrokProcessorGetAction() {/* no instances */} - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { private final boolean sorted; private final String ecsCompatibility; diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MultiSearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MultiSearchTemplateRequest.java index 07a00b33407f1..29d9ad6a8eedb 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MultiSearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/MultiSearchTemplateRequest.java @@ -9,9 +9,9 @@ package org.elasticsearch.script.mustache; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.IndicesOptions; @@ -30,7 +30,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class MultiSearchTemplateRequest extends ActionRequest implements CompositeIndicesRequest { +public class MultiSearchTemplateRequest extends LegacyActionRequest implements CompositeIndicesRequest { private int maxConcurrentSearchRequests = 0; private List requests = new ArrayList<>(); diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateRequest.java index b535b3556b5be..2190b39bb658e 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/SearchTemplateRequest.java @@ -9,9 +9,9 @@ package org.elasticsearch.script.mustache; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; @@ -34,7 +34,7 @@ /** * A request to execute a search based on a search template. */ -public class SearchTemplateRequest extends ActionRequest implements CompositeIndicesRequest, ToXContentObject { +public class SearchTemplateRequest extends LegacyActionRequest implements CompositeIndicesRequest, ToXContentObject { private SearchRequest request; private boolean simulate = false; @@ -171,6 +171,7 @@ public ActionRequestValidationException validate() { private static final ParseField PROFILE_FIELD = new ParseField("profile"); private static final ObjectParser PARSER; + static { PARSER = new ObjectParser<>("search_template"); PARSER.declareField((parser, request, s) -> request.setScriptParams(parser.map()), PARAMS_FIELD, ObjectParser.ValueType.OBJECT); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextAction.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextAction.java index efcd63be46888..5df68e889d7c4 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextAction.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/action/PainlessContextAction.java @@ -10,10 +10,10 @@ package org.elasticsearch.painless.action; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.internal.node.NodeClient; @@ -61,7 +61,7 @@ public class PainlessContextAction { private PainlessContextAction() {/* no instances */} - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { private String scriptContextName; diff --git a/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequest.java b/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequest.java index 4bb30fdb0dd01..53d947cea681f 100644 --- a/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequest.java +++ b/modules/rank-eval/src/main/java/org/elasticsearch/index/rankeval/RankEvalRequest.java @@ -9,9 +9,9 @@ package org.elasticsearch.index.rankeval; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.support.IndicesOptions; @@ -26,7 +26,7 @@ /** * Request to perform a search ranking evaluation. */ -public final class RankEvalRequest extends ActionRequest implements IndicesRequest.Replaceable { +public final class RankEvalRequest extends LegacyActionRequest implements IndicesRequest.Replaceable { private RankEvalSpec rankingEvaluationSpec; diff --git a/modules/rest-root/src/main/java/org/elasticsearch/rest/root/MainRequest.java b/modules/rest-root/src/main/java/org/elasticsearch/rest/root/MainRequest.java index edcf845512924..d6bbac30fb8f4 100644 --- a/modules/rest-root/src/main/java/org/elasticsearch/rest/root/MainRequest.java +++ b/modules/rest-root/src/main/java/org/elasticsearch/rest/root/MainRequest.java @@ -9,14 +9,14 @@ package org.elasticsearch.rest.root; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.common.io.stream.StreamOutput; import java.io.IOException; -public class MainRequest extends ActionRequest { +public class MainRequest extends LegacyActionRequest { @Override public ActionRequestValidationException validate() { return null; diff --git a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java index 80d91acfde284..5d0023c0deaad 100644 --- a/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java +++ b/modules/transport-netty4/src/internalClusterTest/java/org/elasticsearch/http/netty4/Netty4ChunkedContinuationsIT.java @@ -15,11 +15,11 @@ import org.elasticsearch.ESNetty4IntegTestCase; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.CountDownActionListener; import org.elasticsearch.action.support.SubscribableListener; @@ -337,14 +337,14 @@ public static class YieldsContinuationsPlugin extends Plugin implements ActionPl static final String ROUTE = "/_test/yields_continuations"; static final String FAIL_INDEX_PARAM = "fail_index"; - private static final ActionType TYPE = new ActionType<>("test:yields_continuations"); + private static final ActionType TYPE = new ActionType<>("test:yields_continuations"); @Override public Collection getActions() { return List.of(new ActionHandler(TYPE, TransportYieldsContinuationsAction.class)); } - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { final int failIndex; public Request(int failIndex) { @@ -525,7 +525,7 @@ public Collection getActions() { return List.of(new ActionHandler(TYPE, TransportInfiniteContinuationsAction.class)); } - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { @Override public ActionRequestValidationException validate() { return null; diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksIT.java index 5607585dca0b8..3f3ee0c5598f9 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/CancellableTasksIT.java @@ -14,11 +14,11 @@ import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.LatchedActionListener; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.GroupedActionListener; @@ -118,6 +118,7 @@ static void randomDescendants(TestRequest request, Set result) { /** * Allow some parts of the request to be completed + * * @return a pending child requests */ static Set allowPartialRequest(TestRequest request) throws Exception { @@ -418,7 +419,7 @@ static void waitForRootTask(ActionFuture rootTask, boolean expectT } } - static class TestRequest extends ActionRequest { + static class TestRequest extends LegacyActionRequest { final int id; final DiscoveryNode node; final List subRequests; diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/tasks/ListTasksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/tasks/ListTasksIT.java index 3419a3d95ebd3..92e740ee8984f 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/tasks/ListTasksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/tasks/ListTasksIT.java @@ -10,11 +10,11 @@ package org.elasticsearch.action.admin.cluster.tasks; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.PlainActionFuture; @@ -142,7 +142,7 @@ public List getActions() { } } - public static class TestRequest extends ActionRequest { + public static class TestRequest extends LegacyActionRequest { @Override public ActionRequestValidationException validate() { return null; diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/rank/MockedRequestActionBasedRerankerIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/rank/MockedRequestActionBasedRerankerIT.java index 43cae6788d4f0..fa9e58d179a26 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/rank/MockedRequestActionBasedRerankerIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/rank/MockedRequestActionBasedRerankerIT.java @@ -16,10 +16,10 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.search.SearchPhaseController; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; @@ -142,7 +142,7 @@ public static class TestRerankingActionType extends ActionType docFeatures; diff --git a/server/src/main/java/org/elasticsearch/action/LegacyActionRequest.java b/server/src/main/java/org/elasticsearch/action/LegacyActionRequest.java new file mode 100644 index 0000000000000..0a19eb7f4d471 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/LegacyActionRequest.java @@ -0,0 +1,30 @@ +/* + * 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.action; + +import org.elasticsearch.common.io.stream.StreamInput; + +import java.io.IOException; + +/** + * An action request with an unspecified response type. + * + * @deprecated Use {@link ActionRequest} with a specific {@link ActionResponse} type. + */ +@Deprecated +public abstract class LegacyActionRequest extends ActionRequest { + public LegacyActionRequest() { + super(); + } + + public LegacyActionRequest(StreamInput in) throws IOException { + super(in); + } +} diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/ClusterFormationInfoAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/ClusterFormationInfoAction.java index f5f12f5d17ba0..bf3b3915b62f6 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/ClusterFormationInfoAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/ClusterFormationInfoAction.java @@ -10,10 +10,10 @@ package org.elasticsearch.action.admin.cluster.coordination; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.coordination.ClusterFormationFailureHelper; @@ -41,7 +41,7 @@ private ClusterFormationInfoAction() { super(NAME); } - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { public Request() {} @@ -105,7 +105,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - ClusterFormationInfoAction.Response response = (ClusterFormationInfoAction.Response) o; + Response response = (Response) o; return clusterFormationState.equals(response.clusterFormationState); } @@ -118,9 +118,7 @@ public int hashCode() { /** * This transport action fetches the ClusterFormationState from a remote node. */ - public static class TransportAction extends HandledTransportAction< - ClusterFormationInfoAction.Request, - ClusterFormationInfoAction.Response> { + public static class TransportAction extends HandledTransportAction { private final Coordinator coordinator; @Inject @@ -129,19 +127,15 @@ public TransportAction(TransportService transportService, ActionFilters actionFi ClusterFormationInfoAction.NAME, transportService, actionFilters, - ClusterFormationInfoAction.Request::new, + Request::new, transportService.getThreadPool().executor(ThreadPool.Names.CLUSTER_COORDINATION) ); this.coordinator = coordinator; } @Override - protected void doExecute( - Task task, - ClusterFormationInfoAction.Request request, - ActionListener listener - ) { - listener.onResponse(new ClusterFormationInfoAction.Response(coordinator.getClusterFormationState())); + protected void doExecute(Task task, Request request, ActionListener listener) { + listener.onResponse(new Response(coordinator.getClusterFormationState())); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/CoordinationDiagnosticsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/CoordinationDiagnosticsAction.java index 47ad8e2610abe..fe2493aa1a640 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/CoordinationDiagnosticsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/CoordinationDiagnosticsAction.java @@ -10,10 +10,10 @@ package org.elasticsearch.action.admin.cluster.coordination; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.coordination.CoordinationDiagnosticsService; @@ -42,7 +42,7 @@ private CoordinationDiagnosticsAction() { super(NAME); } - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { final boolean explain; // Non-private for testing public Request(boolean explain) { @@ -104,7 +104,7 @@ public void writeTo(StreamOutput out) throws IOException { public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - CoordinationDiagnosticsAction.Response response = (CoordinationDiagnosticsAction.Response) o; + Response response = (Response) o; return result.equals(response.result); } @@ -130,14 +130,14 @@ public TransportAction( CoordinationDiagnosticsAction.NAME, transportService, actionFilters, - CoordinationDiagnosticsAction.Request::new, + Request::new, transportService.getThreadPool().executor(ThreadPool.Names.CLUSTER_COORDINATION) ); this.coordinationDiagnosticsService = coordinationDiagnosticsService; } @Override - protected void doExecute(Task task, CoordinationDiagnosticsAction.Request request, ActionListener listener) { + protected void doExecute(Task task, Request request, ActionListener listener) { listener.onResponse(new Response(coordinationDiagnosticsService.diagnoseMasterStability(request.explain))); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/MasterHistoryAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/MasterHistoryAction.java index 40c634968581d..11118a89efe45 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/MasterHistoryAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/coordination/MasterHistoryAction.java @@ -10,10 +10,10 @@ package org.elasticsearch.action.admin.cluster.coordination; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.coordination.MasterHistoryService; @@ -42,7 +42,7 @@ private MasterHistoryAction() { super(NAME); } - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { public Request() {} @@ -94,6 +94,7 @@ public Response(StreamInput in) throws IOException { /** * Returns an ordered list of DiscoveryNodes that the node responding has seen to be master nodes over the last 30 minutes, ordered * oldest first. Note that these DiscoveryNodes can be null. + * * @return a list of DiscoveryNodes that the node responding has seen to be master nodes over the last 30 minutes, ordered oldest * first */ @@ -113,7 +114,7 @@ public void writeTo(StreamOutput out) throws IOException { public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - MasterHistoryAction.Response response = (MasterHistoryAction.Response) o; + Response response = (Response) o; return masterHistory.equals(response.masterHistory); } @@ -136,8 +137,8 @@ public TransportAction(TransportService transportService, ActionFilters actionFi } @Override - protected void doExecute(Task task, MasterHistoryAction.Request request, ActionListener listener) { - listener.onResponse(new MasterHistoryAction.Response(masterHistoryService.getLocalMasterHistory().getRawNodes())); + protected void doExecute(Task task, Request request, ActionListener listener) { + listener.onResponse(new Response(masterHistoryService.getLocalMasterHistory().getRawNodes())); } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskRequest.java index 0b36091cf27e3..736212bdf5419 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/get/GetTaskRequest.java @@ -9,8 +9,8 @@ package org.elasticsearch.action.admin.cluster.node.tasks.get; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.core.TimeValue; @@ -23,7 +23,7 @@ /** * A request to get node tasks */ -public class GetTaskRequest extends ActionRequest { +public class GetTaskRequest extends LegacyActionRequest { private TaskId taskId = TaskId.EMPTY_TASK_ID; private boolean waitForCompletion = false; private TimeValue timeout = null; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesAction.java index 4e52107933c24..dd07ea22d378c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesAction.java @@ -10,10 +10,10 @@ package org.elasticsearch.action.admin.cluster.remote; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.RemoteClusterActionType; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoMetrics; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoRequest; @@ -40,13 +40,10 @@ public class RemoteClusterNodesAction { public static final String NAME = "cluster:internal/remote_cluster/nodes"; - public static final ActionType TYPE = new ActionType<>(NAME); - public static final RemoteClusterActionType REMOTE_TYPE = new RemoteClusterActionType<>( - NAME, - RemoteClusterNodesAction.Response::new - ); + public static final ActionType TYPE = new ActionType<>(NAME); + public static final RemoteClusterActionType REMOTE_TYPE = new RemoteClusterActionType<>(NAME, Response::new); - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { public static final Request ALL_NODES = new Request(false); public static final Request REMOTE_CLUSTER_SERVER_NODES = new Request(true); private final boolean remoteClusterServer; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteInfoRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteInfoRequest.java index 5a95b122f331c..644de985ecd9d 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteInfoRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteInfoRequest.java @@ -9,13 +9,13 @@ package org.elasticsearch.action.admin.cluster.remote; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.common.io.stream.StreamInput; import java.io.IOException; -public final class RemoteInfoRequest extends ActionRequest { +public final class RemoteInfoRequest extends LegacyActionRequest { public RemoteInfoRequest() {} diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/RemoteClusterStatsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/RemoteClusterStatsRequest.java index 6c3c5cbb50ece..8b1c0f3314970 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/RemoteClusterStatsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/stats/RemoteClusterStatsRequest.java @@ -10,8 +10,8 @@ package org.elasticsearch.action.admin.cluster.stats; import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -20,7 +20,7 @@ /** * A request to get cluster level stats from the remote cluster. */ -public class RemoteClusterStatsRequest extends ActionRequest { +public class RemoteClusterStatsRequest extends LegacyActionRequest { public RemoteClusterStatsRequest(StreamInput in) throws IOException { super(in); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetScriptContextRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetScriptContextRequest.java index 63fd3c0b8d1c3..3e27d6fe7df27 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetScriptContextRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetScriptContextRequest.java @@ -8,13 +8,13 @@ */ package org.elasticsearch.action.admin.cluster.storedscripts; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.common.io.stream.StreamInput; import java.io.IOException; -public class GetScriptContextRequest extends ActionRequest { +public class GetScriptContextRequest extends LegacyActionRequest { public GetScriptContextRequest() { super(); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetScriptLanguageRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetScriptLanguageRequest.java index 5e9c923ef2684..7294f02c04cb4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetScriptLanguageRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/storedscripts/GetScriptLanguageRequest.java @@ -9,13 +9,13 @@ package org.elasticsearch.action.admin.cluster.storedscripts; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.common.io.stream.StreamInput; import java.io.IOException; -public class GetScriptLanguageRequest extends ActionRequest { +public class GetScriptLanguageRequest extends LegacyActionRequest { public GetScriptLanguageRequest() { super(); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsRequest.java index 415ccd5bf2a73..1bab7267b108e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsRequest.java @@ -10,9 +10,9 @@ package org.elasticsearch.action.admin.indices.mapping.get; import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -23,11 +23,11 @@ /** * Request the mappings of specific fields - * + *

* Note: there is a new class with the same name for the Java HLRC that uses a typeless format. * Any changes done to this class should go to that client class as well. */ -public class GetFieldMappingsRequest extends ActionRequest implements IndicesRequest.Replaceable { +public class GetFieldMappingsRequest extends LegacyActionRequest implements IndicesRequest.Replaceable { private String[] fields = Strings.EMPTY_ARRAY; @@ -84,7 +84,9 @@ public boolean includeDataStreams() { return true; } - /** @param fields a list of fields to retrieve the mapping for */ + /** + * @param fields a list of fields to retrieve the mapping for + */ public GetFieldMappingsRequest fields(String... fields) { this.fields = fields; return this; @@ -98,7 +100,9 @@ public boolean includeDefaults() { return includeDefaults; } - /** Indicates whether default mapping settings should be returned */ + /** + * Indicates whether default mapping settings should be returned + */ public GetFieldMappingsRequest includeDefaults(boolean includeDefaults) { this.includeDefaults = includeDefaults; return this; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/resolve/ResolveClusterActionRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/resolve/ResolveClusterActionRequest.java index 3f9e0cbf299e6..6e42eddd80216 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/resolve/ResolveClusterActionRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/resolve/ResolveClusterActionRequest.java @@ -11,9 +11,9 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -29,7 +29,7 @@ import java.util.Map; import java.util.Objects; -public class ResolveClusterActionRequest extends ActionRequest implements IndicesRequest.Replaceable { +public class ResolveClusterActionRequest extends LegacyActionRequest implements IndicesRequest.Replaceable { public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandOpen(); public static final String TRANSPORT_VERSION_ERROR_MESSAGE_PREFIX = "ResolveClusterAction requires at least version"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/resolve/ResolveIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/resolve/ResolveIndexAction.java index 1db614a502cc6..6c9e9ddf59e04 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/resolve/ResolveIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/resolve/ResolveIndexAction.java @@ -10,11 +10,11 @@ package org.elasticsearch.action.admin.indices.resolve; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.RemoteClusterActionType; import org.elasticsearch.action.support.ActionFilters; @@ -75,7 +75,7 @@ private ResolveIndexAction() { super(NAME); } - public static class Request extends ActionRequest implements IndicesRequest.Replaceable { + public static class Request extends LegacyActionRequest implements IndicesRequest.Replaceable { public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandOpen(); @@ -517,12 +517,12 @@ protected void doExecute(Task task, Request request, final ActionListener, diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java index 565f2491e5d01..c305d77972fbf 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesNodeRequest.java @@ -10,9 +10,9 @@ package org.elasticsearch.action.fieldcaps; import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.Strings; @@ -30,7 +30,7 @@ import java.util.Map; import java.util.Objects; -class FieldCapabilitiesNodeRequest extends ActionRequest implements IndicesRequest { +class FieldCapabilitiesNodeRequest extends LegacyActionRequest implements IndicesRequest { private final List shardIds; private final String[] fields; diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java index efbde6264e91c..5d4f6c5f0cc53 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/FieldCapabilitiesRequest.java @@ -10,9 +10,9 @@ package org.elasticsearch.action.fieldcaps; import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.ValidateActions; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.Strings; @@ -34,7 +34,7 @@ import java.util.Objects; import java.util.Set; -public final class FieldCapabilitiesRequest extends ActionRequest implements IndicesRequest.Replaceable, ToXContentObject { +public final class FieldCapabilitiesRequest extends LegacyActionRequest implements IndicesRequest.Replaceable, ToXContentObject { public static final String NAME = "field_caps_request"; public static final IndicesOptions DEFAULT_INDICES_OPTIONS = IndicesOptions.strictExpandOpenAndForbidClosed(); diff --git a/server/src/main/java/org/elasticsearch/action/get/MultiGetRequest.java b/server/src/main/java/org/elasticsearch/action/get/MultiGetRequest.java index e6469ab75299e..b73aaffbbf012 100644 --- a/server/src/main/java/org/elasticsearch/action/get/MultiGetRequest.java +++ b/server/src/main/java/org/elasticsearch/action/get/MultiGetRequest.java @@ -11,10 +11,10 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.RealtimeRequest; import org.elasticsearch.action.ValidateActions; import org.elasticsearch.action.support.IndicesOptions; @@ -45,7 +45,7 @@ // It's not possible to suppress teh warning at #realtime(boolean) at a method-level. @SuppressWarnings("unchecked") -public class MultiGetRequest extends ActionRequest +public class MultiGetRequest extends LegacyActionRequest implements Iterable, CompositeIndicesRequest, diff --git a/server/src/main/java/org/elasticsearch/action/get/TransportGetFromTranslogAction.java b/server/src/main/java/org/elasticsearch/action/get/TransportGetFromTranslogAction.java index c4c7d8e5e7c7f..556017c81e799 100644 --- a/server/src/main/java/org/elasticsearch/action/get/TransportGetFromTranslogAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/TransportGetFromTranslogAction.java @@ -12,10 +12,10 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.IndicesOptions; @@ -85,7 +85,7 @@ protected void doExecute(Task task, Request request, ActionListener li }); } - public static class Request extends ActionRequest implements IndicesRequest { + public static class Request extends LegacyActionRequest implements IndicesRequest { private final GetRequest getRequest; private final ShardId shardId; diff --git a/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetFomTranslogAction.java b/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetFomTranslogAction.java index 46893c9240a4e..643d4076f7133 100644 --- a/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetFomTranslogAction.java +++ b/server/src/main/java/org/elasticsearch/action/get/TransportShardMultiGetFomTranslogAction.java @@ -11,9 +11,9 @@ import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.TransportActions; @@ -106,7 +106,7 @@ protected void doExecute(Task task, Request request, ActionListener li }); } - public static class Request extends ActionRequest { + public static class Request extends LegacyActionRequest { private final MultiGetShardRequest multiGetShardRequest; private final ShardId shardId; diff --git a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java index 258aa71d00d12..926c1c2e85000 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java @@ -9,8 +9,8 @@ package org.elasticsearch.action.ingest; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.cluster.metadata.ProjectId; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.ReleasableBytesReference; @@ -39,7 +39,7 @@ import java.util.Map; import java.util.Objects; -public class SimulatePipelineRequest extends ActionRequest implements ToXContentObject { +public class SimulatePipelineRequest extends LegacyActionRequest implements ToXContentObject { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(SimulatePipelineRequest.class); private String id; private boolean verbose; diff --git a/server/src/main/java/org/elasticsearch/action/search/ClearScrollRequest.java b/server/src/main/java/org/elasticsearch/action/search/ClearScrollRequest.java index 4909d8c4d229d..2588576feb0de 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ClearScrollRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/ClearScrollRequest.java @@ -9,8 +9,8 @@ package org.elasticsearch.action.search; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.xcontent.ToXContentObject; @@ -24,7 +24,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public class ClearScrollRequest extends ActionRequest implements ToXContentObject { +public class ClearScrollRequest extends LegacyActionRequest implements ToXContentObject { private List scrollIds; diff --git a/server/src/main/java/org/elasticsearch/action/search/ClosePointInTimeRequest.java b/server/src/main/java/org/elasticsearch/action/search/ClosePointInTimeRequest.java index b36ec586535e8..adace209f5338 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ClosePointInTimeRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/ClosePointInTimeRequest.java @@ -9,8 +9,8 @@ package org.elasticsearch.action.search; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.ValidateActions; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -24,7 +24,7 @@ import java.io.IOException; import java.util.Base64; -public class ClosePointInTimeRequest extends ActionRequest implements ToXContentObject { +public class ClosePointInTimeRequest extends LegacyActionRequest implements ToXContentObject { private static final ParseField ID = new ParseField("id"); private final BytesReference id; diff --git a/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java index 2022180475529..aa728e888b87e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequest.java @@ -9,9 +9,9 @@ package org.elasticsearch.action.search; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.TriFunction; @@ -46,7 +46,7 @@ /** * A multi search API request. */ -public class MultiSearchRequest extends ActionRequest implements CompositeIndicesRequest { +public class MultiSearchRequest extends LegacyActionRequest implements CompositeIndicesRequest { public static final int MAX_CONCURRENT_SEARCH_REQUESTS_DEFAULT = 0; private int maxConcurrentSearchRequests = 0; diff --git a/server/src/main/java/org/elasticsearch/action/search/OpenPointInTimeRequest.java b/server/src/main/java/org/elasticsearch/action/search/OpenPointInTimeRequest.java index d68e2ce1b02b7..aa707d72bc6f1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/OpenPointInTimeRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/OpenPointInTimeRequest.java @@ -10,9 +10,9 @@ package org.elasticsearch.action.search; import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -29,7 +29,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError; -public final class OpenPointInTimeRequest extends ActionRequest implements IndicesRequest.Replaceable { +public final class OpenPointInTimeRequest extends LegacyActionRequest implements IndicesRequest.Replaceable { private String[] indices; private IndicesOptions indicesOptions = DEFAULT_INDICES_OPTIONS; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 862831f33cab5..fda2df81d3f94 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -11,10 +11,11 @@ import org.elasticsearch.TransportVersions; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.LegacyActionRequest; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -45,14 +46,14 @@ /** * A request to execute search against one or more indices (or all). *

- * Note, the search {@link #source(org.elasticsearch.search.builder.SearchSourceBuilder)} + * Note, the search {@link #source(SearchSourceBuilder)} * is required. The search source is the different search options, including aggregations and such. *

* - * @see org.elasticsearch.client.internal.Client#search(SearchRequest) + * @see Client#search(SearchRequest) * @see SearchResponse */ -public class SearchRequest extends ActionRequest implements IndicesRequest.Replaceable, Rewriteable { +public class SearchRequest extends LegacyActionRequest implements IndicesRequest.Replaceable, Rewriteable { public static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Collections.singletonMap("pretty", "false")); @@ -160,12 +161,12 @@ public boolean allowsRemoteIndices() { * Used when a {@link SearchRequest} is created and executed as part of a cross-cluster search request * performing reduction on each cluster in order to minimize network round-trips between the coordinating node and the remote clusters. * - * @param parentTaskId the parent taskId of the original search request + * @param parentTaskId the parent taskId of the original search request * @param originalSearchRequest the original search request - * @param indices the indices to search against - * @param clusterAlias the alias to prefix index names with in the returned search results - * @param absoluteStartMillis the absolute start time to be used on the remote clusters to ensure that the same value is used - * @param finalReduce whether the reduction should be final or not + * @param indices the indices to search against + * @param clusterAlias the alias to prefix index names with in the returned search results + * @param absoluteStartMillis the absolute start time to be used on the remote clusters to ensure that the same value is used + * @param finalReduce whether the reduction should be final or not */ static SearchRequest subSearchRequest( TaskId parentTaskId, @@ -629,7 +630,7 @@ public void setWaitForCheckpointsTimeout(final TimeValue waitForCheckpointsTimeo * the search request expands to exceeds the threshold. This filter roundtrip can limit the number of shards significantly if for * instance a shard can not match any documents based on its rewrite method ie. if date filters are mandatory to match but the shard * bounds and the query are disjoint. - * + *

* When unspecified, the pre-filter phase is executed if any of these conditions is met: *