From 69150c8de16335e0c3444d6bf5ccb4b705d359ad Mon Sep 17 00:00:00 2001 From: Pooya Salehi Date: Mon, 4 Nov 2024 14:01:16 +0100 Subject: [PATCH 01/35] Export desired balance node weight and its components as metrics (#115854) Note that this includes only the three node-level weight components (out of the four), as we were not sure how to aggregate and expose the index-specific component and how useful it will be at all. - some of the weight components are also recalculated or exposed as stats (not APM metrics) else where (e.g. `AllocationStatsService`) but since they are available right where we calculate the weight (which we also want), I have just exported all of them together. - How to pass the weight from the BalancedAllocator which is used as a delegated allocator in the desired balance allocator, and from there to the reconciler where we publish, could probably also be done differently, but using `RoutingNodes` and `DesiredBalance` seemed to make more sense to me. Not sure if it is blasphemy for those more familiar with the allocation code! - I liked the `DesiredBalanceMetrics` and how its used so I tried to clean up its existing usage a bit and colocate the new metrics. Relates ES-9866 --- .../DesiredBalanceReconcilerMetricsIT.java | 86 +++++++++++++- .../cluster/routing/RoutingNodes.java | 12 ++ .../allocator/BalancedShardsAllocator.java | 28 ++++- .../allocation/allocator/DesiredBalance.java | 13 +- .../allocator/DesiredBalanceComputer.java | 2 +- .../allocator/DesiredBalanceMetrics.java | 112 +++++++++++++++++- .../allocator/DesiredBalanceReconciler.java | 13 +- .../allocator/DesiredBalanceMetricsTests.java | 9 +- .../DesiredBalanceShardsAllocatorTests.java | 1 + 9 files changed, 255 insertions(+), 21 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconcilerMetricsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconcilerMetricsIT.java index cb279c93b402e..bfe46dc4c90f2 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconcilerMetricsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconcilerMetricsIT.java @@ -9,6 +9,7 @@ package org.elasticsearch.cluster.routing.allocation.allocator; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.PluginsService; @@ -17,10 +18,16 @@ import org.hamcrest.Matcher; import java.util.Collection; +import java.util.stream.Collectors; import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.in; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) public class DesiredBalanceReconcilerMetricsIT extends ESIntegTestCase { @Override @@ -31,6 +38,7 @@ protected Collection> nodePlugins() { public void testDesiredBalanceGaugeMetricsAreOnlyPublishedByCurrentMaster() throws Exception { internalCluster().ensureAtLeastNumDataNodes(2); prepareCreate("test").setSettings(indexSettings(2, 1)).get(); + indexRandom(randomBoolean(), "test", between(50, 100)); ensureGreen(); assertOnlyMasterIsPublishingMetrics(); @@ -45,6 +53,59 @@ public void testDesiredBalanceGaugeMetricsAreOnlyPublishedByCurrentMaster() thro } } + public void testDesiredBalanceNodeWeightMetrics() { + internalCluster().startNodes(2); + prepareCreate("test").setSettings(indexSettings(2, 1)).get(); + indexRandom(randomBoolean(), "test", between(50, 100)); + ensureGreen(); + final var telemetryPlugin = getTelemetryPlugin(internalCluster().getMasterName()); + telemetryPlugin.collect(); + assertThat(telemetryPlugin.getLongGaugeMeasurement(DesiredBalanceMetrics.UNASSIGNED_SHARDS_METRIC_NAME), not(empty())); + assertThat(telemetryPlugin.getLongGaugeMeasurement(DesiredBalanceMetrics.TOTAL_SHARDS_METRIC_NAME), not(empty())); + assertThat(telemetryPlugin.getLongGaugeMeasurement(DesiredBalanceMetrics.UNDESIRED_ALLOCATION_COUNT_METRIC_NAME), not(empty())); + assertThat(telemetryPlugin.getDoubleGaugeMeasurement(DesiredBalanceMetrics.UNDESIRED_ALLOCATION_RATIO_METRIC_NAME), not(empty())); + + var nodeIds = internalCluster().clusterService().state().nodes().stream().map(DiscoveryNode::getId).collect(Collectors.toSet()); + var nodeNames = internalCluster().clusterService().state().nodes().stream().map(DiscoveryNode::getName).collect(Collectors.toSet()); + + final var nodeWeightsMetrics = telemetryPlugin.getDoubleGaugeMeasurement( + DesiredBalanceMetrics.DESIRED_BALANCE_NODE_WEIGHT_METRIC_NAME + ); + assertThat(nodeWeightsMetrics.size(), equalTo(2)); + for (var nodeStat : nodeWeightsMetrics) { + assertThat(nodeStat.value().doubleValue(), greaterThanOrEqualTo(0.0)); + assertThat((String) nodeStat.attributes().get("node_id"), is(in(nodeIds))); + assertThat((String) nodeStat.attributes().get("node_name"), is(in(nodeNames))); + } + final var nodeShardCountMetrics = telemetryPlugin.getLongGaugeMeasurement( + DesiredBalanceMetrics.DESIRED_BALANCE_NODE_SHARD_COUNT_METRIC_NAME + ); + assertThat(nodeShardCountMetrics.size(), equalTo(2)); + for (var nodeStat : nodeShardCountMetrics) { + assertThat(nodeStat.value().longValue(), equalTo(2L)); + assertThat((String) nodeStat.attributes().get("node_id"), is(in(nodeIds))); + assertThat((String) nodeStat.attributes().get("node_name"), is(in(nodeNames))); + } + final var nodeWriteLoadMetrics = telemetryPlugin.getDoubleGaugeMeasurement( + DesiredBalanceMetrics.DESIRED_BALANCE_NODE_WRITE_LOAD_METRIC_NAME + ); + assertThat(nodeWriteLoadMetrics.size(), equalTo(2)); + for (var nodeStat : nodeWriteLoadMetrics) { + assertThat(nodeStat.value().doubleValue(), greaterThanOrEqualTo(0.0)); + assertThat((String) nodeStat.attributes().get("node_id"), is(in(nodeIds))); + assertThat((String) nodeStat.attributes().get("node_name"), is(in(nodeNames))); + } + final var nodeDiskUsageMetrics = telemetryPlugin.getDoubleGaugeMeasurement( + DesiredBalanceMetrics.DESIRED_BALANCE_NODE_DISK_USAGE_METRIC_NAME + ); + assertThat(nodeDiskUsageMetrics.size(), equalTo(2)); + for (var nodeStat : nodeDiskUsageMetrics) { + assertThat(nodeStat.value().doubleValue(), greaterThanOrEqualTo(0.0)); + assertThat((String) nodeStat.attributes().get("node_id"), is(in(nodeIds))); + assertThat((String) nodeStat.attributes().get("node_name"), is(in(nodeNames))); + } + } + private static void assertOnlyMasterIsPublishingMetrics() { String masterNodeName = internalCluster().getMasterName(); String[] nodeNames = internalCluster().getNodeNames(); @@ -54,10 +115,7 @@ private static void assertOnlyMasterIsPublishingMetrics() { } private static void assertMetricsAreBeingPublished(String nodeName, boolean shouldBePublishing) { - final TestTelemetryPlugin testTelemetryPlugin = internalCluster().getInstance(PluginsService.class, nodeName) - .filterPlugins(TestTelemetryPlugin.class) - .findFirst() - .orElseThrow(); + final TestTelemetryPlugin testTelemetryPlugin = getTelemetryPlugin(nodeName); testTelemetryPlugin.resetMeter(); testTelemetryPlugin.collect(); Matcher> matcher = shouldBePublishing ? not(empty()) : empty(); @@ -65,5 +123,25 @@ private static void assertMetricsAreBeingPublished(String nodeName, boolean shou assertThat(testTelemetryPlugin.getLongGaugeMeasurement(DesiredBalanceMetrics.TOTAL_SHARDS_METRIC_NAME), matcher); assertThat(testTelemetryPlugin.getLongGaugeMeasurement(DesiredBalanceMetrics.UNDESIRED_ALLOCATION_COUNT_METRIC_NAME), matcher); assertThat(testTelemetryPlugin.getDoubleGaugeMeasurement(DesiredBalanceMetrics.UNDESIRED_ALLOCATION_RATIO_METRIC_NAME), matcher); + assertThat(testTelemetryPlugin.getDoubleGaugeMeasurement(DesiredBalanceMetrics.DESIRED_BALANCE_NODE_WEIGHT_METRIC_NAME), matcher); + assertThat( + testTelemetryPlugin.getDoubleGaugeMeasurement(DesiredBalanceMetrics.DESIRED_BALANCE_NODE_WRITE_LOAD_METRIC_NAME), + matcher + ); + assertThat( + testTelemetryPlugin.getDoubleGaugeMeasurement(DesiredBalanceMetrics.DESIRED_BALANCE_NODE_DISK_USAGE_METRIC_NAME), + matcher + ); + assertThat( + testTelemetryPlugin.getLongGaugeMeasurement(DesiredBalanceMetrics.DESIRED_BALANCE_NODE_SHARD_COUNT_METRIC_NAME), + matcher + ); + } + + private static TestTelemetryPlugin getTelemetryPlugin(String nodeName) { + return internalCluster().getInstance(PluginsService.class, nodeName) + .filterPlugins(TestTelemetryPlugin.class) + .findFirst() + .orElseThrow(); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index 7f03a2861a807..fb5393c1961f8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.ExistingShardsAllocator; +import org.elasticsearch.cluster.routing.allocation.allocator.DesiredBalanceMetrics; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.util.Maps; import org.elasticsearch.core.Assertions; @@ -76,6 +77,8 @@ public class RoutingNodes implements Iterable { private final Map> attributeValuesByAttribute; private final Map recoveriesPerNode; + private Map balanceWeightStatsPerNode; + /** * Creates an immutable instance from the {@link RoutingTable} and {@link DiscoveryNodes} found in a cluster state. Used to initialize * the routing nodes in {@link ClusterState#getRoutingNodes()}. This method should not be used directly, use @@ -89,6 +92,14 @@ public static RoutingNodes mutable(RoutingTable routingTable, DiscoveryNodes dis return new RoutingNodes(routingTable, discoveryNodes, false); } + public void setBalanceWeightStatsPerNode(Map weightStatsPerNode) { + this.balanceWeightStatsPerNode = weightStatsPerNode; + } + + public Map getBalanceWeightStatsPerNode() { + return balanceWeightStatsPerNode; + } + private RoutingNodes(RoutingTable routingTable, DiscoveryNodes discoveryNodes, boolean readOnly) { this.readOnly = readOnly; this.recoveriesPerNode = new HashMap<>(); @@ -97,6 +108,7 @@ private RoutingNodes(RoutingTable routingTable, DiscoveryNodes discoveryNodes, b this.unassignedShards = new UnassignedShards(this); this.attributeValuesByAttribute = Collections.synchronizedMap(new HashMap<>()); + balanceWeightStatsPerNode = Maps.newMapWithExpectedSize(discoveryNodes.getDataNodes().size()); nodesToShards = Maps.newMapWithExpectedSize(discoveryNodes.getDataNodes().size()); // fill in the nodeToShards with the "live" nodes var dataNodes = discoveryNodes.getDataNodes().keySet(); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index 108bb83d90871..5b8fb0c7e9203 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -16,6 +16,7 @@ import org.elasticsearch.cluster.ClusterInfo; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; @@ -159,6 +160,25 @@ public void allocate(RoutingAllocation allocation) { balancer.allocateUnassigned(); balancer.moveShards(); balancer.balance(); + + collectAndRecordNodeWeightStats(balancer, weightFunction, allocation); + } + + private void collectAndRecordNodeWeightStats(Balancer balancer, WeightFunction weightFunction, RoutingAllocation allocation) { + Map nodeLevelWeights = new HashMap<>(); + for (var entry : balancer.nodes.entrySet()) { + var node = entry.getValue(); + nodeLevelWeights.put( + node.routingNode.node(), + new DesiredBalanceMetrics.NodeWeightStats( + node.numShards(), + node.diskUsageInBytes(), + node.writeLoad(), + weightFunction.nodeWeight(balancer, node) + ) + ); + } + allocation.routingNodes().setBalanceWeightStatsPerNode(nodeLevelWeights); } @Override @@ -275,11 +295,15 @@ private static class WeightFunction { } float weight(Balancer balancer, ModelNode node, String index) { - final float weightShard = node.numShards() - balancer.avgShardsPerNode(); final float weightIndex = node.numShards(index) - balancer.avgShardsPerNode(index); + return nodeWeight(balancer, node) + theta1 * weightIndex; + } + + float nodeWeight(Balancer balancer, ModelNode node) { + final float weightShard = node.numShards() - balancer.avgShardsPerNode(); final float ingestLoad = (float) (node.writeLoad() - balancer.avgWriteLoadPerNode()); final float diskUsage = (float) (node.diskUsageInBytes() - balancer.avgDiskUsageInBytesPerNode()); - return theta0 * weightShard + theta1 * weightIndex + theta2 * ingestLoad + theta3 * diskUsage; + return theta0 * weightShard + theta2 * ingestLoad + theta3 * diskUsage; } float minWeightDelta(Balancer balancer, String index) { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalance.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalance.java index 5224d1d920b2e..aeedbb56b9df2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalance.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalance.java @@ -9,6 +9,7 @@ package org.elasticsearch.cluster.routing.allocation.allocator; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.ShardId; @@ -19,8 +20,18 @@ * The desired balance of the cluster, indicating which nodes should hold a copy of each shard. * * @param assignments a set of the (persistent) node IDs to which each {@link ShardId} should be allocated + * @param weightsPerNode The node weights calculated based on + * {@link org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator.WeightFunction#nodeWeight} */ -public record DesiredBalance(long lastConvergedIndex, Map assignments) { +public record DesiredBalance( + long lastConvergedIndex, + Map assignments, + Map weightsPerNode +) { + + public DesiredBalance(long lastConvergedIndex, Map assignments) { + this(lastConvergedIndex, assignments, Map.of()); + } public static final DesiredBalance INITIAL = new DesiredBalance(-1, Map.of()); diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputer.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputer.java index 2acb4827a9585..56c48492a2051 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputer.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceComputer.java @@ -368,7 +368,7 @@ public DesiredBalance compute( } long lastConvergedIndex = hasChanges ? previousDesiredBalance.lastConvergedIndex() : desiredBalanceInput.index(); - return new DesiredBalance(lastConvergedIndex, assignments); + return new DesiredBalance(lastConvergedIndex, assignments, routingNodes.getBalanceWeightStatsPerNode()); } private static Map collectShardAssignments(RoutingNodes routingNodes) { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceMetrics.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceMetrics.java index 436f1ac38c0c2..d8a2d01f56dff 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceMetrics.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceMetrics.java @@ -9,39 +9,63 @@ package org.elasticsearch.cluster.routing.allocation.allocator; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.telemetry.metric.DoubleWithAttributes; import org.elasticsearch.telemetry.metric.LongWithAttributes; import org.elasticsearch.telemetry.metric.MeterRegistry; +import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; public class DesiredBalanceMetrics { + public record AllocationStats(long unassignedShards, long totalAllocations, long undesiredAllocationsExcludingShuttingDownNodes) {} + + public record NodeWeightStats(long shardCount, double diskUsageInBytes, double writeLoad, double nodeWeight) {} + public static final DesiredBalanceMetrics NOOP = new DesiredBalanceMetrics(MeterRegistry.NOOP); public static final String UNASSIGNED_SHARDS_METRIC_NAME = "es.allocator.desired_balance.shards.unassigned.current"; public static final String TOTAL_SHARDS_METRIC_NAME = "es.allocator.desired_balance.shards.current"; public static final String UNDESIRED_ALLOCATION_COUNT_METRIC_NAME = "es.allocator.desired_balance.allocations.undesired.current"; public static final String UNDESIRED_ALLOCATION_RATIO_METRIC_NAME = "es.allocator.desired_balance.allocations.undesired.ratio"; + public static final String DESIRED_BALANCE_NODE_WEIGHT_METRIC_NAME = "es.allocator.desired_balance.allocations.node_weight.current"; + public static final String DESIRED_BALANCE_NODE_SHARD_COUNT_METRIC_NAME = + "es.allocator.desired_balance.allocations.node_shard_count.current"; + public static final String DESIRED_BALANCE_NODE_WRITE_LOAD_METRIC_NAME = + "es.allocator.desired_balance.allocations.node_write_load.current"; + public static final String DESIRED_BALANCE_NODE_DISK_USAGE_METRIC_NAME = + "es.allocator.desired_balance.allocations.node_disk_usage_bytes.current"; + public static final AllocationStats EMPTY_ALLOCATION_STATS = new AllocationStats(-1, -1, -1); private volatile boolean nodeIsMaster = false; - /** * Number of unassigned shards during last reconciliation */ private volatile long unassignedShards; + /** * Total number of assigned shards during last reconciliation */ private volatile long totalAllocations; + /** * Number of assigned shards during last reconciliation that are not allocated on desired node and need to be moved */ private volatile long undesiredAllocations; - public void updateMetrics(long unassignedShards, long totalAllocations, long undesiredAllocations) { - this.unassignedShards = unassignedShards; - this.totalAllocations = totalAllocations; - this.undesiredAllocations = undesiredAllocations; + private final AtomicReference> weightStatsPerNodeRef = new AtomicReference<>(Map.of()); + + public void updateMetrics(AllocationStats allocationStats, Map weightStatsPerNode) { + assert allocationStats != null : "allocation stats cannot be null"; + assert weightStatsPerNode != null : "node balance weight stats cannot be null"; + if (allocationStats != EMPTY_ALLOCATION_STATS) { + this.unassignedShards = allocationStats.unassignedShards; + this.totalAllocations = allocationStats.totalAllocations; + this.undesiredAllocations = allocationStats.undesiredAllocationsExcludingShuttingDownNodes; + } + weightStatsPerNodeRef.set(weightStatsPerNode); } public DesiredBalanceMetrics(MeterRegistry meterRegistry) { @@ -64,6 +88,30 @@ public DesiredBalanceMetrics(MeterRegistry meterRegistry) { "1", this::getUndesiredAllocationsRatioMetrics ); + meterRegistry.registerDoublesGauge( + DESIRED_BALANCE_NODE_WEIGHT_METRIC_NAME, + "Weight of nodes in the computed desired balance", + "unit", + this::getDesiredBalanceNodeWeightMetrics + ); + meterRegistry.registerDoublesGauge( + DESIRED_BALANCE_NODE_WRITE_LOAD_METRIC_NAME, + "Write load of nodes in the computed desired balance", + "threads", + this::getDesiredBalanceNodeWriteLoadMetrics + ); + meterRegistry.registerDoublesGauge( + DESIRED_BALANCE_NODE_DISK_USAGE_METRIC_NAME, + "Disk usage of nodes in the computed desired balance", + "bytes", + this::getDesiredBalanceNodeDiskUsageMetrics + ); + meterRegistry.registerLongsGauge( + DESIRED_BALANCE_NODE_SHARD_COUNT_METRIC_NAME, + "Shard count of nodes in the computed desired balance", + "unit", + this::getDesiredBalanceNodeShardCountMetrics + ); } public void setNodeIsMaster(boolean nodeIsMaster) { @@ -86,6 +134,59 @@ private List getUnassignedShardsMetrics() { return getIfPublishing(unassignedShards); } + private List getDesiredBalanceNodeWeightMetrics() { + if (nodeIsMaster == false) { + return List.of(); + } + var stats = weightStatsPerNodeRef.get(); + List doubles = new ArrayList<>(stats.size()); + for (var node : stats.keySet()) { + var stat = stats.get(node); + doubles.add(new DoubleWithAttributes(stat.nodeWeight(), getNodeAttributes(node))); + } + return doubles; + } + + private List getDesiredBalanceNodeWriteLoadMetrics() { + if (nodeIsMaster == false) { + return List.of(); + } + var stats = weightStatsPerNodeRef.get(); + List doubles = new ArrayList<>(stats.size()); + for (var node : stats.keySet()) { + doubles.add(new DoubleWithAttributes(stats.get(node).writeLoad(), getNodeAttributes(node))); + } + return doubles; + } + + private List getDesiredBalanceNodeDiskUsageMetrics() { + if (nodeIsMaster == false) { + return List.of(); + } + var stats = weightStatsPerNodeRef.get(); + List doubles = new ArrayList<>(stats.size()); + for (var node : stats.keySet()) { + doubles.add(new DoubleWithAttributes(stats.get(node).diskUsageInBytes(), getNodeAttributes(node))); + } + return doubles; + } + + private List getDesiredBalanceNodeShardCountMetrics() { + if (nodeIsMaster == false) { + return List.of(); + } + var stats = weightStatsPerNodeRef.get(); + List values = new ArrayList<>(stats.size()); + for (var node : stats.keySet()) { + values.add(new LongWithAttributes(stats.get(node).shardCount(), getNodeAttributes(node))); + } + return values; + } + + private Map getNodeAttributes(DiscoveryNode node) { + return Map.of("node_id", node.getId(), "node_name", node.getName()); + } + private List getTotalAllocationsMetrics() { return getIfPublishing(totalAllocations); } @@ -114,5 +215,6 @@ public void zeroAllMetrics() { unassignedShards = 0; totalAllocations = 0; undesiredAllocations = 0; + weightStatsPerNodeRef.set(Map.of()); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconciler.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconciler.java index dced9214a3245..129144a3d734b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconciler.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconciler.java @@ -21,6 +21,7 @@ import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; +import org.elasticsearch.cluster.routing.allocation.allocator.DesiredBalanceMetrics.AllocationStats; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.ClusterSettings; @@ -138,9 +139,11 @@ void run() { moveShards(); // 3. move any other shards that are desired elsewhere logger.trace("Reconciler#balance"); - balance(); + var allocationStats = balance(); logger.debug("Reconciliation is complete"); + + desiredBalanceMetrics.updateMetrics(allocationStats, desiredBalance.weightsPerNode()); } } @@ -464,9 +467,9 @@ private void moveShards() { } } - private void balance() { + private AllocationStats balance() { if (allocation.deciders().canRebalance(allocation).type() != Decision.Type.YES) { - return; + return DesiredBalanceMetrics.EMPTY_ALLOCATION_STATS; } int unassignedShards = routingNodes.unassigned().size() + routingNodes.unassigned().ignored().size(); @@ -532,9 +535,9 @@ private void balance() { } } - desiredBalanceMetrics.updateMetrics(unassignedShards, totalAllocations, undesiredAllocationsExcludingShuttingDownNodes); - maybeLogUndesiredAllocationsWarning(totalAllocations, undesiredAllocationsExcludingShuttingDownNodes, routingNodes.size()); + + return new AllocationStats(unassignedShards, totalAllocations, undesiredAllocationsExcludingShuttingDownNodes); } private void maybeLogUndesiredAllocationsWarning(int totalAllocations, int undesiredAllocations, int nodeCount) { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceMetricsTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceMetricsTests.java index 2c642da665051..85dc5c9dcd6a9 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceMetricsTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceMetricsTests.java @@ -9,11 +9,14 @@ package org.elasticsearch.cluster.routing.allocation.allocator; +import org.elasticsearch.cluster.routing.allocation.allocator.DesiredBalanceMetrics.AllocationStats; import org.elasticsearch.telemetry.InstrumentType; import org.elasticsearch.telemetry.RecordingMeterRegistry; import org.elasticsearch.telemetry.metric.MeterRegistry; import org.elasticsearch.test.ESTestCase; +import java.util.Map; + import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; @@ -24,7 +27,7 @@ public void testZeroAllMetrics() { long unassignedShards = randomNonNegativeLong(); long totalAllocations = randomNonNegativeLong(); long undesiredAllocations = randomNonNegativeLong(); - metrics.updateMetrics(unassignedShards, totalAllocations, undesiredAllocations); + metrics.updateMetrics(new AllocationStats(unassignedShards, totalAllocations, undesiredAllocations), Map.of()); assertEquals(totalAllocations, metrics.totalAllocations()); assertEquals(unassignedShards, metrics.unassignedShards()); assertEquals(undesiredAllocations, metrics.undesiredAllocations()); @@ -41,7 +44,7 @@ public void testMetricsAreOnlyPublishedWhenNodeIsMaster() { long unassignedShards = randomNonNegativeLong(); long totalAllocations = randomLongBetween(100, 10000000); long undesiredAllocations = randomLongBetween(0, totalAllocations); - metrics.updateMetrics(unassignedShards, totalAllocations, undesiredAllocations); + metrics.updateMetrics(new AllocationStats(unassignedShards, totalAllocations, undesiredAllocations), Map.of()); // Collect when not master meterRegistry.getRecorder().collect(); @@ -101,7 +104,7 @@ public void testUndesiredAllocationRatioIsZeroWhenTotalShardsIsZero() { RecordingMeterRegistry meterRegistry = new RecordingMeterRegistry(); DesiredBalanceMetrics metrics = new DesiredBalanceMetrics(meterRegistry); long unassignedShards = randomNonNegativeLong(); - metrics.updateMetrics(unassignedShards, 0, 0); + metrics.updateMetrics(new AllocationStats(unassignedShards, 0, 0), Map.of()); metrics.setNodeIsMaster(true); meterRegistry.getRecorder().collect(); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocatorTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocatorTests.java index 1d2f6cffa3777..739f81ed6d110 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceShardsAllocatorTests.java @@ -808,6 +808,7 @@ public void allocate(RoutingAllocation allocation) { unassignedIterator.next(); unassignedIterator.initialize(dataNodeId, null, 0L, allocation.changes()); } + allocation.routingNodes().setBalanceWeightStatsPerNode(Map.of()); } @Override From b2781c76f5a8986533aa08cbc13fbba5e116ac5e Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Tue, 5 Nov 2024 01:14:17 +1100 Subject: [PATCH 02/35] Mute org.elasticsearch.xpack.test.rest.XPackRestIT test {p0=terms_enum/10_basic/Test security} #116178 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index d7b4c472273a6..3a471ad5d12d2 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -282,6 +282,9 @@ tests: - class: org.elasticsearch.xpack.test.rest.XPackRestIT method: test {p0=ml/forecast/Test forecast unknown job} issue: https://github.com/elastic/elasticsearch/issues/116150 +- class: org.elasticsearch.xpack.test.rest.XPackRestIT + method: test {p0=terms_enum/10_basic/Test security} + issue: https://github.com/elastic/elasticsearch/issues/116178 # Examples: # From 744eb507f6d74aba45a960595aab0cd93bb8d48e Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Mon, 4 Nov 2024 09:32:53 -0500 Subject: [PATCH 03/35] [ESQL] clean up date trunc tests (#116111) While working on #110008 I discovered that the Date Trunc tests were only running in folding mode, because the interval types are marked as not representable. The correct way to test this is to set the forceLiteral flag for those fields, which will (as the name suggests) force them to be literals even in non-folding tests. Doing that turned up errors in the evaluatorToString tests, which I fixed. There are two big changes here. First, the second parameter to the evaluator is a Rounding instance, not the actual interval. Since Rounding includes some information about the specific rounding in the toString results, I am just using a starts with matcher to validate the majority of the string, rather than trying to reconstruct the expected rounding string. Second, passing in a literal null for the interval parameter folds the whole expression to null, and thus a completely different toString. I added a clause in AnyNullIsNull to account for this. While I was in there, I moved some specific test cases to a different file. I know moving code is something we're trying to minimize right now, but this seemed worth it. The tests in question do not depend on the parameters of the test case, but all methods in the class get run for every set of parameters. This was causing these tests to be run many times with the same values, which bloats our test run time and test count. Moving them to a distinct class means they'll only be executed once per test run. I feel like this benefit outweighs the cost of git history complexity. --- .../kibana/definition/date_diff.json | 2 +- .../esql/functions/signature/match.svg | 2 +- .../function/AbstractFunctionTestCase.java | 3 +- .../scalar/date/DateTruncRoundingTests.java | 111 ++++++++++++++++++ .../function/scalar/date/DateTruncTests.java | 96 ++------------- 5 files changed, 123 insertions(+), 91 deletions(-) create mode 100644 x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncRoundingTests.java diff --git a/docs/reference/esql/functions/kibana/definition/date_diff.json b/docs/reference/esql/functions/kibana/definition/date_diff.json index a4812af5f930b..d32028d455348 100644 --- a/docs/reference/esql/functions/kibana/definition/date_diff.json +++ b/docs/reference/esql/functions/kibana/definition/date_diff.json @@ -55,7 +55,7 @@ ], "examples" : [ "ROW date1 = TO_DATETIME(\"2023-12-02T11:00:00.000Z\"), date2 = TO_DATETIME(\"2023-12-02T11:00:00.001Z\")\n| EVAL dd_ms = DATE_DIFF(\"microseconds\", date1, date2)", - "ROW end_23=\"2023-12-31T23:59:59.999Z\"::DATETIME,\n start_24=\"2024-01-01T00:00:00.000Z\"::DATETIME,\n end_24=\"2024-12-31T23:59:59.999\"::DATETIME\n| EVAL end23_to_start24=DATE_DIFF(\"year\", end_23, start_24)\n| EVAL end23_to_end24=DATE_DIFF(\"year\", end_23, end_24)\n| EVAL start_to_end_24=DATE_DIFF(\"year\", start_24, end_24)" + "ROW end_23=TO_DATETIME(\"2023-12-31T23:59:59.999Z\"),\n start_24=TO_DATETIME(\"2024-01-01T00:00:00.000Z\"),\n end_24=TO_DATETIME(\"2024-12-31T23:59:59.999\")\n| EVAL end23_to_start24=DATE_DIFF(\"year\", end_23, start_24)\n| EVAL end23_to_end24=DATE_DIFF(\"year\", end_23, end_24)\n| EVAL start_to_end_24=DATE_DIFF(\"year\", start_24, end_24)" ], "preview" : false, "snapshot_only" : false diff --git a/docs/reference/esql/functions/signature/match.svg b/docs/reference/esql/functions/signature/match.svg index e7bb001247a9d..14ddb87468e70 100644 --- a/docs/reference/esql/functions/signature/match.svg +++ b/docs/reference/esql/functions/signature/match.svg @@ -1 +1 @@ -MATCH(field,query) +MATCH(field,query) \ No newline at end of file diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java index c05f8e0990b3c..6a552f400d36e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/AbstractFunctionTestCase.java @@ -67,6 +67,7 @@ import org.elasticsearch.xpack.esql.planner.PlannerUtils; import org.elasticsearch.xpack.esql.session.Configuration; import org.hamcrest.Matcher; +import org.hamcrest.Matchers; import org.junit.After; import org.junit.AfterClass; @@ -170,7 +171,7 @@ protected static List anyNullIsNull(boolean entirelyNullPreser (nullPosition, nullValueDataType, original) -> entirelyNullPreservesType == false && nullValueDataType == DataType.NULL && original.getData().size() == 1 ? DataType.NULL : original.expectedType(), - (nullPosition, nullData, original) -> original + (nullPosition, nullData, original) -> nullData.isForceLiteral() ? Matchers.equalTo("LiteralsEvaluator[lit=null]") : original ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncRoundingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncRoundingTests.java new file mode 100644 index 0000000000000..5af5c8e493177 --- /dev/null +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncRoundingTests.java @@ -0,0 +1,111 @@ +/* + * 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.date; + +import org.elasticsearch.common.Rounding; +import org.elasticsearch.test.ESTestCase; + +import java.time.Duration; +import java.time.Instant; +import java.time.Period; + +import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.createRounding; +import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.process; +import static org.hamcrest.Matchers.containsString; + +/** + * This class supplements {@link DateTruncTests}. The tests in this class are not run via the parametrized runner, + * and exercise specific helper functions within the class. + */ +public class DateTruncRoundingTests extends ESTestCase { + + public void testCreateRoundingDuration() { + Rounding.Prepared rounding; + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(Duration.ofHours(0))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + e = expectThrows(IllegalArgumentException.class, () -> createRounding(Duration.ofHours(-10))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + rounding = createRounding(Duration.ofHours(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.HOUR_OF_DAY), 0d); + + rounding = createRounding(Duration.ofHours(10)); + assertEquals(10, rounding.roundingSize(Rounding.DateTimeUnit.HOUR_OF_DAY), 0d); + + rounding = createRounding(Duration.ofMinutes(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); + + rounding = createRounding(Duration.ofMinutes(100)); + assertEquals(100, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); + + rounding = createRounding(Duration.ofSeconds(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.SECOND_OF_MINUTE), 0d); + + rounding = createRounding(Duration.ofSeconds(120)); + assertEquals(120, rounding.roundingSize(Rounding.DateTimeUnit.SECOND_OF_MINUTE), 0d); + + rounding = createRounding(Duration.ofSeconds(60).plusMinutes(5).plusHours(1)); + assertEquals(1 + 5 + 60, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); + } + + public void testCreateRoundingPeriod() { + Rounding.Prepared rounding; + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofMonths(0))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofYears(-10))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.of(0, 1, 1))); + assertThat(e.getMessage(), containsString("Time interval with multiple periods is not supported")); + + rounding = createRounding(Period.ofDays(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.DAY_OF_MONTH), 0d); + + rounding = createRounding(Period.ofDays(4)); + assertEquals(4, rounding.roundingSize(Rounding.DateTimeUnit.DAY_OF_MONTH), 0d); + + rounding = createRounding(Period.ofDays(7)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR), 0d); + + rounding = createRounding(Period.ofMonths(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.MONTH_OF_YEAR), 0d); + + rounding = createRounding(Period.ofMonths(3)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.QUARTER_OF_YEAR), 0d); + + rounding = createRounding(Period.ofYears(1)); + assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.YEAR_OF_CENTURY), 0d); + + e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofYears(3))); + assertThat(e.getMessage(), containsString("Time interval is not supported")); + } + + public void testCreateRoundingNullInterval() { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(null)); + assertThat(e.getMessage(), containsString("Time interval is not supported")); + } + + public void testDateTruncFunction() { + long ts = toMillis("2023-02-17T10:25:33.38Z"); + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> process(ts, createRounding(Period.ofDays(-1)))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + + e = expectThrows(IllegalArgumentException.class, () -> process(ts, createRounding(Duration.ofHours(-1)))); + assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); + } + + private static long toMillis(String timestamp) { + return Instant.parse(timestamp).toEpochMilli(); + } + +} diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java index 48b23ed5c8840..0e4968cc2a504 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/scalar/date/DateTruncTests.java @@ -10,12 +10,12 @@ import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.common.Rounding; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.expression.function.AbstractScalarFunctionTestCase; import org.elasticsearch.xpack.esql.expression.function.TestCaseSupplier; +import org.hamcrest.Matchers; import java.time.Duration; import java.time.Instant; @@ -23,11 +23,11 @@ import java.util.List; import java.util.function.Supplier; -import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.createRounding; -import static org.elasticsearch.xpack.esql.expression.function.scalar.date.DateTrunc.process; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +/** + * Parameterized testing for {@link DateTrunc}. See also {@link DateTruncRoundingTests} for non-parametrized tests. + */ public class DateTruncTests extends AbstractScalarFunctionTestCase { public DateTruncTests(@Name("TestCase") Supplier testCaseSupplier) { @@ -61,95 +61,15 @@ public static Iterable parameters() { }); } - public void testCreateRoundingDuration() { - Rounding.Prepared rounding; - - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(Duration.ofHours(0))); - assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); - - e = expectThrows(IllegalArgumentException.class, () -> createRounding(Duration.ofHours(-10))); - assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); - - rounding = createRounding(Duration.ofHours(1)); - assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.HOUR_OF_DAY), 0d); - - rounding = createRounding(Duration.ofHours(10)); - assertEquals(10, rounding.roundingSize(Rounding.DateTimeUnit.HOUR_OF_DAY), 0d); - - rounding = createRounding(Duration.ofMinutes(1)); - assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); - - rounding = createRounding(Duration.ofMinutes(100)); - assertEquals(100, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); - - rounding = createRounding(Duration.ofSeconds(1)); - assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.SECOND_OF_MINUTE), 0d); - - rounding = createRounding(Duration.ofSeconds(120)); - assertEquals(120, rounding.roundingSize(Rounding.DateTimeUnit.SECOND_OF_MINUTE), 0d); - - rounding = createRounding(Duration.ofSeconds(60).plusMinutes(5).plusHours(1)); - assertEquals(1 + 5 + 60, rounding.roundingSize(Rounding.DateTimeUnit.MINUTES_OF_HOUR), 0d); - } - - public void testCreateRoundingPeriod() { - Rounding.Prepared rounding; - - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofMonths(0))); - assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); - - e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofYears(-10))); - assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); - - e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.of(0, 1, 1))); - assertThat(e.getMessage(), containsString("Time interval with multiple periods is not supported")); - - rounding = createRounding(Period.ofDays(1)); - assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.DAY_OF_MONTH), 0d); - - rounding = createRounding(Period.ofDays(4)); - assertEquals(4, rounding.roundingSize(Rounding.DateTimeUnit.DAY_OF_MONTH), 0d); - - rounding = createRounding(Period.ofDays(7)); - assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.WEEK_OF_WEEKYEAR), 0d); - - rounding = createRounding(Period.ofMonths(1)); - assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.MONTH_OF_YEAR), 0d); - - rounding = createRounding(Period.ofMonths(3)); - assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.QUARTER_OF_YEAR), 0d); - - rounding = createRounding(Period.ofYears(1)); - assertEquals(1, rounding.roundingSize(Rounding.DateTimeUnit.YEAR_OF_CENTURY), 0d); - - e = expectThrows(IllegalArgumentException.class, () -> createRounding(Period.ofYears(3))); - assertThat(e.getMessage(), containsString("Time interval is not supported")); - } - - public void testCreateRoundingNullInterval() { - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> createRounding(null)); - assertThat(e.getMessage(), containsString("Time interval is not supported")); - } - - public void testDateTruncFunction() { - long ts = toMillis("2023-02-17T10:25:33.38Z"); - - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> process(ts, createRounding(Period.ofDays(-1)))); - assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); - - e = expectThrows(IllegalArgumentException.class, () -> process(ts, createRounding(Duration.ofHours(-1)))); - assertThat(e.getMessage(), containsString("Zero or negative time interval is not supported")); - } - private static TestCaseSupplier ofDatePeriod(Period period, long value, String expectedDate) { return new TestCaseSupplier( List.of(DataType.DATE_PERIOD, DataType.DATETIME), () -> new TestCaseSupplier.TestCase( List.of( - new TestCaseSupplier.TypedData(period, DataType.DATE_PERIOD, "interval"), + new TestCaseSupplier.TypedData(period, DataType.DATE_PERIOD, "interval").forceLiteral(), new TestCaseSupplier.TypedData(value, DataType.DATETIME, "date") ), - "DateTruncEvaluator[date=Attribute[channel=1], interval=Attribute[channel=0]]", + Matchers.startsWith("DateTruncEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding["), DataType.DATETIME, equalTo(toMillis(expectedDate)) ) @@ -161,10 +81,10 @@ private static TestCaseSupplier ofDuration(Duration duration, long value, String List.of(DataType.TIME_DURATION, DataType.DATETIME), () -> new TestCaseSupplier.TestCase( List.of( - new TestCaseSupplier.TypedData(duration, DataType.TIME_DURATION, "interval"), + new TestCaseSupplier.TypedData(duration, DataType.TIME_DURATION, "interval").forceLiteral(), new TestCaseSupplier.TypedData(value, DataType.DATETIME, "date") ), - "DateTruncEvaluator[date=Attribute[channel=1], interval=Attribute[channel=0]]", + Matchers.startsWith("DateTruncEvaluator[fieldVal=Attribute[channel=0], rounding=Rounding["), DataType.DATETIME, equalTo(toMillis(expectedDate)) ) From 9ad09b6ee0d99f69f239ab2be7423cbd3da0d6f8 Mon Sep 17 00:00:00 2001 From: Giorgos Bamparopoulos Date: Mon, 4 Nov 2024 17:06:16 +0200 Subject: [PATCH 04/35] Fix a typo in the example for using pre-existing pipeline definitions (#116084) --- docs/reference/ingest/apis/simulate-ingest.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/ingest/apis/simulate-ingest.asciidoc b/docs/reference/ingest/apis/simulate-ingest.asciidoc index da591eed7546f..52ed09b1d32c2 100644 --- a/docs/reference/ingest/apis/simulate-ingest.asciidoc +++ b/docs/reference/ingest/apis/simulate-ingest.asciidoc @@ -265,8 +265,8 @@ Definition of a mapping that will be merged into the index's mapping for validat [[simulate-ingest-api-pre-existing-pipelines-ex]] ===== Use pre-existing pipeline definitions -In this example the index `index` has a default pipeline called `my-pipeline` and a final -pipeline called `my-final-pipeline`. Since both documents are being ingested into `index`, +In this example the index `my-index` has a default pipeline called `my-pipeline` and a final +pipeline called `my-final-pipeline`. Since both documents are being ingested into `my-index`, both pipelines are executed using the pipeline definitions that are already in the system. [source,console] From 544750c2655d1ef72e9044642a6164727e5d0f0e Mon Sep 17 00:00:00 2001 From: Dan Rubinstein Date: Mon, 4 Nov 2024 10:25:37 -0500 Subject: [PATCH 05/35] Fixing ScheduledEventTest generating same start and end time (#115877) Co-authored-by: Elastic Machine --- muted-tests.yml | 3 --- .../xpack/core/ml/calendars/ScheduledEventTests.java | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 3a471ad5d12d2..71bbcddd6cc9a 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -175,9 +175,6 @@ tests: - class: org.elasticsearch.xpack.spatial.search.GeoGridAggAndQueryConsistencyIT method: testGeoShapeGeoHex issue: https://github.com/elastic/elasticsearch/issues/115705 -- class: org.elasticsearch.xpack.core.ml.calendars.ScheduledEventTests - method: testBuild_SucceedsWithDefaultSkipResultAndSkipModelUpdatesValues - issue: https://github.com/elastic/elasticsearch/issues/115476 - class: org.elasticsearch.xpack.test.rest.XPackRestIT method: test {p0=transform/transforms_start_stop/Verify start transform reuses destination index} issue: https://github.com/elastic/elasticsearch/issues/115808 diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/calendars/ScheduledEventTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/calendars/ScheduledEventTests.java index 891430057513e..21fadee4e78a8 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/calendars/ScheduledEventTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/calendars/ScheduledEventTests.java @@ -207,7 +207,7 @@ private void validateScheduledEventSuccessfulBuild( String description = randomAlphaOfLength(10); String calendarId = randomAlphaOfLength(10); Instant startTime = Instant.ofEpochMilli(Instant.now().toEpochMilli()); - Instant endTime = startTime.plusSeconds(randomInt(3600)); + Instant endTime = startTime.plusSeconds(randomIntBetween(1, 3600)); ScheduledEvent.Builder builder = new ScheduledEvent.Builder().description(description) .calendarId(calendarId) From 589738c355590a4eb11dc99ec986291e4d2606c6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 4 Nov 2024 16:29:07 +0100 Subject: [PATCH 06/35] Fix incorrect mutex used in QueryPhaseResultConsumer (#116171) Everything synchronizes on PendingMerges, this was just a typo. closes #115716 --- muted-tests.yml | 3 --- .../elasticsearch/action/search/QueryPhaseResultConsumer.java | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 71bbcddd6cc9a..8498032d07b2c 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -223,9 +223,6 @@ tests: - class: org.elasticsearch.search.functionscore.QueryRescorerIT method: testScoring issue: https://github.com/elastic/elasticsearch/issues/116050 -- class: org.elasticsearch.indexing.IndexActionIT - method: testAutoGenerateIdNoDuplicates - issue: https://github.com/elastic/elasticsearch/issues/115716 - class: org.elasticsearch.xpack.application.connector.ConnectorIndexServiceTests issue: https://github.com/elastic/elasticsearch/issues/116087 - class: org.elasticsearch.compute.operator.FilterOperatorTests diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 6c654d9235ec2..249f49c951fcd 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -480,7 +480,7 @@ protected void doRun() { onMergeFailure(t); return; } - synchronized (QueryPhaseResultConsumer.this) { + synchronized (QueryPhaseResultConsumer.PendingMerges.this) { if (hasFailure()) { return; } @@ -501,7 +501,7 @@ protected void doRun() { } } Runnable r = mergeTask.consumeListener(); - synchronized (QueryPhaseResultConsumer.this) { + synchronized (QueryPhaseResultConsumer.PendingMerges.this) { while (true) { mergeTask = queue.poll(); runningTask.set(mergeTask); From 2a4a12b064be12207aa5dbbf1fa316391f3c06b6 Mon Sep 17 00:00:00 2001 From: Max Hniebergall <137079448+maxhniebergall@users.noreply.github.com> Date: Mon, 4 Nov 2024 10:31:14 -0500 Subject: [PATCH 07/35] Prevent randomized end time from being zero-offset (#116118) From 0d11e88d43f862f4bad3a27e26b34af533c5bdba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lorenzo=20Dematt=C3=A9?= Date: Mon, 4 Nov 2024 16:40:54 +0100 Subject: [PATCH 08/35] Entitlement tools: SecurityManager scanner (#116020) --- libs/entitlement/tools/build.gradle | 0 libs/entitlement/tools/common/build.gradle | 15 + .../entitlement/tools/Utils.java | 45 +++ .../securitymanager-scanner/build.gradle | 61 ++++ .../licenses/asm-LICENSE.txt | 26 ++ .../licenses/asm-NOTICE.txt | 1 + .../securitymanager-scanner/src/README.md | 47 +++ .../tools/securitymanager/scanner/Main.java | 103 +++++++ .../scanner/SecurityCheckClassVisitor.java | 279 ++++++++++++++++++ 9 files changed, 577 insertions(+) create mode 100644 libs/entitlement/tools/build.gradle create mode 100644 libs/entitlement/tools/common/build.gradle create mode 100644 libs/entitlement/tools/common/src/main/java/org/elasticsearch/entitlement/tools/Utils.java create mode 100644 libs/entitlement/tools/securitymanager-scanner/build.gradle create mode 100644 libs/entitlement/tools/securitymanager-scanner/licenses/asm-LICENSE.txt create mode 100644 libs/entitlement/tools/securitymanager-scanner/licenses/asm-NOTICE.txt create mode 100644 libs/entitlement/tools/securitymanager-scanner/src/README.md create mode 100644 libs/entitlement/tools/securitymanager-scanner/src/main/java/org/elasticsearch/entitlement/tools/securitymanager/scanner/Main.java create mode 100644 libs/entitlement/tools/securitymanager-scanner/src/main/java/org/elasticsearch/entitlement/tools/securitymanager/scanner/SecurityCheckClassVisitor.java diff --git a/libs/entitlement/tools/build.gradle b/libs/entitlement/tools/build.gradle new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/libs/entitlement/tools/common/build.gradle b/libs/entitlement/tools/common/build.gradle new file mode 100644 index 0000000000000..3373a8f747430 --- /dev/null +++ b/libs/entitlement/tools/common/build.gradle @@ -0,0 +1,15 @@ +/* + * 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". + */ + +plugins { + id 'java' +} + +group = 'org.elasticsearch.entitlement.tools' + diff --git a/libs/entitlement/tools/common/src/main/java/org/elasticsearch/entitlement/tools/Utils.java b/libs/entitlement/tools/common/src/main/java/org/elasticsearch/entitlement/tools/Utils.java new file mode 100644 index 0000000000000..c72e550a529cd --- /dev/null +++ b/libs/entitlement/tools/common/src/main/java/org/elasticsearch/entitlement/tools/Utils.java @@ -0,0 +1,45 @@ +/* + * 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.entitlement.tools; + +import java.io.IOException; +import java.lang.module.ModuleDescriptor; +import java.nio.file.FileSystem; +import java.nio.file.Files; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class Utils { + + public static Map> findModuleExports(FileSystem fs) throws IOException { + var modulesExports = new HashMap>(); + try (var stream = Files.walk(fs.getPath("modules"))) { + stream.filter(p -> p.getFileName().toString().equals("module-info.class")).forEach(x -> { + try (var is = Files.newInputStream(x)) { + var md = ModuleDescriptor.read(is); + modulesExports.put( + md.name(), + md.exports() + .stream() + .filter(e -> e.isQualified() == false) + .map(ModuleDescriptor.Exports::source) + .collect(Collectors.toSet()) + ); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + return modulesExports; + } + +} diff --git a/libs/entitlement/tools/securitymanager-scanner/build.gradle b/libs/entitlement/tools/securitymanager-scanner/build.gradle new file mode 100644 index 0000000000000..8d035c9e847c6 --- /dev/null +++ b/libs/entitlement/tools/securitymanager-scanner/build.gradle @@ -0,0 +1,61 @@ +plugins { + id 'application' +} + +apply plugin: 'elasticsearch.build' +apply plugin: 'elasticsearch.publish' + +tasks.named("dependencyLicenses").configure { + mapping from: /asm-.*/, to: 'asm' +} + +group = 'org.elasticsearch.entitlement.tools' + +ext { + javaMainClass = "org.elasticsearch.entitlement.tools.securitymanager.scanner.Main" +} + +application { + mainClass.set(javaMainClass) + applicationDefaultJvmArgs = [ + '--add-exports', 'java.base/sun.security.util=ALL-UNNAMED', + '--add-opens', 'java.base/java.lang=ALL-UNNAMED', + '--add-opens', 'java.base/java.net=ALL-UNNAMED', + '--add-opens', 'java.base/java.net.spi=ALL-UNNAMED', + '--add-opens', 'java.base/java.util.concurrent=ALL-UNNAMED', + '--add-opens', 'java.base/javax.crypto=ALL-UNNAMED', + '--add-opens', 'java.base/javax.security.auth=ALL-UNNAMED', + '--add-opens', 'java.base/jdk.internal.logger=ALL-UNNAMED', + '--add-opens', 'java.base/sun.nio.ch=ALL-UNNAMED', + '--add-opens', 'jdk.management.jfr/jdk.management.jfr=ALL-UNNAMED', + '--add-opens', 'java.logging/java.util.logging=ALL-UNNAMED', + '--add-opens', 'java.logging/sun.util.logging.internal=ALL-UNNAMED', + '--add-opens', 'java.naming/javax.naming.ldap.spi=ALL-UNNAMED', + '--add-opens', 'java.rmi/sun.rmi.runtime=ALL-UNNAMED', + '--add-opens', 'jdk.dynalink/jdk.dynalink=ALL-UNNAMED', + '--add-opens', 'jdk.dynalink/jdk.dynalink.linker=ALL-UNNAMED', + '--add-opens', 'java.desktop/sun.awt=ALL-UNNAMED', + '--add-opens', 'java.sql.rowset/javax.sql.rowset.spi=ALL-UNNAMED', + '--add-opens', 'java.sql/java.sql=ALL-UNNAMED', + '--add-opens', 'java.xml.crypto/com.sun.org.apache.xml.internal.security.utils=ALL-UNNAMED' + ] +} + +repositories { + mavenCentral() +} + +dependencies { + compileOnly(project(':libs:core')) + implementation 'org.ow2.asm:asm:9.7' + implementation 'org.ow2.asm:asm-util:9.7' + implementation(project(':libs:entitlement:tools:common')) +} + +tasks.named('forbiddenApisMain').configure { + replaceSignatureFiles 'jdk-signatures' +} + +tasks.named("thirdPartyAudit").configure { + ignoreMissingClasses() +} diff --git a/libs/entitlement/tools/securitymanager-scanner/licenses/asm-LICENSE.txt b/libs/entitlement/tools/securitymanager-scanner/licenses/asm-LICENSE.txt new file mode 100644 index 0000000000000..afb064f2f2666 --- /dev/null +++ b/libs/entitlement/tools/securitymanager-scanner/licenses/asm-LICENSE.txt @@ -0,0 +1,26 @@ +Copyright (c) 2012 France Télécom +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: +1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. +2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. +3. Neither the name of the copyright holders nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR +CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF +SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS +INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN +CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) +ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF +THE POSSIBILITY OF SUCH DAMAGE. diff --git a/libs/entitlement/tools/securitymanager-scanner/licenses/asm-NOTICE.txt b/libs/entitlement/tools/securitymanager-scanner/licenses/asm-NOTICE.txt new file mode 100644 index 0000000000000..8d1c8b69c3fce --- /dev/null +++ b/libs/entitlement/tools/securitymanager-scanner/licenses/asm-NOTICE.txt @@ -0,0 +1 @@ + diff --git a/libs/entitlement/tools/securitymanager-scanner/src/README.md b/libs/entitlement/tools/securitymanager-scanner/src/README.md new file mode 100644 index 0000000000000..c01ba1387d1c8 --- /dev/null +++ b/libs/entitlement/tools/securitymanager-scanner/src/README.md @@ -0,0 +1,47 @@ +This tool scans the JDK on which it is running, looking for any location where `SecurityManager` is currently used, thus giving us a list of "entry points" inside the JDK where security checks are currently happening. + +More in detail, the tool scans for calls to any `SecurityManager` method starting with `check` (e.g. `checkWrite`). The tool treats the generic `checkPermission` method a little bit differently: `checkPermission` accepts a generic `Permission` object, it tries to read the permission type and permission name to give more information about it, trying to match two patterns that are used frequently inside the JDK: + +Pattern 1: private static permission field + +```java +private static final RuntimePermission INET_ADDRESS_RESOLVER_PERMISSION = +new RuntimePermission("inetAddressResolverProvider"); +... +sm.checkPermission(INET_ADDRESS_RESOLVER_PERMISSION); +``` +Pattern 2: direct object creation + +```java +sm.checkPermission(new LinkPermission("symbolic")); +``` + +The tool will recognize this pattern, and report the permission type and name alongside the `checkPermission` entry point (type `RuntimePermission` and name `inetAddressResolverProvider` in the first case, type `LinkPermission` and name `symbolic` in the second). + +This allows to give more information (either a specific type like `LinkPermission`, or a specific name like `inetAddressResolverProvider`) to generic `checkPermission` to help in deciding how to classify the permission check. The 2 patterns work quite well and cover roughly 90% of the cases. + +In order to run the tool, use: +```shell +./gradlew :libs:entitlement:tools:securitymanager-scanner:run +``` +The output of the tool is a CSV file, with one line for each entry-point, columns separated by `TAB` + +The columns are: +1. Module name +2. File name (from source root) +3. Line number +4. Fully qualified class name (ASM style, with `/` separators) +5. Method name +6. Method descriptor (ASM signature) +6. Visibility (PUBLIC/PUBLIC-METHOD/PRIVATE) +7. Check detail 1 (method name, or in case of checkPermission, permission name. Might be `MISSING`) +8. Check detail 2 (in case of checkPermission, the argument type (`Permission` subtype). Might be `MISSING`) + +Examples: +``` +java.base sun/nio/ch/DatagramChannelImpl.java 1360 sun/nio/ch/DatagramChannelImpl connect (Ljava/net/SocketAddress;Z)Ljava/nio/channels/DatagramChannel; PRIVATE checkConnect +``` +or +``` +java.base java/net/ResponseCache.java 118 java/net/ResponseCache setDefault (Ljava/net/ResponseCache;)V PUBLIC setResponseCache java/net/NetPermission +``` diff --git a/libs/entitlement/tools/securitymanager-scanner/src/main/java/org/elasticsearch/entitlement/tools/securitymanager/scanner/Main.java b/libs/entitlement/tools/securitymanager-scanner/src/main/java/org/elasticsearch/entitlement/tools/securitymanager/scanner/Main.java new file mode 100644 index 0000000000000..bea49e0296e67 --- /dev/null +++ b/libs/entitlement/tools/securitymanager-scanner/src/main/java/org/elasticsearch/entitlement/tools/securitymanager/scanner/Main.java @@ -0,0 +1,103 @@ +/* + * 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.entitlement.tools.securitymanager.scanner; + +import org.elasticsearch.core.SuppressForbidden; +import org.elasticsearch.entitlement.tools.Utils; +import org.objectweb.asm.ClassReader; + +import java.io.IOException; +import java.net.URI; +import java.nio.file.FileSystem; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.util.HashMap; +import java.util.List; +import java.util.Set; + +public class Main { + + static final Set excludedModules = Set.of("java.desktop"); + + private static void identifySMChecksEntryPoints() throws IOException { + + FileSystem fs = FileSystems.getFileSystem(URI.create("jrt:/")); + + var moduleExports = Utils.findModuleExports(fs); + + var callers = new HashMap>(); + var visitor = new SecurityCheckClassVisitor(callers); + + try (var stream = Files.walk(fs.getPath("modules"))) { + stream.filter(x -> x.toString().endsWith(".class")).forEach(x -> { + var moduleName = x.subpath(1, 2).toString(); + if (excludedModules.contains(moduleName) == false) { + try { + ClassReader cr = new ClassReader(Files.newInputStream(x)); + visitor.setCurrentModule(moduleName, moduleExports.get(moduleName)); + var path = x.getNameCount() > 3 ? x.subpath(2, x.getNameCount() - 1).toString() : ""; + visitor.setCurrentSourcePath(path); + cr.accept(visitor, 0); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }); + } + + printToStdout(callers); + } + + @SuppressForbidden(reason = "This simple tool just prints to System.out") + private static void printToStdout(HashMap> callers) { + for (var kv : callers.entrySet()) { + for (var e : kv.getValue()) { + System.out.println(toString(kv.getKey(), e)); + } + } + } + + private static final String SEPARATOR = "\t"; + + private static String toString(String calleeName, SecurityCheckClassVisitor.CallerInfo callerInfo) { + var s = callerInfo.moduleName() + SEPARATOR + callerInfo.source() + SEPARATOR + callerInfo.line() + SEPARATOR + callerInfo + .className() + SEPARATOR + callerInfo.methodName() + SEPARATOR + callerInfo.methodDescriptor() + SEPARATOR; + + if (callerInfo.externalAccess().contains(SecurityCheckClassVisitor.ExternalAccess.METHOD) + && callerInfo.externalAccess().contains(SecurityCheckClassVisitor.ExternalAccess.CLASS)) { + s += "PUBLIC"; + } else if (callerInfo.externalAccess().contains(SecurityCheckClassVisitor.ExternalAccess.METHOD)) { + s += "PUBLIC-METHOD"; + } else { + s += "PRIVATE"; + } + + if (callerInfo.runtimePermissionType() != null) { + s += SEPARATOR + callerInfo.runtimePermissionType(); + } else if (calleeName.equals("checkPermission")) { + s += SEPARATOR + "MISSING"; // missing information + } else { + s += SEPARATOR + calleeName; + } + + if (callerInfo.permissionType() != null) { + s += SEPARATOR + callerInfo.permissionType(); + } else if (calleeName.equals("checkPermission")) { + s += SEPARATOR + "MISSING"; // missing information + } else { + s += SEPARATOR; + } + return s; + } + + public static void main(String[] args) throws IOException { + identifySMChecksEntryPoints(); + } +} diff --git a/libs/entitlement/tools/securitymanager-scanner/src/main/java/org/elasticsearch/entitlement/tools/securitymanager/scanner/SecurityCheckClassVisitor.java b/libs/entitlement/tools/securitymanager-scanner/src/main/java/org/elasticsearch/entitlement/tools/securitymanager/scanner/SecurityCheckClassVisitor.java new file mode 100644 index 0000000000000..a75fd5fc685f1 --- /dev/null +++ b/libs/entitlement/tools/securitymanager-scanner/src/main/java/org/elasticsearch/entitlement/tools/securitymanager/scanner/SecurityCheckClassVisitor.java @@ -0,0 +1,279 @@ +/* + * 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.entitlement.tools.securitymanager.scanner; + +import org.elasticsearch.core.SuppressForbidden; +import org.objectweb.asm.ClassVisitor; +import org.objectweb.asm.Label; +import org.objectweb.asm.MethodVisitor; +import org.objectweb.asm.Type; + +import java.lang.constant.ClassDesc; +import java.lang.reflect.InaccessibleObjectException; +import java.lang.reflect.Modifier; +import java.nio.file.Path; +import java.security.Permission; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.objectweb.asm.Opcodes.ACC_PUBLIC; +import static org.objectweb.asm.Opcodes.ASM9; +import static org.objectweb.asm.Opcodes.GETSTATIC; +import static org.objectweb.asm.Opcodes.INVOKEDYNAMIC; +import static org.objectweb.asm.Opcodes.INVOKEINTERFACE; +import static org.objectweb.asm.Opcodes.INVOKESPECIAL; +import static org.objectweb.asm.Opcodes.INVOKESTATIC; +import static org.objectweb.asm.Opcodes.INVOKEVIRTUAL; +import static org.objectweb.asm.Opcodes.NEW; + +class SecurityCheckClassVisitor extends ClassVisitor { + + static final String SECURITY_MANAGER_INTERNAL_NAME = "java/lang/SecurityManager"; + static final Set excludedClasses = Set.of(SECURITY_MANAGER_INTERNAL_NAME); + + enum ExternalAccess { + CLASS, + METHOD + } + + record CallerInfo( + String moduleName, + String source, + int line, + String className, + String methodName, + String methodDescriptor, + EnumSet externalAccess, + String permissionType, + String runtimePermissionType + ) {} + + private final Map> callerInfoByMethod; + private String className; + private int classAccess; + private String source; + private String moduleName; + private String sourcePath; + private Set moduleExports; + + protected SecurityCheckClassVisitor(Map> callerInfoByMethod) { + super(ASM9); + this.callerInfoByMethod = callerInfoByMethod; + } + + @Override + public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) { + super.visit(version, access, name, signature, superName, interfaces); + this.className = name; + this.classAccess = access; + } + + @Override + public void visitSource(String source, String debug) { + super.visitSource(source, debug); + this.source = source; + } + + @Override + public MethodVisitor visitMethod(int access, String name, String descriptor, String signature, String[] exceptions) { + if (excludedClasses.contains(this.className)) { + return super.visitMethod(access, name, descriptor, signature, exceptions); + } + return new SecurityCheckMethodVisitor(super.visitMethod(access, name, descriptor, signature, exceptions), name, access, descriptor); + } + + public void setCurrentModule(String moduleName, Set moduleExports) { + this.moduleName = moduleName; + this.moduleExports = moduleExports; + } + + public void setCurrentSourcePath(String path) { + this.sourcePath = path; + } + + private class SecurityCheckMethodVisitor extends MethodVisitor { + + private final String methodName; + private final String methodDescriptor; + private int line; + private String permissionType; + private String runtimePermissionType; + private final int methodAccess; + + protected SecurityCheckMethodVisitor(MethodVisitor mv, String methodName, int methodAccess, String methodDescriptor) { + super(ASM9, mv); + this.methodName = methodName; + this.methodAccess = methodAccess; + this.methodDescriptor = methodDescriptor; + } + + private static final Set KNOWN_PERMISSIONS = Set.of("jdk.vm.ci.services.JVMCIPermission"); + + @SuppressForbidden(reason = "System.err is OK for this simple command-line tool") + private void handleException(String className, Throwable e) { + System.err.println("Cannot process " + className + ": " + e.getMessage()); + } + + @Override + public void visitTypeInsn(int opcode, String type) { + super.visitTypeInsn(opcode, type); + if (opcode == NEW) { + if (type.endsWith("Permission")) { + var objectType = Type.getObjectType(type); + if (KNOWN_PERMISSIONS.contains(objectType.getClassName())) { + permissionType = type; + } else { + try { + var clazz = Class.forName(objectType.getClassName()); + if (Permission.class.isAssignableFrom(clazz)) { + permissionType = type; + } + } catch (ClassNotFoundException e) { + handleException(objectType.getClassName(), e); + } + } + } + } + } + + @Override + @SuppressForbidden(reason = "We need to violate java's access system to access private parts") + public void visitFieldInsn(int opcode, String owner, String name, String descriptor) { + super.visitFieldInsn(opcode, owner, name, descriptor); + if (opcode == GETSTATIC && descriptor.endsWith("Permission;")) { + var permissionType = Type.getType(descriptor); + if (permissionType.getSort() == Type.ARRAY) { + permissionType = permissionType.getElementType(); + } + try { + var clazz = Class.forName(permissionType.getClassName()); + if (Permission.class.isAssignableFrom(clazz)) { + this.permissionType = permissionType.getInternalName(); + } + } catch (ClassNotFoundException e) { + handleException(permissionType.getClassName(), e); + } + + var objectType = Type.getObjectType(owner); + try { + var clazz = Class.forName(objectType.getClassName()); + Arrays.stream(clazz.getDeclaredFields()) + .filter(f -> Modifier.isStatic(f.getModifiers()) && Modifier.isFinal(f.getModifiers())) + .filter(f -> f.getName().equals(name)) + .findFirst() + .ifPresent(x -> { + if (Permission.class.isAssignableFrom(x.getType())) { + try { + x.setAccessible(true); + var p = (Permission) (x.get(null)); + this.runtimePermissionType = p.getName(); + } catch (IllegalAccessException | InaccessibleObjectException e) { + handleException(x.getName(), e); + } + } + }); + + } catch (ClassNotFoundException | NoClassDefFoundError | UnsatisfiedLinkError e) { + handleException(objectType.getClassName(), e); + } + } + } + + @Override + public void visitLdcInsn(Object value) { + super.visitLdcInsn(value); + if (permissionType != null && permissionType.equals("java/lang/RuntimePermission")) { + this.runtimePermissionType = value.toString(); + } + } + + @Override + public void visitMethodInsn(int opcode, String owner, String name, String descriptor, boolean isInterface) { + super.visitMethodInsn(opcode, owner, name, descriptor, isInterface); + if (opcode == INVOKEVIRTUAL + || opcode == INVOKESPECIAL + || opcode == INVOKESTATIC + || opcode == INVOKEINTERFACE + || opcode == INVOKEDYNAMIC) { + + if (SECURITY_MANAGER_INTERNAL_NAME.equals(owner)) { + EnumSet externalAccesses = EnumSet.noneOf(ExternalAccess.class); + if (moduleExports.contains(getPackageName(className))) { + if ((methodAccess & ACC_PUBLIC) != 0) { + externalAccesses.add(ExternalAccess.METHOD); + } + if ((classAccess & ACC_PUBLIC) != 0) { + externalAccesses.add(ExternalAccess.CLASS); + } + } + + if (name.equals("checkPermission")) { + var callers = callerInfoByMethod.computeIfAbsent(name, ignored -> new ArrayList<>()); + callers.add( + new CallerInfo( + moduleName, + Path.of(sourcePath, source).toString(), + line, + className, + methodName, + methodDescriptor, + externalAccesses, + permissionType, + runtimePermissionType + ) + ); + this.permissionType = null; + this.runtimePermissionType = null; + } else if (name.startsWith("check")) { + // Non-generic methods (named methods that which already tell us the permission type) + var callers = callerInfoByMethod.computeIfAbsent(name, ignored -> new ArrayList<>()); + callers.add( + new CallerInfo( + moduleName, + Path.of(sourcePath, source).toString(), + line, + className, + methodName, + methodDescriptor, + externalAccesses, + null, + null + ) + ); + } + } + } + } + + private String getPackageName(String className) { + return ClassDesc.ofInternalName(className).packageName(); + } + + @Override + public void visitParameter(String name, int access) { + if (name != null) super.visitParameter(name, access); + } + + @Override + public void visitLineNumber(int line, Label start) { + super.visitLineNumber(line, start); + this.line = line; + } + + @Override + public void visitEnd() { + super.visitEnd(); + } + } +} From a1daddc9e35e105e1040659127cb5673865d17ce Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 4 Nov 2024 16:07:54 +0000 Subject: [PATCH 09/35] Add note about incompleteness of CBs (#116176) The docs kinda imply that circuit breakers protect against OOMEs, at least that's how some customers seem to interpret them. This commit adds a note spelling out that this isn't the case. --- .../modules/indices/circuit_breaker.asciidoc | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/reference/modules/indices/circuit_breaker.asciidoc b/docs/reference/modules/indices/circuit_breaker.asciidoc index 452d4e99704ce..13d81821c4f33 100644 --- a/docs/reference/modules/indices/circuit_breaker.asciidoc +++ b/docs/reference/modules/indices/circuit_breaker.asciidoc @@ -2,7 +2,16 @@ === Circuit breaker settings [[circuit-breaker-description]] // tag::circuit-breaker-description-tag[] -{es} contains multiple circuit breakers used to prevent operations from causing an OutOfMemoryError. Each breaker specifies a limit for how much memory it can use. Additionally, there is a parent-level breaker that specifies the total amount of memory that can be used across all breakers. +{es} contains multiple circuit breakers used to prevent operations from using an excessive amount of memory. Each breaker tracks the memory +used by certain operations and specifies a limit for how much memory it may track. Additionally, there +is a parent-level breaker that specifies the total amount of memory that may be tracked across all breakers. + +When a circuit breaker reaches its limit, {es} will reject further operations. See <> for information about errors +raised by circuit breakers. + +Circuit breakers do not track all memory usage in {es} and therefore provide only incomplete protection against excessive memory usage. If +{es} uses too much memory then it may suffer from performance issues and nodes may even fail with an `OutOfMemoryError`. See +<> for help with troubleshooting high heap usage. Except where noted otherwise, these settings can be dynamically updated on a live cluster with the <> API. From 90892c73b17d051c9100078e5a90fce7bf63c517 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Tue, 5 Nov 2024 03:22:30 +1100 Subject: [PATCH 10/35] Mute org.elasticsearch.search.basic.SearchWithRandomDisconnectsIT testSearchWithRandomDisconnects #116175 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 8498032d07b2c..036daa0105c61 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -279,6 +279,9 @@ tests: - class: org.elasticsearch.xpack.test.rest.XPackRestIT method: test {p0=terms_enum/10_basic/Test security} issue: https://github.com/elastic/elasticsearch/issues/116178 +- class: org.elasticsearch.search.basic.SearchWithRandomDisconnectsIT + method: testSearchWithRandomDisconnects + issue: https://github.com/elastic/elasticsearch/issues/116175 # Examples: # From 599ab7a8ad5dd7773893ae414b7db83422d9e843 Mon Sep 17 00:00:00 2001 From: Pete Gillin Date: Mon, 4 Nov 2024 16:23:10 +0000 Subject: [PATCH 11/35] Remove ignored fallback option on GeoIP processor (#116112) This removes the option `fallback_to_default_databases` on the `geoip` ingest processor has been deprecated and ignored since 8.0.0. --- docs/changelog/116112.yaml | 13 +++++++++++++ .../elasticsearch/ingest/geoip/GeoIpProcessor.java | 8 -------- .../ingest/geoip/GeoIpProcessorFactoryTests.java | 9 --------- 3 files changed, 13 insertions(+), 17 deletions(-) create mode 100644 docs/changelog/116112.yaml diff --git a/docs/changelog/116112.yaml b/docs/changelog/116112.yaml new file mode 100644 index 0000000000000..9e15d691a77d3 --- /dev/null +++ b/docs/changelog/116112.yaml @@ -0,0 +1,13 @@ +pr: 116112 +summary: Remove ignored fallback option on GeoIP processor +area: Ingest Node +type: breaking +issues: [] +breaking: + title: Remove ignored fallback option on GeoIP processor + area: Ingest + details: >- + The option fallback_to_default_databases on the geoip ingest processor has been removed. + (It was deprecated and ignored since 8.0.0.) + impact: Customers should stop remove the noop fallback_to_default_databases option on any geoip ingest processors. + notable: false diff --git a/modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java b/modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java index f99f8dbe2fdd0..9e0392b2b7974 100644 --- a/modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java +++ b/modules/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java @@ -36,8 +36,6 @@ public final class GeoIpProcessor extends AbstractProcessor { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(GeoIpProcessor.class); - static final String DEFAULT_DATABASES_DEPRECATION_MESSAGE = "the [fallback_to_default_databases] has been deprecated, because " - + "Elasticsearch no longer includes the default Maxmind geoip databases. This setting will be removed in Elasticsearch 9.0"; static final String UNSUPPORTED_DATABASE_DEPRECATION_MESSAGE = "the geoip processor will no longer support database type [{}] " + "in a future version of Elasticsearch"; // TODO add a message about migration? @@ -241,12 +239,6 @@ public Processor create( // validate (and consume) the download_database_on_pipeline_creation property even though the result is not used by the factory readBooleanProperty(type, processorTag, config, "download_database_on_pipeline_creation", true); - // noop, should be removed in 9.0 - Object value = config.remove("fallback_to_default_databases"); - if (value != null) { - deprecationLogger.warn(DeprecationCategory.OTHER, "default_databases_message", DEFAULT_DATABASES_DEPRECATION_MESSAGE); - } - final String databaseType; try (IpDatabase ipDatabase = ipDatabaseProvider.getDatabase(databaseFile)) { if (ipDatabase == null) { diff --git a/modules/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java b/modules/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java index 5ac0c76054d33..34003b79fc18b 100644 --- a/modules/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java +++ b/modules/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java @@ -473,15 +473,6 @@ public void testLoadingCustomDatabase() throws IOException { threadPool.shutdown(); } - public void testFallbackUsingDefaultDatabases() throws Exception { - GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(GEOIP_TYPE, databaseNodeService); - Map config = new HashMap<>(); - config.put("field", "source_field"); - config.put("fallback_to_default_databases", randomBoolean()); - factory.create(null, null, null, config); - assertWarnings(GeoIpProcessor.DEFAULT_DATABASES_DEPRECATION_MESSAGE); - } - public void testDownloadDatabaseOnPipelineCreation() throws IOException { GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(GEOIP_TYPE, databaseNodeService); Map config = new HashMap<>(); From 9658940a518099f3e7f1b9e8d0f802a4be788094 Mon Sep 17 00:00:00 2001 From: Felix Barnsteiner Date: Mon, 4 Nov 2024 17:26:50 +0100 Subject: [PATCH 12/35] Ignore conflicting fields during dynamic mapping update (#114227) This fixes a bug when concurrently executing index requests that have different types for the same field. --- docs/changelog/114227.yaml | 6 ++++ .../index/mapper/DynamicMappingIT.java | 31 +++++++++++++++++ .../index/mapper/ObjectMapper.java | 26 ++++++++++++++ .../index/mapper/ObjectMapperMergeTests.java | 34 +++++++++++++++++++ 4 files changed, 97 insertions(+) create mode 100644 docs/changelog/114227.yaml diff --git a/docs/changelog/114227.yaml b/docs/changelog/114227.yaml new file mode 100644 index 0000000000000..9b508f07c9e5a --- /dev/null +++ b/docs/changelog/114227.yaml @@ -0,0 +1,6 @@ +pr: 114227 +summary: Ignore conflicting fields during dynamic mapping update +area: Mapping +type: bug +issues: + - 114228 diff --git a/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java b/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java index 9b9b23e71abed..f7bf775bc4f8b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/index/mapper/DynamicMappingIT.java @@ -63,6 +63,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.oneOf; public class DynamicMappingIT extends ESIntegTestCase { @@ -190,6 +192,35 @@ private Map indexConcurrently(int numberOfFieldsToCreate, Settin return properties; } + public void testConcurrentDynamicMappingsWithConflictingType() throws Throwable { + int numberOfDocsToCreate = 16; + indicesAdmin().prepareCreate("index").setSettings(Settings.builder()).get(); + ensureGreen("index"); + final AtomicReference error = new AtomicReference<>(); + startInParallel(numberOfDocsToCreate, i -> { + try { + assertEquals( + DocWriteResponse.Result.CREATED, + prepareIndex("index").setId(Integer.toString(i)).setSource("field" + i, 0, "field" + (i + 1), 0.1).get().getResult() + ); + } catch (Exception e) { + error.compareAndSet(null, e); + } + }); + if (error.get() != null) { + throw error.get(); + } + client().admin().indices().prepareRefresh("index").get(); + for (int i = 0; i < numberOfDocsToCreate; ++i) { + assertTrue(client().prepareGet("index", Integer.toString(i)).get().isExists()); + } + Map index = indicesAdmin().prepareGetMappings("index").get().getMappings().get("index").getSourceAsMap(); + for (int i = 0, j = 1; i < numberOfDocsToCreate; i++, j++) { + assertThat(new WriteField("properties.field" + i + ".type", () -> index).get(null), is(oneOf("long", "float"))); + assertThat(new WriteField("properties.field" + j + ".type", () -> index).get(null), is(oneOf("long", "float"))); + } + } + public void testPreflightCheckAvoidsMaster() throws InterruptedException, IOException { // can't use INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING nor INDEX_MAPPING_DEPTH_LIMIT_SETTING as a check here, as that is already // checked at parse time, see testTotalFieldsLimitForDynamicMappingsUpdateCheckedAtDocumentParseTime diff --git a/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java index 70c4a3ac213a2..023f6fcea0bfe 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/ObjectMapper.java @@ -9,6 +9,8 @@ package org.elasticsearch.index.mapper; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.index.LeafReader; import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchParseException; @@ -41,6 +43,7 @@ import java.util.stream.Stream; public class ObjectMapper extends Mapper { + private static final Logger logger = LogManager.getLogger(ObjectMapper.class); private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(ObjectMapper.class); public static final FeatureFlag SUB_OBJECTS_AUTO_FEATURE_FLAG = new FeatureFlag("sub_objects_auto"); @@ -679,6 +682,13 @@ private static Map buildMergedMappers( // replaces an existing one. if (objectMergeContext.getMapperBuilderContext().getMergeReason() == MergeReason.INDEX_TEMPLATE) { putMergedMapper(mergedMappers, mergeWithMapper); + } else if (isConflictingDynamicMapping(objectMergeContext, mergeWithMapper, mergeIntoMapper)) { + logger.trace( + "ignoring conflicting dynamic mapping update for field={} current_type={} new_type={}", + mergeIntoMapper.fullPath(), + mergeIntoMapper.typeName(), + mergeWithMapper.typeName() + ); } else { putMergedMapper(mergedMappers, mergeIntoMapper.merge(mergeWithMapper, objectMergeContext)); } @@ -687,6 +697,22 @@ private static Map buildMergedMappers( return Map.copyOf(mergedMappers); } + /* + * We're ignoring the field if a dynamic mapping update tries to define a conflicting field type. + * This is caused by another index request with a different value racing to update the mappings. + * After updating the mappings, the index request will be re-tried and sees the updated mappings for this field. + * The updated mappings will then be taken into account when parsing the document + * (for example by coercing the value, ignore_malformed values, or failing the index request due to a type conflict). + */ + private static boolean isConflictingDynamicMapping( + MapperMergeContext objectMergeContext, + Mapper mergeWithMapper, + Mapper mergeIntoMapper + ) { + return objectMergeContext.getMapperBuilderContext().getMergeReason().isAutoUpdate() + && mergeIntoMapper.typeName().equals(mergeWithMapper.typeName()) == false; + } + private static void putMergedMapper(Map mergedMappers, @Nullable Mapper merged) { if (merged != null) { mergedMappers.put(merged.leafName(), merged); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java index 3a68ad301ce5c..1f8a2a754428b 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/ObjectMapperMergeTests.java @@ -9,13 +9,19 @@ package org.elasticsearch.index.mapper; import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.script.ScriptCompiler; import org.elasticsearch.test.ESTestCase; import java.util.Collections; import java.util.Optional; import static org.elasticsearch.index.mapper.MapperService.MergeReason.INDEX_TEMPLATE; +import static org.elasticsearch.index.mapper.MapperService.MergeReason.MAPPING_AUTO_UPDATE; +import static org.elasticsearch.index.mapper.MapperService.MergeReason.MAPPING_AUTO_UPDATE_PREFLIGHT; import static org.elasticsearch.index.mapper.MapperService.MergeReason.MAPPING_UPDATE; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; public final class ObjectMapperMergeTests extends ESTestCase { @@ -318,6 +324,34 @@ public void testMergeSubobjectsFalseWithObject() { assertNotNull(parentMapper.getMapper("child.grandchild")); } + public void testConflictingDynamicUpdate() { + RootObjectMapper mergeInto = new RootObjectMapper.Builder("_doc", Optional.empty()).add( + new KeywordFieldMapper.Builder("http.status_code", IndexVersion.current()) + ).build(MapperBuilderContext.root(false, false)); + RootObjectMapper mergeWith = new RootObjectMapper.Builder("_doc", Optional.empty()).add( + new NumberFieldMapper.Builder( + "http.status_code", + NumberFieldMapper.NumberType.LONG, + ScriptCompiler.NONE, + false, + true, + IndexVersion.current(), + null + ) + ).build(MapperBuilderContext.root(false, false)); + + MapperService.MergeReason autoUpdateMergeReason = randomFrom(MAPPING_AUTO_UPDATE, MAPPING_AUTO_UPDATE_PREFLIGHT); + ObjectMapper merged = mergeInto.merge(mergeWith, MapperMergeContext.root(false, false, autoUpdateMergeReason, Long.MAX_VALUE)); + FieldMapper httpStatusCode = (FieldMapper) merged.getMapper("http.status_code"); + assertThat(httpStatusCode, is(instanceOf(KeywordFieldMapper.class))); + + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> mergeInto.merge(mergeWith, MapperMergeContext.root(false, false, MAPPING_UPDATE, Long.MAX_VALUE)) + ); + assertThat(e.getMessage(), equalTo("mapper [http.status_code] cannot be changed from type [keyword] to [long]")); + } + private static RootObjectMapper createRootSubobjectFalseLeafWithDots() { FieldMapper.Builder fieldBuilder = new KeywordFieldMapper.Builder("host.name", IndexVersion.current()); FieldMapper fieldMapper = fieldBuilder.build(MapperBuilderContext.root(false, false)); From de9851aea5c509fd586bf054a450c37e2b5befa3 Mon Sep 17 00:00:00 2001 From: Alexey Ivanov Date: Mon, 4 Nov 2024 16:28:57 +0000 Subject: [PATCH 13/35] Don't allow secure settings in YML config (109115) (#115779) * Don't allow secure settings in YML config (109115) Elasticsearch should refuse to start if a secure setting is defined in elasticsearch.yml, in order to protect users from accidentally putting their secrets in a place where they are unexpectedly visible Fixes #109115 --- docs/changelog/115779.yaml | 6 ++++ .../settings/AbstractScopedSettings.java | 9 +++++ .../common/settings/SecureSetting.java | 13 ++----- .../common/settings/ScopedSettingsTests.java | 36 +++++++++++++++++++ .../common/settings/SettingsTests.java | 7 ---- 5 files changed, 54 insertions(+), 17 deletions(-) create mode 100644 docs/changelog/115779.yaml diff --git a/docs/changelog/115779.yaml b/docs/changelog/115779.yaml new file mode 100644 index 0000000000000..326751db7750b --- /dev/null +++ b/docs/changelog/115779.yaml @@ -0,0 +1,6 @@ +pr: 115779 +summary: Don't allow secure settings in YML config (109115) +area: Infra/Settings +type: bug +issues: + - 109115 diff --git a/server/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java index 60626b9e2375f..c65f75df663d2 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/AbstractScopedSettings.java @@ -599,6 +599,15 @@ void validate(final String key, final Settings settings, final boolean validateV ); } } + + if (setting instanceof SecureSetting && settings.hasValue(key)) { + throw new IllegalArgumentException( + "Setting [" + + key + + "] is a secure setting" + + " and must be stored inside the Elasticsearch keystore, but was found inside elasticsearch.yml" + ); + } } if (validateValue) { setting.get(settings); diff --git a/server/src/main/java/org/elasticsearch/common/settings/SecureSetting.java b/server/src/main/java/org/elasticsearch/common/settings/SecureSetting.java index 67ac55f7b19eb..36ca2df08724d 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/SecureSetting.java +++ b/server/src/main/java/org/elasticsearch/common/settings/SecureSetting.java @@ -82,21 +82,14 @@ public boolean exists(Settings.Builder builder) { public T get(Settings settings) { checkDeprecation(settings); final SecureSettings secureSettings = settings.getSecureSettings(); - if (secureSettings == null || secureSettings.getSettingNames().contains(getKey()) == false) { - if (super.exists(settings)) { - throw new IllegalArgumentException( - "Setting [" - + getKey() - + "] is a secure setting" - + " and must be stored inside the Elasticsearch keystore, but was found inside elasticsearch.yml" - ); - } + String key = getKey(); + if (secureSettings == null || secureSettings.getSettingNames().contains(key) == false) { return getFallback(settings); } try { return getSecret(secureSettings); } catch (GeneralSecurityException e) { - throw new RuntimeException("failed to read secure setting " + getKey(), e); + throw new RuntimeException("failed to read secure setting " + key, e); } } diff --git a/server/src/test/java/org/elasticsearch/common/settings/ScopedSettingsTests.java b/server/src/test/java/org/elasticsearch/common/settings/ScopedSettingsTests.java index 8051437cf6e12..47026fe713c5c 100644 --- a/server/src/test/java/org/elasticsearch/common/settings/ScopedSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/common/settings/ScopedSettingsTests.java @@ -1138,6 +1138,42 @@ public void testDiffSecureSettings() { assertTrue(diffed.isEmpty()); } + public void testValidateSecureSettingInsecureOverride() { + MockSecureSettings secureSettings = new MockSecureSettings(); + String settingName = "something.secure"; + secureSettings.setString(settingName, "secure"); + Settings settings = Settings.builder().put(settingName, "notreallysecure").setSecureSettings(secureSettings).build(); + + ClusterSettings clusterSettings = new ClusterSettings( + settings, + Collections.singleton(SecureSetting.secureString(settingName, null)) + ); + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> clusterSettings.validate(settings, false)); + assertEquals( + e.getMessage(), + "Setting [something.secure] is a secure setting " + + "and must be stored inside the Elasticsearch keystore, but was found inside elasticsearch.yml" + ); + } + + public void testValidateSecureSettingInInsecureSettings() { + String settingName = "something.secure"; + Settings settings = Settings.builder().put(settingName, "notreallysecure").build(); + + ClusterSettings clusterSettings = new ClusterSettings( + settings, + Collections.singleton(SecureSetting.secureString(settingName, null)) + ); + + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> clusterSettings.validate(settings, false)); + assertEquals( + e.getMessage(), + "Setting [something.secure] is a secure setting " + + "and must be stored inside the Elasticsearch keystore, but was found inside elasticsearch.yml" + ); + } + public static IndexMetadata newIndexMeta(String name, Settings indexSettings) { return IndexMetadata.builder(name).settings(indexSettings(IndexVersion.current(), 1, 0).put(indexSettings)).build(); } diff --git a/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java b/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java index cfdc5e6befaaa..5fefd92d176a5 100644 --- a/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java +++ b/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java @@ -473,13 +473,6 @@ public void testDiff() throws IOException { } } - public void testSecureSettingConflict() { - Setting setting = SecureSetting.secureString("something.secure", null); - Settings settings = Settings.builder().put("something.secure", "notreallysecure").build(); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> setting.get(settings)); - assertTrue(e.getMessage().contains("must be stored inside the Elasticsearch keystore")); - } - public void testSecureSettingIllegalName() { IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> SecureSetting.secureString("*IllegalName", null)); assertTrue(e.getMessage().contains("does not match the allowed setting name pattern")); From 409fb8db7235be96c0cfdf8cf3514ddfdf0aa8ba Mon Sep 17 00:00:00 2001 From: Craig Taverner Date: Mon, 4 Nov 2024 17:58:59 +0100 Subject: [PATCH 14/35] The common type between any two string types is KEYWORD (#116107) * The common type between any two string types is KEYWORD The only time we return TEXT or SEMANTIC_TEXT is if both types are of that type. * Simplify --- .../xpack/esql/type/EsqlDataTypeConverter.java | 11 ++--------- .../xpack/esql/type/EsqlDataTypeConverterTests.java | 6 +----- 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java index 7fb998e82001e..c9c292769b570 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java @@ -74,8 +74,6 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; import static org.elasticsearch.xpack.esql.core.type.DataType.LONG; import static org.elasticsearch.xpack.esql.core.type.DataType.NULL; -import static org.elasticsearch.xpack.esql.core.type.DataType.SEMANTIC_TEXT; -import static org.elasticsearch.xpack.esql.core.type.DataType.TEXT; import static org.elasticsearch.xpack.esql.core.type.DataType.TIME_DURATION; import static org.elasticsearch.xpack.esql.core.type.DataType.UNSIGNED_LONG; import static org.elasticsearch.xpack.esql.core.type.DataType.VERSION; @@ -366,13 +364,8 @@ public static DataType commonType(DataType left, DataType right) { } } if (isString(left) && isString(right)) { - if (left == SEMANTIC_TEXT || right == SEMANTIC_TEXT) { - return KEYWORD; - } - if (left == TEXT || right == TEXT) { - return TEXT; - } - return right; + // Both TEXT and SEMANTIC_TEXT are processed as KEYWORD + return KEYWORD; } if (left.isNumeric() && right.isNumeric()) { int lsize = left.estimatedSize().orElseThrow(); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverterTests.java index b2228b5543ef2..b30f0870496e3 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverterTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverterTests.java @@ -36,10 +36,8 @@ import static org.elasticsearch.xpack.esql.core.type.DataType.OBJECT; import static org.elasticsearch.xpack.esql.core.type.DataType.PARTIAL_AGG; import static org.elasticsearch.xpack.esql.core.type.DataType.SCALED_FLOAT; -import static org.elasticsearch.xpack.esql.core.type.DataType.SEMANTIC_TEXT; import static org.elasticsearch.xpack.esql.core.type.DataType.SHORT; import static org.elasticsearch.xpack.esql.core.type.DataType.SOURCE; -import static org.elasticsearch.xpack.esql.core.type.DataType.TEXT; import static org.elasticsearch.xpack.esql.core.type.DataType.TSID_DATA_TYPE; import static org.elasticsearch.xpack.esql.core.type.DataType.UNSIGNED_LONG; import static org.elasticsearch.xpack.esql.core.type.DataType.UNSUPPORTED; @@ -71,10 +69,8 @@ public void testCommonTypeStrings() { } else if ((isString(dataType1) && isString(dataType2))) { if (dataType1 == dataType2) { assertEqualsCommonType(dataType1, dataType2, dataType1); - } else if (dataType1 == SEMANTIC_TEXT || dataType2 == SEMANTIC_TEXT) { - assertEqualsCommonType(dataType1, dataType2, KEYWORD); } else { - assertEqualsCommonType(dataType1, dataType2, TEXT); + assertEqualsCommonType(dataType1, dataType2, KEYWORD); } } else { assertNullCommonType(dataType1, dataType2); From 7d0d50df63568da4db2ac430853ad1a9fb72b2fa Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 4 Nov 2024 19:01:52 +0100 Subject: [PATCH 15/35] Remove dead code from o.e.index.query (#116148) Just a bunch of obvious dead code removal. --- .../query/AbstractGeometryQueryBuilder.java | 38 ------------ .../query/CombinedFieldsQueryBuilder.java | 12 ---- .../query/CoordinatorRewriteContext.java | 7 --- .../index/query/ExistsQueryBuilder.java | 2 +- .../index/query/FuzzyQueryBuilder.java | 62 ------------------- .../query/GeoBoundingBoxQueryBuilder.java | 21 ------- .../index/query/GeoDistanceQueryBuilder.java | 19 ------ .../index/query/GeoPolygonQueryBuilder.java | 14 ----- .../index/query/GeoShapeQueryBuilder.java | 9 +-- .../index/query/GeoValidationMethod.java | 11 ---- .../index/query/InnerHitBuilder.java | 1 - .../index/query/IntervalQueryBuilder.java | 4 -- .../query/MatchBoolPrefixQueryBuilder.java | 30 --------- .../query/MatchPhrasePrefixQueryBuilder.java | 5 -- .../index/query/MoreLikeThisQueryBuilder.java | 45 -------------- .../index/query/MultiMatchQueryBuilder.java | 2 +- .../index/query/QueryBuilders.java | 32 ---------- .../GaussDecayFunctionBuilder.java | 2 - .../RandomScoreFunctionBuilder.java | 8 --- 19 files changed, 4 insertions(+), 320 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/query/AbstractGeometryQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/AbstractGeometryQueryBuilder.java index 033151da362ef..b2a816c9e5690 100644 --- a/server/src/main/java/org/elasticsearch/index/query/AbstractGeometryQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/AbstractGeometryQueryBuilder.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.geometry.Geometry; @@ -44,10 +43,6 @@ * Base {@link QueryBuilder} that builds a Geometry Query */ public abstract class AbstractGeometryQueryBuilder> extends AbstractQueryBuilder { - static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Types are deprecated in [geo_shape] queries. " - + "The type should no longer be specified in the [indexed_shape] section."; - private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(AbstractGeometryQueryBuilder.class); - public static final String DEFAULT_SHAPE_INDEX_NAME = "shapes"; public static final String DEFAULT_SHAPE_FIELD_NAME = "shape"; public static final ShapeRelation DEFAULT_SHAPE_RELATION = ShapeRelation.INTERSECTS; @@ -59,7 +54,6 @@ public abstract class AbstractGeometryQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) { return this; } - /** - * Gets whether the query builder will ignore unmapped fields (and run a - * {@link MatchNoDocsQuery} in place of this query) or throw an exception if - * the field is unmapped. - */ - public boolean ignoreUnmapped() { - return ignoreUnmapped; - } - /** builds the appropriate lucene shape query */ protected abstract Query buildShapeQuery(SearchExecutionContext context, MappedFieldType fieldType); diff --git a/server/src/main/java/org/elasticsearch/index/query/CombinedFieldsQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/CombinedFieldsQueryBuilder.java index 1560004b13785..1ef233f952e77 100644 --- a/server/src/main/java/org/elasticsearch/index/query/CombinedFieldsQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/CombinedFieldsQueryBuilder.java @@ -231,23 +231,11 @@ public CombinedFieldsQueryBuilder zeroTermsQuery(ZeroTermsQueryOption zeroTermsQ return this; } - public ZeroTermsQueryOption zeroTermsQuery() { - return zeroTermsQuery; - } - public CombinedFieldsQueryBuilder autoGenerateSynonymsPhraseQuery(boolean enable) { this.autoGenerateSynonymsPhraseQuery = enable; return this; } - /** - * Whether phrase queries should be automatically generated for multi terms synonyms. - * Defaults to {@code true}. - */ - public boolean autoGenerateSynonymsPhraseQuery() { - return autoGenerateSynonymsPhraseQuery; - } - private static void validateFieldBoost(float boost) { if (boost < 1.0f) { throw new IllegalArgumentException("[" + NAME + "] requires field boosts to be >= 1.0"); diff --git a/server/src/main/java/org/elasticsearch/index/query/CoordinatorRewriteContext.java b/server/src/main/java/org/elasticsearch/index/query/CoordinatorRewriteContext.java index b0d3065ba3a3f..e054f17ef64d6 100644 --- a/server/src/main/java/org/elasticsearch/index/query/CoordinatorRewriteContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/CoordinatorRewriteContext.java @@ -154,11 +154,4 @@ public String getTierPreference() { return tier.isEmpty() == false ? tier : null; } - /** - * We're holding on to the index tier in the context as otherwise we'd need - * to re-parse it from the index settings when evaluating the _tier field. - */ - public String tier() { - return tier; - } } diff --git a/server/src/main/java/org/elasticsearch/index/query/ExistsQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/ExistsQueryBuilder.java index e5c1b16b65059..fb96f85835548 100644 --- a/server/src/main/java/org/elasticsearch/index/query/ExistsQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/ExistsQueryBuilder.java @@ -68,7 +68,7 @@ public String fieldName() { } @Override - protected QueryBuilder doIndexMetadataRewrite(QueryRewriteContext context) throws IOException { + protected QueryBuilder doIndexMetadataRewrite(QueryRewriteContext context) { if (getMappedFields(context, fieldName).isEmpty()) { return new MatchNoneQueryBuilder("The \"" + getName() + "\" query was rewritten to a \"match_none\" query."); } else { diff --git a/server/src/main/java/org/elasticsearch/index/query/FuzzyQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/FuzzyQueryBuilder.java index 16e21f1d5650e..ada82b7a37c91 100644 --- a/server/src/main/java/org/elasticsearch/index/query/FuzzyQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/FuzzyQueryBuilder.java @@ -78,56 +78,6 @@ public FuzzyQueryBuilder(String fieldName, String value) { this(fieldName, (Object) value); } - /** - * Constructs a new fuzzy query. - * - * @param fieldName The name of the field - * @param value The value of the text - */ - public FuzzyQueryBuilder(String fieldName, int value) { - this(fieldName, (Object) value); - } - - /** - * Constructs a new fuzzy query. - * - * @param fieldName The name of the field - * @param value The value of the text - */ - public FuzzyQueryBuilder(String fieldName, long value) { - this(fieldName, (Object) value); - } - - /** - * Constructs a new fuzzy query. - * - * @param fieldName The name of the field - * @param value The value of the text - */ - public FuzzyQueryBuilder(String fieldName, float value) { - this(fieldName, (Object) value); - } - - /** - * Constructs a new fuzzy query. - * - * @param fieldName The name of the field - * @param value The value of the text - */ - public FuzzyQueryBuilder(String fieldName, double value) { - this(fieldName, (Object) value); - } - - /** - * Constructs a new fuzzy query. - * - * @param fieldName The name of the field - * @param value The value of the text - */ - public FuzzyQueryBuilder(String fieldName, boolean value) { - this(fieldName, (Object) value); - } - /** * Constructs a new fuzzy query. * @@ -193,19 +143,11 @@ public FuzzyQueryBuilder prefixLength(int prefixLength) { return this; } - public int prefixLength() { - return this.prefixLength; - } - public FuzzyQueryBuilder maxExpansions(int maxExpansions) { this.maxExpansions = maxExpansions; return this; } - public int maxExpansions() { - return this.maxExpansions; - } - public FuzzyQueryBuilder transpositions(boolean transpositions) { this.transpositions = transpositions; return this; @@ -220,10 +162,6 @@ public FuzzyQueryBuilder rewrite(String rewrite) { return this; } - public String rewrite() { - return this.rewrite; - } - @Override protected void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(NAME); diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java index 7d773ce6c3fd2..e91be82730222 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.geo.SpatialStrategy; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.geometry.Rectangle; import org.elasticsearch.geometry.utils.Geohash; import org.elasticsearch.index.mapper.GeoShapeQueryable; @@ -45,10 +44,6 @@ public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder { public static final String NAME = "geo_bounding_box"; - private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(GeoBoundingBoxQueryBuilder.class); - - private static final String TYPE_PARAMETER_DEPRECATION_MESSAGE = "Deprecated parameter [type] used, it should no longer be specified."; - /** * The default value for ignore_unmapped. */ @@ -204,13 +199,6 @@ public GeoBoundingBoxQueryBuilder setValidationMethod(GeoValidationMethod method return this; } - /** - * Returns geo coordinate validation method to use. - * */ - public GeoValidationMethod getValidationMethod() { - return this.validationMethod; - } - /** Returns the name of the field to base the bounding box computation on. */ public String fieldName() { return this.fieldName; @@ -226,15 +214,6 @@ public GeoBoundingBoxQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) { return this; } - /** - * Gets whether the query builder will ignore unmapped fields (and run a - * {@link MatchNoDocsQuery} in place of this query) or throw an exception if - * the field is unmapped. - */ - public boolean ignoreUnmapped() { - return ignoreUnmapped; - } - QueryValidationException checkLatLon() { if (GeoValidationMethod.isIgnoreMalformed(validationMethod)) { return null; diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoDistanceQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoDistanceQueryBuilder.java index 479534321ba30..14271fb01696a 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoDistanceQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoDistanceQueryBuilder.java @@ -183,21 +183,11 @@ public GeoDistanceQueryBuilder geoDistance(GeoDistance geoDistance) { return this; } - /** Returns geo distance calculation type to use. */ - public GeoDistance geoDistance() { - return this.geoDistance; - } - /** Set validation method for geo coordinates. */ public void setValidationMethod(GeoValidationMethod method) { this.validationMethod = method; } - /** Returns validation method for geo coordinates. */ - public GeoValidationMethod getValidationMethod() { - return this.validationMethod; - } - /** * Sets whether the query builder should ignore unmapped fields (and run a * {@link MatchNoDocsQuery} in place of this query) or throw an exception if @@ -208,15 +198,6 @@ public GeoDistanceQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) { return this; } - /** - * Gets whether the query builder will ignore unmapped fields (and run a - * {@link MatchNoDocsQuery} in place of this query) or throw an exception if - * the field is unmapped. - */ - public boolean ignoreUnmapped() { - return ignoreUnmapped; - } - @Override protected Query doToQuery(SearchExecutionContext context) throws IOException { MappedFieldType fieldType = context.getFieldType(fieldName); diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java index d2459c72eebed..575c15d5c063e 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoPolygonQueryBuilder.java @@ -126,11 +126,6 @@ public GeoPolygonQueryBuilder setValidationMethod(GeoValidationMethod method) { return this; } - /** Returns the validation method to use for geo coordinates. */ - public GeoValidationMethod getValidationMethod() { - return this.validationMethod; - } - /** * Sets whether the query builder should ignore unmapped fields (and run a * {@link MatchNoDocsQuery} in place of this query) or throw an exception if @@ -141,15 +136,6 @@ public GeoPolygonQueryBuilder ignoreUnmapped(boolean ignoreUnmapped) { return this; } - /** - * Gets whether the query builder will ignore unmapped fields (and run a - * {@link MatchNoDocsQuery} in place of this query) or throw an exception if - * the field is unmapped. - */ - public boolean ignoreUnmapped() { - return ignoreUnmapped; - } - @Override protected Query doToQuery(SearchExecutionContext context) throws IOException { MappedFieldType fieldType = context.getFieldType(fieldName); diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java index f7639a55dd9b0..845023d2d832d 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java @@ -13,7 +13,6 @@ import org.apache.lucene.search.Query; import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; -import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.geo.GeometryParser; import org.elasticsearch.common.geo.ShapeRelation; import org.elasticsearch.common.geo.SpatialStrategy; @@ -132,7 +131,7 @@ public GeoShapeQueryBuilder relation(ShapeRelation relation) { * @return this */ public GeoShapeQueryBuilder strategy(SpatialStrategy strategy) { - if (strategy != null && strategy == SpatialStrategy.TERM && relation != ShapeRelation.INTERSECTS) { + if (strategy == SpatialStrategy.TERM && relation != ShapeRelation.INTERSECTS) { throw new IllegalArgumentException( "strategy [" + strategy.getStrategyName() @@ -217,11 +216,7 @@ protected boolean parseXContentField(XContentParser parser) throws IOException { } else if (STRATEGY_FIELD.match(parser.currentName(), parser.getDeprecationHandler())) { String strategyName = parser.text(); strategy = SpatialStrategy.fromString(strategyName); - if (strategy == null) { - throw new ParsingException(parser.getTokenLocation(), "Unknown strategy [" + strategyName + " ]"); - } else { - this.strategy = strategy; - } + this.strategy = strategy; return true; } return false; diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoValidationMethod.java b/server/src/main/java/org/elasticsearch/index/query/GeoValidationMethod.java index f101ee456c8c4..2c678719a25d4 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoValidationMethod.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoValidationMethod.java @@ -62,15 +62,4 @@ public static boolean isCoerce(GeoValidationMethod method) { return method == GeoValidationMethod.COERCE; } - /** Returns validation method corresponding to given coerce and ignoreMalformed values. */ - public static GeoValidationMethod infer(boolean coerce, boolean ignoreMalformed) { - if (coerce) { - return GeoValidationMethod.COERCE; - } else if (ignoreMalformed) { - return GeoValidationMethod.IGNORE_MALFORMED; - } else { - return GeoValidationMethod.STRICT; - } - } - } diff --git a/server/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java b/server/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java index 806f28d72647a..edf32caaee4ac 100644 --- a/server/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/InnerHitBuilder.java @@ -46,7 +46,6 @@ public final class InnerHitBuilder implements Writeable, ToXContentObject { public static final ParseField NAME_FIELD = new ParseField("name"); public static final ParseField IGNORE_UNMAPPED = new ParseField("ignore_unmapped"); - public static final QueryBuilder DEFAULT_INNER_HIT_QUERY = new MatchAllQueryBuilder(); public static final ParseField COLLAPSE_FIELD = new ParseField("collapse"); public static final ParseField FIELD_FIELD = new ParseField("field"); diff --git a/server/src/main/java/org/elasticsearch/index/query/IntervalQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/IntervalQueryBuilder.java index bd26bb21e1f01..0731eef5bfe35 100644 --- a/server/src/main/java/org/elasticsearch/index/query/IntervalQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/IntervalQueryBuilder.java @@ -47,10 +47,6 @@ public IntervalQueryBuilder(StreamInput in) throws IOException { this.sourceProvider = in.readNamedWriteable(IntervalsSourceProvider.class); } - public String getField() { - return field; - } - public IntervalsSourceProvider getSourceProvider() { return sourceProvider; } diff --git a/server/src/main/java/org/elasticsearch/index/query/MatchBoolPrefixQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MatchBoolPrefixQueryBuilder.java index d28fe251806f4..094627ef0711b 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MatchBoolPrefixQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MatchBoolPrefixQueryBuilder.java @@ -103,21 +103,11 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeOptionalString(fuzzyRewrite); } - /** Returns the field name used in this query. */ - public String fieldName() { - return this.fieldName; - } - /** Returns the value used in this query. */ public Object value() { return this.value; } - /** Get the analyzer to use, if previously set, otherwise {@code null} */ - public String analyzer() { - return this.analyzer; - } - /** * Explicitly set the analyzer to use. Defaults to use explicit mapping * config for the field, or, if not set, the default search analyzer. @@ -136,11 +126,6 @@ public MatchBoolPrefixQueryBuilder operator(Operator operator) { return this; } - /** Returns the operator to use in a boolean query.*/ - public Operator operator() { - return this.operator; - } - /** Sets optional minimumShouldMatch value to apply to the query */ public MatchBoolPrefixQueryBuilder minimumShouldMatch(String minimumShouldMatch) { this.minimumShouldMatch = minimumShouldMatch; @@ -194,13 +179,6 @@ public MatchBoolPrefixQueryBuilder maxExpansions(int maxExpansions) { return this; } - /** - * Get the (optional) number of term expansions when using fuzzy or prefix type query. - */ - public int maxExpansions() { - return this.maxExpansions; - } - /** * Sets whether transpositions are supported in fuzzy queries.

* The default metric used by fuzzy queries to determine a match is the Damerau-Levenshtein @@ -224,14 +202,6 @@ public MatchBoolPrefixQueryBuilder fuzzyRewrite(String fuzzyRewrite) { return this; } - /** - * Get the fuzzy_rewrite parameter - * @see #fuzzyRewrite(String) - */ - public String fuzzyRewrite() { - return this.fuzzyRewrite; - } - @Override protected void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(NAME); diff --git a/server/src/main/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilder.java index dae2444573124..da1a760d1414b 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MatchPhrasePrefixQueryBuilder.java @@ -102,11 +102,6 @@ public MatchPhrasePrefixQueryBuilder analyzer(String analyzer) { return this; } - /** Get the analyzer to use, if previously set, otherwise {@code null} */ - public String analyzer() { - return this.analyzer; - } - /** Sets a slop factor for phrase queries */ public MatchPhrasePrefixQueryBuilder slop(int slop) { if (slop < 0) { diff --git a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java index 7e644a8800bbd..30b6134650524 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java @@ -31,7 +31,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.lucene.search.MoreLikeThisQuery; import org.elasticsearch.common.lucene.search.XMoreLikeThis; import org.elasticsearch.common.lucene.uid.Versions; @@ -69,9 +68,6 @@ */ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder { public static final String NAME = "more_like_this"; - private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(MoreLikeThisQueryBuilder.class); - static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Types are deprecated in [more_like_this] " - + "queries. The type should no longer be specified in the [like] and [unlike] sections."; public static final int DEFAULT_MAX_QUERY_TERMS = XMoreLikeThis.DEFAULT_MAX_QUERY_TERMS; public static final int DEFAULT_MIN_TERM_FREQ = XMoreLikeThis.DEFAULT_MIN_TERM_FREQ; @@ -105,7 +101,6 @@ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder perFieldAnalyzer() { - return perFieldAnalyzer; - } - /** * Sets the analyzer(s) to use at any given field. */ @@ -312,10 +303,6 @@ public Item versionType(VersionType versionType) { return this; } - XContentType xContentType() { - return xContentType; - } - /** * Convert this to a {@link TermVectorsRequest} for fetching the terms of the document. */ @@ -541,10 +528,6 @@ public MoreLikeThisQueryBuilder unlike(String[] unlikeTexts) { return this; } - public String[] unlikeTexts() { - return unlikeTexts; - } - /** * Sets the documents from which the terms should not be selected from. */ @@ -553,10 +536,6 @@ public MoreLikeThisQueryBuilder unlike(Item[] unlikeItems) { return this; } - public Item[] unlikeItems() { - return unlikeItems; - } - /** * Sets the maximum number of query terms that will be included in any generated query. * Defaults to {@code 25}. @@ -569,10 +548,6 @@ public MoreLikeThisQueryBuilder maxQueryTerms(int maxQueryTerms) { return this; } - public int maxQueryTerms() { - return maxQueryTerms; - } - /** * The frequency below which terms will be ignored in the source doc. The default * frequency is {@code 2}. @@ -582,10 +557,6 @@ public MoreLikeThisQueryBuilder minTermFreq(int minTermFreq) { return this; } - public int minTermFreq() { - return minTermFreq; - } - /** * Sets the frequency at which words will be ignored which do not occur in at least this * many docs. Defaults to {@code 5}. @@ -608,10 +579,6 @@ public MoreLikeThisQueryBuilder maxDocFreq(int maxDocFreq) { return this; } - public int maxDocFreq() { - return maxDocFreq; - } - /** * Sets the minimum word length below which words will be ignored. Defaults * to {@code 0}. @@ -634,10 +601,6 @@ public MoreLikeThisQueryBuilder maxWordLength(int maxWordLength) { return this; } - public int maxWordLength() { - return maxWordLength; - } - /** * Set the set of stopwords. *

@@ -700,10 +663,6 @@ public MoreLikeThisQueryBuilder boostTerms(float boostTerms) { return this; } - public float boostTerms() { - return boostTerms; - } - /** * Whether to include the input documents. Defaults to {@code false} */ @@ -724,10 +683,6 @@ public MoreLikeThisQueryBuilder failOnUnsupportedField(boolean fail) { return this; } - public boolean failOnUnsupportedField() { - return failOnUnsupportedField; - } - @Override protected void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(NAME); diff --git a/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java index 17e651ab24696..94a52e57d257e 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MultiMatchQueryBuilder.java @@ -124,7 +124,7 @@ public enum Type implements Writeable { */ BOOL_PREFIX(MatchQueryParser.Type.BOOLEAN_PREFIX, 1.0f, new ParseField("bool_prefix")); - private MatchQueryParser.Type matchQueryType; + private final MatchQueryParser.Type matchQueryType; private final float tieBreaker; private final ParseField parseField; diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryBuilders.java b/server/src/main/java/org/elasticsearch/index/query/QueryBuilders.java index 564050c972b92..66f1c9a74d4c2 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryBuilders.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryBuilders.java @@ -341,14 +341,6 @@ public static SpanOrQueryBuilder spanOrQuery(SpanQueryBuilder initialClause) { return new SpanOrQueryBuilder(initialClause); } - /** Creates a new {@code span_within} builder. - * @param big the big clause, it must enclose {@code little} for a match. - * @param little the little clause, it must be contained within {@code big} for a match. - */ - public static SpanWithinQueryBuilder spanWithinQuery(SpanQueryBuilder big, SpanQueryBuilder little) { - return new SpanWithinQueryBuilder(big, little); - } - /** * Creates a new {@code span_containing} builder. * @param big the big clause, it must enclose {@code little} for a match. @@ -660,36 +652,12 @@ public static GeoShapeQueryBuilder geoIntersectionQuery(String name, String inde return builder; } - /** - * A filter to filter indexed shapes that are contained by a shape - * - * @param name The shape field name - * @param shape Shape to use in the filter - */ - public static GeoShapeQueryBuilder geoWithinQuery(String name, Geometry shape) throws IOException { - GeoShapeQueryBuilder builder = geoShapeQuery(name, shape); - builder.relation(ShapeRelation.WITHIN); - return builder; - } - public static GeoShapeQueryBuilder geoWithinQuery(String name, String indexedShapeId) { GeoShapeQueryBuilder builder = geoShapeQuery(name, indexedShapeId); builder.relation(ShapeRelation.WITHIN); return builder; } - /** - * A filter to filter indexed shapes that are not intersection with the query shape - * - * @param name The shape field name - * @param shape Shape to use in the filter - */ - public static GeoShapeQueryBuilder geoDisjointQuery(String name, Geometry shape) throws IOException { - GeoShapeQueryBuilder builder = geoShapeQuery(name, shape); - builder.relation(ShapeRelation.DISJOINT); - return builder; - } - public static GeoShapeQueryBuilder geoDisjointQuery(String name, String indexedShapeId) { GeoShapeQueryBuilder builder = geoShapeQuery(name, indexedShapeId); builder.relation(ShapeRelation.DISJOINT); diff --git a/server/src/main/java/org/elasticsearch/index/query/functionscore/GaussDecayFunctionBuilder.java b/server/src/main/java/org/elasticsearch/index/query/functionscore/GaussDecayFunctionBuilder.java index 7d9bcbd4aef14..6dc74ca571647 100644 --- a/server/src/main/java/org/elasticsearch/index/query/functionscore/GaussDecayFunctionBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/functionscore/GaussDecayFunctionBuilder.java @@ -14,13 +14,11 @@ import org.elasticsearch.TransportVersions; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.xcontent.ParseField; import java.io.IOException; public class GaussDecayFunctionBuilder extends DecayFunctionBuilder { public static final String NAME = "gauss"; - public static final ParseField FUNCTION_NAME_FIELD = new ParseField(NAME); public static final ScoreFunctionParser PARSER = new DecayFunctionParser<>(GaussDecayFunctionBuilder::new); public static final DecayFunction GAUSS_DECAY_FUNCTION = new GaussScoreFunction(); diff --git a/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java b/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java index 06e4a6a3bbbd5..6d4b2dd4ab1f5 100644 --- a/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java @@ -111,14 +111,6 @@ public RandomScoreFunctionBuilder setField(String field) { return this; } - /** - * Get the field to use for random number generation. - * @see #setField(String) - */ - public String getField() { - return field; - } - @Override public void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(getName()); From c00abac6d1f14fe1cb793562d04c715f3b89b822 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 4 Nov 2024 19:05:52 +0100 Subject: [PATCH 16/35] Simplify AbstractSearchAsyncAction.doPerformPhaseOnShard (#116104) 1. No need to catch here any longer, we fixed the connection related exceptions separately now, throwing from an method that consumes a listener was smelly to begin with. 2. Not need to try-with-resources that `releasable`, just release it as soon as possible to get the next per-shard request going while we process a result. No need to waste time on an idle data node here. --- .../search/AbstractSearchAsyncAction.java | 36 +++++++------------ 1 file changed, 13 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 317d117174e94..ec441cd4e58c4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -296,33 +296,23 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat } private void doPerformPhaseOnShard(int shardIndex, SearchShardIterator shardIt, SearchShardTarget shard, Releasable releasable) { - try { - executePhaseOnShard(shardIt, shard, new SearchActionListener<>(shard, shardIndex) { - @Override - public void innerOnResponse(Result result) { - try (releasable) { - onShardResult(result, shardIt); - } catch (Exception exc) { - onShardFailure(shardIndex, shard, shardIt, exc); - } + executePhaseOnShard(shardIt, shard, new SearchActionListener<>(shard, shardIndex) { + @Override + public void innerOnResponse(Result result) { + try { + releasable.close(); + onShardResult(result, shardIt); + } catch (Exception exc) { + onShardFailure(shardIndex, shard, shardIt, exc); } + } - @Override - public void onFailure(Exception e) { - try (releasable) { - onShardFailure(shardIndex, shard, shardIt, e); - } - } - }); - } catch (final Exception e) { - /* - * It is possible to run into connection exceptions here because we are getting the connection early and might - * run into nodes that are not connected. In this case, on shard failure will move us to the next shard copy. - */ - try (releasable) { + @Override + public void onFailure(Exception e) { + releasable.close(); onShardFailure(shardIndex, shard, shardIt, e); } - } + }); } private void failOnUnavailable(int shardIndex, SearchShardIterator shardIt) { From aa9939233c4b3168e96105c925c9d9729ff3dfe8 Mon Sep 17 00:00:00 2001 From: Craig Taverner Date: Mon, 4 Nov 2024 19:06:50 +0100 Subject: [PATCH 17/35] Support partial sort fields in TopN pushdown (#116043) * Support partial sort fields in TopN pushdown * Update docs/changelog/116043.yaml * Update docs/changelog/116043.yaml * Refine physical planner tests --- docs/changelog/116043.yaml | 6 + .../physical/local/PushTopNToSource.java | 3 +- .../optimizer/PhysicalPlanOptimizerTests.java | 105 +++++++++++++++++- 3 files changed, 108 insertions(+), 6 deletions(-) create mode 100644 docs/changelog/116043.yaml diff --git a/docs/changelog/116043.yaml b/docs/changelog/116043.yaml new file mode 100644 index 0000000000000..9f90257ecd7d8 --- /dev/null +++ b/docs/changelog/116043.yaml @@ -0,0 +1,6 @@ +pr: 116043 +summary: Support partial sort fields in TopN pushdown +area: ES|QL +type: enhancement +issues: + - 114515 diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java index 855faf9df5ed2..2ae496b55ac00 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/physical/local/PushTopNToSource.java @@ -184,8 +184,7 @@ && canPushDownOrders(topNExec.order(), hasIdenticalDelegate)) { break; } } - // TODO: We can push down partial sorts where `pushableSorts.size() < orders.size()`, but that should involve benchmarks - if (pushableSorts.size() > 0 && pushableSorts.size() == orders.size()) { + if (pushableSorts.isEmpty() == false) { return new PushableCompoundExec(evalExec, queryExec, pushableSorts); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index 3b59a1d176a98..9f5d6440e4a06 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -4984,8 +4984,101 @@ public void testPushTopNDistanceWithCompoundFilterToSource() { } /** - * This test shows that with an additional EVAL used in the filter, we can no longer push down the SORT distance. - * TODO: This could be optimized in future work. Consider moving much of EnableSpatialDistancePushdown into logical planning. + * Tests that multiple sorts, including distance and a field, are pushed down to the source. + * + * ProjectExec[[abbrev{f}#25, name{f}#26, location{f}#29, country{f}#30, city{f}#31, scalerank{f}#27, scale{r}#7]] + * \_TopNExec[[ + * Order[distance{r}#4,ASC,LAST], + * Order[scalerank{f}#27,ASC,LAST], + * Order[scale{r}#7,DESC,FIRST], + * Order[loc{r}#10,DESC,FIRST] + * ],5[INTEGER],0] + * \_ExchangeExec[[abbrev{f}#25, name{f}#26, location{f}#29, country{f}#30, city{f}#31, scalerank{f}#27, scale{r}#7, + * distance{r}#4, loc{r}#10],false] + * \_ProjectExec[[abbrev{f}#25, name{f}#26, location{f}#29, country{f}#30, city{f}#31, scalerank{f}#27, scale{r}#7, + * distance{r}#4, loc{r}#10]] + * \_FieldExtractExec[abbrev{f}#25, name{f}#26, country{f}#30, city{f}#31][] + * \_EvalExec[[ + * STDISTANCE(location{f}#29,[1 1 0 0 0 e1 7a 14 ae 47 21 29 40 a0 1a 2f dd 24 d6 4b 40][GEO_POINT]) AS distance, + * 10[INTEGER] - scalerank{f}#27 AS scale, TOSTRING(location{f}#29) AS loc + * ]] + * \_FieldExtractExec[location{f}#29, scalerank{f}#27][] + * \_EsQueryExec[airports], indexMode[standard], query[{ + * "bool":{ + * "filter":[ + * {"esql_single_value":{"field":"scalerank","next":{...},"source":"scalerank < 6@3:31"}}, + * {"bool":{ + * "must":[ + * {"geo_shape":{"location":{"relation":"INTERSECTS","shape":{...}}}}, + * {"geo_shape":{"location":{"relation":"DISJOINT","shape":{...}}}} + * ],"boost":1.0}}],"boost":1.0}}][_doc{f}#44], limit[5], sort[[ + * GeoDistanceSort[field=location{f}#29, direction=ASC, lat=55.673, lon=12.565], + * FieldSort[field=scalerank{f}#27, direction=ASC, nulls=LAST] + * ]] estimatedRowSize[303] + * + */ + public void testPushTopNDistanceAndPushableFieldWithCompoundFilterToSource() { + var optimized = optimizedPlan(physicalPlan(""" + FROM airports + | EVAL distance = ST_DISTANCE(location, TO_GEOPOINT("POINT(12.565 55.673)")), scale = 10 - scalerank, loc = location::string + | WHERE distance < 500000 AND scalerank < 6 AND distance > 10000 + | SORT distance ASC, scalerank ASC, scale DESC, loc DESC + | LIMIT 5 + | KEEP abbrev, name, location, country, city, scalerank, scale + """, airports)); + + var project = as(optimized, ProjectExec.class); + var topN = as(project.child(), TopNExec.class); + assertThat(topN.order().size(), is(4)); + var exchange = asRemoteExchange(topN.child()); + + project = as(exchange.child(), ProjectExec.class); + assertThat( + names(project.projections()), + contains("abbrev", "name", "location", "country", "city", "scalerank", "scale", "distance", "loc") + ); + var extract = as(project.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("abbrev", "name", "country", "city")); + var evalExec = as(extract.child(), EvalExec.class); + var alias = as(evalExec.fields().get(0), Alias.class); + assertThat(alias.name(), is("distance")); + var stDistance = as(alias.child(), StDistance.class); + assertThat(stDistance.left().toString(), startsWith("location")); + extract = as(evalExec.child(), FieldExtractExec.class); + assertThat(names(extract.attributesToExtract()), contains("location", "scalerank")); + var source = source(extract.child()); + + // Assert that the TopN(distance) is pushed down as geo-sort(location) + assertThat(source.limit(), is(topN.limit())); + Set orderSet = orderAsSet(topN.order().subList(0, 2)); + Set sortsSet = sortsAsSet(source.sorts(), Map.of("location", "distance")); + assertThat(orderSet, is(sortsSet)); + + // Fine-grained checks on the pushed down sort + assertThat(source.limit(), is(l(5))); + assertThat(source.sorts().size(), is(2)); + EsQueryExec.Sort sort = source.sorts().get(0); + assertThat(sort.direction(), is(Order.OrderDirection.ASC)); + assertThat(name(sort.field()), is("location")); + assertThat(sort.sortBuilder(), isA(GeoDistanceSortBuilder.class)); + sort = source.sorts().get(1); + assertThat(sort.direction(), is(Order.OrderDirection.ASC)); + assertThat(name(sort.field()), is("scalerank")); + assertThat(sort.sortBuilder(), isA(FieldSortBuilder.class)); + + // Fine-grained checks on the pushed down query + var bool = as(source.query(), BoolQueryBuilder.class); + var rangeQueryBuilders = bool.filter().stream().filter(p -> p instanceof SingleValueQuery.Builder).toList(); + assertThat("Expected one range query builder", rangeQueryBuilders.size(), equalTo(1)); + assertThat(((SingleValueQuery.Builder) rangeQueryBuilders.get(0)).field(), equalTo("scalerank")); + var filterBool = bool.filter().stream().filter(p -> p instanceof BoolQueryBuilder).toList(); + var fb = as(filterBool.get(0), BoolQueryBuilder.class); + var shapeQueryBuilders = fb.must().stream().filter(p -> p instanceof SpatialRelatesQuery.ShapeQueryBuilder).toList(); + assertShapeQueryRange(shapeQueryBuilders, 10000.0, 500000.0); + } + + /** + * This test shows that if the filter contains a predicate on the same field that is sorted, we cannot push down the sort. * * ProjectExec[[abbrev{f}#23, name{f}#24, location{f}#27, country{f}#28, city{f}#29, scalerank{f}#25 AS scale]] * \_TopNExec[[Order[distance{r}#4,ASC,LAST], Order[scalerank{f}#25,ASC,LAST]],5[INTEGER],0] @@ -5021,6 +5114,7 @@ public void testPushTopNDistanceAndNonPushableEvalWithCompoundFilterToSource() { var project = as(optimized, ProjectExec.class); var topN = as(project.child(), TopNExec.class); + assertThat(topN.order().size(), is(2)); var exchange = asRemoteExchange(topN.child()); project = as(exchange.child(), ProjectExec.class); @@ -5059,7 +5153,7 @@ public void testPushTopNDistanceAndNonPushableEvalWithCompoundFilterToSource() { } /** - * This test further shows that with a non-aliasing function, with the same name, less gets pushed down. + * This test shows that if the filter contains a predicate on the same field that is sorted, we cannot push down the sort. * * ProjectExec[[abbrev{f}#23, name{f}#24, location{f}#27, country{f}#28, city{f}#29, scale{r}#10]] * \_TopNExec[[Order[distance{r}#4,ASC,LAST], Order[scale{r}#10,ASC,LAST]],5[INTEGER],0] @@ -5096,6 +5190,7 @@ public void testPushTopNDistanceAndNonPushableEvalsWithCompoundFilterToSource() """, airports)); var project = as(optimized, ProjectExec.class); var topN = as(project.child(), TopNExec.class); + assertThat(topN.order().size(), is(2)); var exchange = asRemoteExchange(topN.child()); project = as(exchange.child(), ProjectExec.class); @@ -5133,7 +5228,8 @@ public void testPushTopNDistanceAndNonPushableEvalsWithCompoundFilterToSource() } /** - * This test shows that with if the top level AND'd predicate contains a non-pushable component, we should not push anything. + * This test shows that with if the top level predicate contains a non-pushable component (eg. disjunction), + * we should not push down the filter. * * ProjectExec[[abbrev{f}#8612, name{f}#8613, location{f}#8616, country{f}#8617, city{f}#8618, scalerank{f}#8614 AS scale]] * \_TopNExec[[Order[distance{r}#8596,ASC,LAST], Order[scalerank{f}#8614,ASC,LAST]],5[INTEGER],0] @@ -5171,6 +5267,7 @@ public void testPushTopNDistanceWithCompoundFilterToSourceAndDisjunctiveNonPusha var project = as(optimized, ProjectExec.class); var topN = as(project.child(), TopNExec.class); + assertThat(topN.order().size(), is(2)); var exchange = asRemoteExchange(topN.child()); project = as(exchange.child(), ProjectExec.class); From 427b594e98d7fbebeb927e58f740c30157e1685f Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 4 Nov 2024 19:07:05 +0100 Subject: [PATCH 18/35] Cleanup listener wrapping in TransportSearchAction a little (#116184) Lets avoid parsing settings during search request processing, remove unused field from the listener and using the delegating listener class. --- .../action/search/TransportSearchAction.java | 41 ++++++++----------- 1 file changed, 18 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 8f718972c2eaa..35f106ab58cbc 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -16,6 +16,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.DelegatingActionListener; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.RemoteClusterActionType; @@ -52,7 +53,6 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.ArrayUtils; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.Maps; @@ -160,7 +160,7 @@ public class TransportSearchAction extends HandledTransportAction listener) { - executeRequest( - (SearchTask) task, - searchRequest, - new SearchResponseActionListener((SearchTask) task, listener), - AsyncSearchActionProvider::new - ); + executeRequest((SearchTask) task, searchRequest, new SearchResponseActionListener(listener), AsyncSearchActionProvider::new); } void executeRequest( @@ -372,7 +368,7 @@ void executeRequest( searchPhaseProvider.apply(delegate) ); } else { - if (listener instanceof TelemetryListener tl) { + if (delegate instanceof TelemetryListener tl) { tl.setRemotes(resolvedIndices.getRemoteClusterIndices().size()); if (task.isAsync()) { tl.setFeature(CCSUsageTelemetry.ASYNC_FEATURE); @@ -398,7 +394,7 @@ void executeRequest( } final TaskId parentTaskId = task.taskInfo(clusterService.localNode().getId(), false).taskId(); if (shouldMinimizeRoundtrips(rewritten)) { - if (listener instanceof TelemetryListener tl) { + if (delegate instanceof TelemetryListener tl) { tl.setFeature(CCSUsageTelemetry.MRT_FEATURE); } final AggregationReduceContext.Builder aggregationReduceContextBuilder = rewritten.source() != null @@ -508,7 +504,7 @@ void executeRequest( // We set the keep alive to -1 to indicate that we don't need the pit id in the response. // This is needed since we delete the pit prior to sending the response so the id doesn't exist anymore. source.pointInTimeBuilder(new PointInTimeBuilder(resp.getPointInTimeId()).setKeepAlive(TimeValue.MINUS_ONE)); - var pitListener = new SearchResponseActionListener(task, listener) { + var pitListener = new SearchResponseActionListener(delegate) { @Override public void onResponse(SearchResponse response) { // we need to close the PIT first so we delay the release of the response to after the closing @@ -516,13 +512,13 @@ public void onResponse(SearchResponse response) { closePIT( client, original.source().pointInTimeBuilder(), - () -> ActionListener.respondAndRelease(listener, response) + () -> ActionListener.respondAndRelease(delegate, response) ); } @Override public void onFailure(Exception e) { - closePIT(client, original.source().pointInTimeBuilder(), () -> listener.onFailure(e)); + closePIT(client, original.source().pointInTimeBuilder(), () -> delegate.onFailure(e)); } }; executeRequest(task, original, pitListener, searchPhaseProvider); @@ -1874,14 +1870,13 @@ private interface TelemetryListener { void setClient(String client); } - private class SearchResponseActionListener implements ActionListener, TelemetryListener { - private final SearchTask task; - private final ActionListener listener; + private class SearchResponseActionListener extends DelegatingActionListener + implements + TelemetryListener { private final CCSUsage.Builder usageBuilder; - SearchResponseActionListener(SearchTask task, ActionListener listener) { - this.task = task; - this.listener = listener; + SearchResponseActionListener(ActionListener listener) { + super(listener); if (listener instanceof SearchResponseActionListener srListener) { usageBuilder = srListener.usageBuilder; } else { @@ -1893,7 +1888,7 @@ private class SearchResponseActionListener implements ActionListener 0; + return collectTelemetry && usageBuilder.getRemotesCount() > 0; } public void setRemotes(int count) { @@ -1942,7 +1937,7 @@ public void onResponse(SearchResponse searchResponse) { return; } // This is last because we want to collect telemetry before returning the response. - listener.onResponse(searchResponse); + delegate.onResponse(searchResponse); } @Override @@ -1952,7 +1947,7 @@ public void onFailure(Exception e) { usageBuilder.setFailure(e); recordTelemetry(); } - listener.onFailure(e); + super.onFailure(e); } private void recordTelemetry() { From 6911227c058fb02084296cc548e888d6256fdbe5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lorenzo=20Dematt=C3=A9?= Date: Mon, 4 Nov 2024 19:07:34 +0100 Subject: [PATCH 19/35] Add ClusterStateSupplier interface + standard implementation (#115931) --- .../cluster/ClusterStateSupplier.java | 25 +++++++++++ .../cluster/SafeClusterStateSupplier.java | 44 +++++++++++++++++++ 2 files changed, 69 insertions(+) create mode 100644 server/src/main/java/org/elasticsearch/cluster/ClusterStateSupplier.java create mode 100644 server/src/main/java/org/elasticsearch/cluster/SafeClusterStateSupplier.java diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterStateSupplier.java b/server/src/main/java/org/elasticsearch/cluster/ClusterStateSupplier.java new file mode 100644 index 0000000000000..61bb049ffd5c5 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterStateSupplier.java @@ -0,0 +1,25 @@ +/* + * 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.cluster; + +import java.util.Optional; +import java.util.function.Function; +import java.util.function.Supplier; + +/** + * Utility to access {@link ClusterState} only when it is "ready", with a fallback if it's not. The definition of "ready" is left to the + * class implementations. + */ +public interface ClusterStateSupplier extends Supplier> { + default T withCurrentClusterState(Function clusterStateFunction, T fallbackIfNotReady) { + var x = get(); + return x.map(clusterStateFunction).orElse(fallbackIfNotReady); + } +} diff --git a/server/src/main/java/org/elasticsearch/cluster/SafeClusterStateSupplier.java b/server/src/main/java/org/elasticsearch/cluster/SafeClusterStateSupplier.java new file mode 100644 index 0000000000000..b12ef3d78f864 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/SafeClusterStateSupplier.java @@ -0,0 +1,44 @@ +/* + * 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.cluster; + +import java.util.Optional; + +import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; + +/** + * Utility to access {@link ClusterState} only when it is "ready", where "ready" means that we received a first clusterChanged event + * with no global block of type {@code STATE_NOT_RECOVERED_BLOCK} + * This guarantees that: + * - the initial cluster state has been set (see + * {@link org.elasticsearch.cluster.service.ClusterApplierService#setInitialState(ClusterState)}); + * - the initial recovery process has completed. + */ +public class SafeClusterStateSupplier implements ClusterStateSupplier, ClusterStateListener { + private volatile ClusterState currentClusterState; + + @Override + public void clusterChanged(ClusterChangedEvent event) { + // In this default implementation, "ready" is really "is cluster state available", which after the initial recovery it should be. + // If you need a different condition, feel free to add a different implementation of ClusterStateSupplier + if (isInitialized() || event.state().blocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK) == false) { + currentClusterState = event.state(); + } + } + + private boolean isInitialized() { + return currentClusterState != null; + } + + @Override + public Optional get() { + return Optional.ofNullable(currentClusterState); + } +} From 82d21f3d891ebc1ffb25b64a3bf85d885903ad4c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lorenzo=20Dematt=C3=A9?= Date: Mon, 4 Nov 2024 20:23:02 +0100 Subject: [PATCH 20/35] Minor fixes to improve SystemIndexDescriptor (#113278) Closes https://github.com/elastic/elasticsearch/issues/112946 --- .../admin/indices/create/AutoCreateAction.java | 11 +++++++---- .../indices/create/TransportCreateIndexAction.java | 7 +++---- .../indices/SystemIndexDescriptor.java | 13 +++++++++---- .../security/support/SecurityIndexManager.java | 10 ++++++++-- .../security/support/SecurityIndexManagerTests.java | 3 ++- 5 files changed, 29 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/AutoCreateAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/AutoCreateAction.java index 823bff904283b..9bc088f944be0 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/AutoCreateAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/AutoCreateAction.java @@ -305,11 +305,14 @@ ClusterState execute( final CreateIndexClusterStateUpdateRequest updateRequest; if (isManagedSystemIndex) { - final SystemIndexDescriptor descriptor = mainDescriptor.getDescriptorCompatibleWith( - currentState.getMinSystemIndexMappingVersions().get(mainDescriptor.getPrimaryIndex()) - ); + final var requiredMinimumMappingVersion = currentState.getMinSystemIndexMappingVersions() + .get(mainDescriptor.getPrimaryIndex()); + final SystemIndexDescriptor descriptor = mainDescriptor.getDescriptorCompatibleWith(requiredMinimumMappingVersion); if (descriptor == null) { - final String message = mainDescriptor.getMinimumMappingsVersionMessage("auto-create index"); + final String message = mainDescriptor.getMinimumMappingsVersionMessage( + "auto-create index", + requiredMinimumMappingVersion + ); logger.warn(message); throw new IllegalStateException(message); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java index b43f2006061a5..73f35e283f7d8 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/TransportCreateIndexAction.java @@ -135,11 +135,10 @@ protected void masterOperation( // We check this via the request's origin. Eventually, `SystemIndexManager` will reconfigure // the index to the latest settings. if (isManagedSystemIndex && Strings.isNullOrEmpty(request.origin())) { - final SystemIndexDescriptor descriptor = mainDescriptor.getDescriptorCompatibleWith( - state.getMinSystemIndexMappingVersions().get(mainDescriptor.getPrimaryIndex()) - ); + final var requiredMinimumMappingVersion = state.getMinSystemIndexMappingVersions().get(mainDescriptor.getPrimaryIndex()); + final SystemIndexDescriptor descriptor = mainDescriptor.getDescriptorCompatibleWith(requiredMinimumMappingVersion); if (descriptor == null) { - final String message = mainDescriptor.getMinimumMappingsVersionMessage("create index"); + final String message = mainDescriptor.getMinimumMappingsVersionMessage("create index", requiredMinimumMappingVersion); logger.warn(message); listener.onFailure(new IllegalStateException(message)); return; diff --git a/server/src/main/java/org/elasticsearch/indices/SystemIndexDescriptor.java b/server/src/main/java/org/elasticsearch/indices/SystemIndexDescriptor.java index 8b9610a52cc3d..9b58c37fff27e 100644 --- a/server/src/main/java/org/elasticsearch/indices/SystemIndexDescriptor.java +++ b/server/src/main/java/org/elasticsearch/indices/SystemIndexDescriptor.java @@ -316,7 +316,7 @@ protected SystemIndexDescriptor( if (prior.primaryIndex.equals(primaryIndex) == false) { throw new IllegalArgumentException("primary index must be the same"); } - if (prior.aliasName.equals(aliasName) == false) { + if (Objects.equals(prior.aliasName, aliasName) == false) { throw new IllegalArgumentException("alias name must be the same"); } } @@ -512,16 +512,21 @@ public MappingsVersion getMappingsVersion() { * @param cause the action being attempted that triggered the check. Used in the error message. * @return the standardized error message */ - public String getMinimumMappingsVersionMessage(String cause) { + public String getMinimumMappingsVersionMessage(String cause, MappingsVersion requiredMinimumMappingVersion) { Objects.requireNonNull(cause); final MappingsVersion actualMinimumMappingsVersion = priorSystemIndexDescriptors.isEmpty() ? getMappingsVersion() : priorSystemIndexDescriptors.get(priorSystemIndexDescriptors.size() - 1).mappingsVersion; return Strings.format( - "[%s] failed - system index [%s] requires all data and master nodes to have mappings versions at least of version [%s]", + "[%s] failed - requested creation of system index [%s] with version [%s], while this cluster minimum supported version is " + + "[%s]. For the cluster to support version [%s], ensure that the system index descriptor for [%s] includes a prior " + + "definition for that version.", cause, this.getPrimaryIndex(), - actualMinimumMappingsVersion + requiredMinimumMappingVersion, + actualMinimumMappingsVersion, + requiredMinimumMappingVersion, + this.getPrimaryIndex() ); } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java index 12ef800a7aae7..f3222a74b530c 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java @@ -652,7 +652,10 @@ public void prepareIndexIfNeededThenExecute(final Consumer consumer, ); if (descriptorForVersion == null) { - final String error = systemIndexDescriptor.getMinimumMappingsVersionMessage("create index"); + final String error = systemIndexDescriptor.getMinimumMappingsVersionMessage( + "create index", + state.minClusterMappingVersion + ); consumer.accept(new IllegalStateException(error)); } else { logger.info( @@ -703,7 +706,10 @@ public void onFailure(Exception e) { ); if (descriptorForVersion == null) { - final String error = systemIndexDescriptor.getMinimumMappingsVersionMessage("updating mapping"); + final String error = systemIndexDescriptor.getMinimumMappingsVersionMessage( + "updating mapping", + state.minClusterMappingVersion + ); consumer.accept(new IllegalStateException(error)); } else { logger.info( diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java index 0b98a595a6ab9..15d900e9d06ae 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/SecurityIndexManagerTests.java @@ -82,6 +82,7 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doReturn; @@ -434,7 +435,7 @@ public void testCannotUpdateIndexMappingsWhenMinMappingVersionTooLow() { final AtomicReference prepareException = new AtomicReference<>(null); // Hard-code a failure here. - doReturn("Nope").when(descriptorSpy).getMinimumMappingsVersionMessage(anyString()); + doReturn("Nope").when(descriptorSpy).getMinimumMappingsVersionMessage(anyString(), any()); doReturn(null).when(descriptorSpy) .getDescriptorCompatibleWith(eq(new SystemIndexDescriptor.MappingsVersion(SecurityMainIndexMappingVersion.latest().id(), 0))); From cc6316f9f65436cc82e9d6712295dfb04eef2c16 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Mon, 4 Nov 2024 15:09:54 -0500 Subject: [PATCH 21/35] ESQL: Fix test (#116096) I missed this in #116086. The test needs to have a good `toString` for it to pass. Closes #116094 --- muted-tests.yml | 3 --- .../compute/operator/FilterOperatorTests.java | 13 ++++++++++++- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 036daa0105c61..534a25a402b2c 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -225,9 +225,6 @@ tests: issue: https://github.com/elastic/elasticsearch/issues/116050 - class: org.elasticsearch.xpack.application.connector.ConnectorIndexServiceTests issue: https://github.com/elastic/elasticsearch/issues/116087 -- class: org.elasticsearch.compute.operator.FilterOperatorTests - method: testSimpleDescription - issue: https://github.com/elastic/elasticsearch/issues/116094 - class: org.elasticsearch.xpack.searchbusinessrules.PinnedQueryBuilderIT method: testPinnedPromotions issue: https://github.com/elastic/elasticsearch/issues/116097 diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java index f1fda67c36dda..4bf9f3942c0a8 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java @@ -53,7 +53,18 @@ public void close() {} @Override protected Operator.OperatorFactory simple() { - return new FilterOperator.FilterOperatorFactory(dvrCtx -> new SameLastDigit(dvrCtx, 0, 1)); + return new FilterOperator.FilterOperatorFactory(new EvalOperator.ExpressionEvaluator.Factory() { + + @Override + public EvalOperator.ExpressionEvaluator get(DriverContext context) { + return new SameLastDigit(context, 0, 1); + } + + @Override + public String toString() { + return "SameLastDigit[lhs=0, rhs=1]"; + } + }); } @Override From 8bc6cec95cad3afaed5cdf27495688b0dc497add Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 4 Nov 2024 13:25:05 -0700 Subject: [PATCH 22/35] Parse bulk lines in individual steps (#114086) Currently our incremental bulk parsing framework only parses once both the action line and document line are available. In addition, it will re-search lines for line delimiters as data is received. This commit ensures that the state is not lost in between parse attempts. --- .../action/bulk/BulkRequestParser.java | 279 ++++++++++++------ .../rest/action/document/RestBulkAction.java | 56 ++-- .../action/bulk/BulkRequestParserTests.java | 104 ++++++- .../action/bulk/BulkRequestTests.java | 2 +- .../action/document/RestBulkActionTests.java | 2 +- 5 files changed, 310 insertions(+), 133 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java index 8712430918fbf..9be1feae5ccfe 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestParser.java @@ -89,13 +89,13 @@ public BulkRequestParser(boolean deprecateOrErrorOnType, RestApiVersion restApiV .withRestApiVersion(restApiVersion); } - private static int findNextMarker(byte marker, int from, BytesReference data, boolean isIncremental) { + private static int findNextMarker(byte marker, int from, BytesReference data, boolean lastData) { final int res = data.indexOf(marker, from); if (res != -1) { assert res >= 0; return res; } - if (from != data.length() && isIncremental == false) { + if (from != data.length() && lastData) { throw new IllegalArgumentException("The bulk request must be terminated by a newline [\\n]"); } return res; @@ -140,13 +140,7 @@ public void parse( Consumer updateRequestConsumer, Consumer deleteRequestConsumer ) throws IOException { - // Bulk requests can contain a lot of repeated strings for the index, pipeline and routing parameters. This map is used to - // deduplicate duplicate strings parsed for these parameters. While it does not prevent instantiating the duplicate strings, it - // reduces their lifetime to the lifetime of this parse call instead of the lifetime of the full bulk request. - final Map stringDeduplicator = new HashMap<>(); - - incrementalParse( - data, + IncrementalParser incrementalParser = new IncrementalParser( defaultIndex, defaultRouting, defaultFetchSourceContext, @@ -158,53 +152,163 @@ public void parse( xContentType, indexRequestConsumer, updateRequestConsumer, - deleteRequestConsumer, - false, - stringDeduplicator + deleteRequestConsumer ); + + incrementalParser.parse(data, true); } - public int incrementalParse( - BytesReference data, - String defaultIndex, - String defaultRouting, - FetchSourceContext defaultFetchSourceContext, - String defaultPipeline, - Boolean defaultRequireAlias, - Boolean defaultRequireDataStream, - Boolean defaultListExecutedPipelines, + public IncrementalParser incrementalParser( + @Nullable String defaultIndex, + @Nullable String defaultRouting, + @Nullable FetchSourceContext defaultFetchSourceContext, + @Nullable String defaultPipeline, + @Nullable Boolean defaultRequireAlias, + @Nullable Boolean defaultRequireDataStream, + @Nullable Boolean defaultListExecutedPipelines, boolean allowExplicitIndex, XContentType xContentType, BiConsumer indexRequestConsumer, Consumer updateRequestConsumer, - Consumer deleteRequestConsumer, - boolean isIncremental, - Map stringDeduplicator - ) throws IOException { - XContent xContent = xContentType.xContent(); - byte marker = xContent.bulkSeparator(); - boolean typesDeprecationLogged = false; - - int line = 0; - int from = 0; - int consumed = 0; - - while (true) { - int nextMarker = findNextMarker(marker, from, data, isIncremental); - if (nextMarker == -1) { - break; + Consumer deleteRequestConsumer + ) { + return new IncrementalParser( + defaultIndex, + defaultRouting, + defaultFetchSourceContext, + defaultPipeline, + defaultRequireAlias, + defaultRequireDataStream, + defaultListExecutedPipelines, + allowExplicitIndex, + xContentType, + indexRequestConsumer, + updateRequestConsumer, + deleteRequestConsumer + ); + } + + public class IncrementalParser { + + // Bulk requests can contain a lot of repeated strings for the index, pipeline and routing parameters. This map is used to + // deduplicate duplicate strings parsed for these parameters. While it does not prevent instantiating the duplicate strings, it + // reduces their lifetime to the lifetime of this parse call instead of the lifetime of the full bulk request. + private final Map stringDeduplicator = new HashMap<>(); + + private final String defaultIndex; + private final String defaultRouting; + private final FetchSourceContext defaultFetchSourceContext; + private final String defaultPipeline; + private final Boolean defaultRequireAlias; + private final Boolean defaultRequireDataStream; + private final Boolean defaultListExecutedPipelines; + private final boolean allowExplicitIndex; + + private final XContentType xContentType; + private final byte marker; + private final BiConsumer indexRequestConsumer; + private final Consumer updateRequestConsumer; + private final Consumer deleteRequestConsumer; + + private Exception failure = null; + private int incrementalFromOffset = 0; + private int line = 0; + + private DocWriteRequest currentRequest = null; + private String currentType = null; + private String currentPipeline = null; + private boolean currentListExecutedPipelines = false; + private FetchSourceContext currentFetchSourceContext = null; + + private IncrementalParser( + @Nullable String defaultIndex, + @Nullable String defaultRouting, + @Nullable FetchSourceContext defaultFetchSourceContext, + @Nullable String defaultPipeline, + @Nullable Boolean defaultRequireAlias, + @Nullable Boolean defaultRequireDataStream, + @Nullable Boolean defaultListExecutedPipelines, + boolean allowExplicitIndex, + XContentType xContentType, + BiConsumer indexRequestConsumer, + Consumer updateRequestConsumer, + Consumer deleteRequestConsumer + ) { + this.defaultIndex = defaultIndex; + this.defaultRouting = defaultRouting; + this.defaultFetchSourceContext = defaultFetchSourceContext; + this.defaultPipeline = defaultPipeline; + this.defaultRequireAlias = defaultRequireAlias; + this.defaultRequireDataStream = defaultRequireDataStream; + this.defaultListExecutedPipelines = defaultListExecutedPipelines; + this.allowExplicitIndex = allowExplicitIndex; + this.xContentType = xContentType; + this.marker = xContentType.xContent().bulkSeparator(); + this.indexRequestConsumer = indexRequestConsumer; + this.updateRequestConsumer = updateRequestConsumer; + this.deleteRequestConsumer = deleteRequestConsumer; + } + + public int parse(BytesReference data, boolean lastData) throws IOException { + if (failure != null) { + assert false : failure.getMessage(); + throw new IllegalStateException("Parser has already encountered exception", failure); } - line++; + try { + return tryParse(data, lastData); + } catch (Exception e) { + failure = e; + throw e; + } + } + + private int tryParse(BytesReference data, boolean lastData) throws IOException { + int from = 0; + int consumed = 0; + + while (true) { + int nextMarker = findNextMarker(marker, incrementalFromOffset, data, lastData); + if (nextMarker == -1) { + incrementalFromOffset = data.length() - consumed; + break; + } + incrementalFromOffset = nextMarker + 1; + line++; + + if (currentRequest == null) { + if (parseActionLine(data, from, nextMarker)) { + if (currentRequest instanceof DeleteRequest deleteRequest) { + deleteRequestConsumer.accept(deleteRequest); + currentRequest = null; + } + } + } else { + parseAndConsumeDocumentLine(data, from, nextMarker); + currentRequest = null; + } - // now parse the action - try (XContentParser parser = createParser(xContent, data, from, nextMarker)) { - // move pointers from = nextMarker + 1; + consumed = from; + } + + return lastData ? from : consumed; + } + + private boolean parseActionLine(BytesReference data, int from, int to) throws IOException { + assert currentRequest == null; + + // Reset the fields which are accessed during document line parsing + currentType = null; + currentPipeline = defaultPipeline; + currentListExecutedPipelines = defaultListExecutedPipelines != null && defaultListExecutedPipelines; + currentFetchSourceContext = defaultFetchSourceContext; + + try (XContentParser parser = createParser(xContentType.xContent(), data, from, to)) { // Move to START_OBJECT XContentParser.Token token = parser.nextToken(); if (token == null) { - continue; + return false; } if (token != XContentParser.Token.START_OBJECT) { throw new IllegalArgumentException( @@ -242,20 +346,16 @@ public int incrementalParse( } String index = defaultIndex; - String type = null; String id = null; String routing = defaultRouting; - FetchSourceContext fetchSourceContext = defaultFetchSourceContext; String opType = null; long version = Versions.MATCH_ANY; VersionType versionType = VersionType.INTERNAL; long ifSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO; long ifPrimaryTerm = UNASSIGNED_PRIMARY_TERM; int retryOnConflict = 0; - String pipeline = defaultPipeline; boolean requireAlias = defaultRequireAlias != null && defaultRequireAlias; boolean requireDataStream = defaultRequireDataStream != null && defaultRequireDataStream; - boolean listExecutedPipelines = defaultListExecutedPipelines != null && defaultListExecutedPipelines; Map dynamicTemplates = Map.of(); // at this stage, next token can either be END_OBJECT (and use default index and type, with auto generated id) @@ -279,7 +379,7 @@ public int incrementalParse( "Action/metadata line [" + line + "] contains an unknown parameter [" + currentFieldName + "]" ); } - type = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); + currentType = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); } else if (ID.match(currentFieldName, parser.getDeprecationHandler())) { id = parser.text(); } else if (ROUTING.match(currentFieldName, parser.getDeprecationHandler())) { @@ -297,15 +397,15 @@ public int incrementalParse( } else if (RETRY_ON_CONFLICT.match(currentFieldName, parser.getDeprecationHandler())) { retryOnConflict = parser.intValue(); } else if (PIPELINE.match(currentFieldName, parser.getDeprecationHandler())) { - pipeline = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); + currentPipeline = stringDeduplicator.computeIfAbsent(parser.text(), Function.identity()); } else if (SOURCE.match(currentFieldName, parser.getDeprecationHandler())) { - fetchSourceContext = FetchSourceContext.fromXContent(parser); + currentFetchSourceContext = FetchSourceContext.fromXContent(parser); } else if (REQUIRE_ALIAS.match(currentFieldName, parser.getDeprecationHandler())) { requireAlias = parser.booleanValue(); } else if (REQUIRE_DATA_STREAM.match(currentFieldName, parser.getDeprecationHandler())) { requireDataStream = parser.booleanValue(); } else if (LIST_EXECUTED_PIPELINES.match(currentFieldName, parser.getDeprecationHandler())) { - listExecutedPipelines = parser.booleanValue(); + currentListExecutedPipelines = parser.booleanValue(); } else { throw new IllegalArgumentException( "Action/metadata line [" + line + "] contains an unknown parameter [" + currentFieldName + "]" @@ -326,7 +426,7 @@ public int incrementalParse( dynamicTemplates = parser.mapStrings(); } else if (token == XContentParser.Token.START_OBJECT && SOURCE.match(currentFieldName, parser.getDeprecationHandler())) { - fetchSourceContext = FetchSourceContext.fromXContent(parser); + currentFetchSourceContext = FetchSourceContext.fromXContent(parser); } else if (token != XContentParser.Token.VALUE_NULL) { throw new IllegalArgumentException( "Malformed action/metadata line [" @@ -360,22 +460,13 @@ public int incrementalParse( "Delete request in line [" + line + "] does not accept " + DYNAMIC_TEMPLATES.getPreferredName() ); } - deleteRequestConsumer.accept( - new DeleteRequest(index).id(id) - .routing(routing) - .version(version) - .versionType(versionType) - .setIfSeqNo(ifSeqNo) - .setIfPrimaryTerm(ifPrimaryTerm) - ); - consumed = from; + currentRequest = new DeleteRequest(index).id(id) + .routing(routing) + .version(version) + .versionType(versionType) + .setIfSeqNo(ifSeqNo) + .setIfPrimaryTerm(ifPrimaryTerm); } else { - nextMarker = findNextMarker(marker, from, data, isIncremental); - if (nextMarker == -1) { - break; - } - line++; - // we use internalAdd so we don't fork here, this allows us not to copy over the big byte array to small chunks // of index request. if ("index".equals(action) || "create".equals(action)) { @@ -383,20 +474,19 @@ public int incrementalParse( .routing(routing) .version(version) .versionType(versionType) - .setPipeline(pipeline) + .setPipeline(currentPipeline) .setIfSeqNo(ifSeqNo) .setIfPrimaryTerm(ifPrimaryTerm) - .source(sliceTrimmingCarriageReturn(data, from, nextMarker, xContentType), xContentType) .setDynamicTemplates(dynamicTemplates) .setRequireAlias(requireAlias) .setRequireDataStream(requireDataStream) - .setListExecutedPipelines(listExecutedPipelines); + .setListExecutedPipelines(currentListExecutedPipelines); if ("create".equals(action)) { indexRequest = indexRequest.create(true); } else if (opType != null) { indexRequest = indexRequest.create("create".equals(opType)); } - indexRequestConsumer.accept(indexRequest, type); + currentRequest = indexRequest; } else if ("update".equals(action)) { if (version != Versions.MATCH_ANY || versionType != VersionType.INTERNAL) { throw new IllegalArgumentException( @@ -423,31 +513,38 @@ public int incrementalParse( .setIfPrimaryTerm(ifPrimaryTerm) .setRequireAlias(requireAlias) .routing(routing); - try ( - XContentParser sliceParser = createParser( - xContent, - sliceTrimmingCarriageReturn(data, from, nextMarker, xContentType) - ) - ) { - updateRequest.fromXContent(sliceParser); - } - if (fetchSourceContext != null) { - updateRequest.fetchSource(fetchSourceContext); - } - IndexRequest upsertRequest = updateRequest.upsertRequest(); - if (upsertRequest != null) { - upsertRequest.setPipeline(pipeline).setListExecutedPipelines(listExecutedPipelines); - } - - updateRequestConsumer.accept(updateRequest); + currentRequest = updateRequest; } - // move pointers - from = nextMarker + 1; - consumed = from; } } + return true; } - return isIncremental ? consumed : from; + + private void parseAndConsumeDocumentLine(BytesReference data, int from, int to) throws IOException { + assert currentRequest != null && currentRequest instanceof DeleteRequest == false; + if (currentRequest instanceof IndexRequest indexRequest) { + indexRequest.source(sliceTrimmingCarriageReturn(data, from, to, xContentType), xContentType); + indexRequestConsumer.accept(indexRequest, currentType); + } else if (currentRequest instanceof UpdateRequest updateRequest) { + try ( + XContentParser sliceParser = createParser( + xContentType.xContent(), + sliceTrimmingCarriageReturn(data, from, to, xContentType) + ) + ) { + updateRequest.fromXContent(sliceParser); + } + if (currentFetchSourceContext != null) { + updateRequest.fetchSource(currentFetchSourceContext); + } + IndexRequest upsertRequest = updateRequest.upsertRequest(); + if (upsertRequest != null) { + upsertRequest.setPipeline(currentPipeline).setListExecutedPipelines(currentListExecutedPipelines); + } + updateRequestConsumer.accept(updateRequest); + } + } + } @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) // Remove lenient parsing in V8 BWC mode diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index 7b82481d3d283..de3fd390ec86d 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -38,9 +38,7 @@ import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.function.Supplier; @@ -142,19 +140,10 @@ private static Exception parseFailureException(Exception e) { static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { - private final boolean allowExplicitIndex; private final RestRequest request; - private final Map stringDeduplicator = new HashMap<>(); - private final String defaultIndex; - private final String defaultRouting; - private final FetchSourceContext defaultFetchSourceContext; - private final String defaultPipeline; - private final boolean defaultListExecutedPipelines; - private final Boolean defaultRequireAlias; - private final boolean defaultRequireDataStream; - private final BulkRequestParser parser; private final Supplier handlerSupplier; + private final BulkRequestParser.IncrementalParser parser; private IncrementalBulkService.Handler handler; private volatile RestChannel restChannel; @@ -164,17 +153,22 @@ static class ChunkHandler implements BaseRestHandler.RequestBodyChunkConsumer { private final ArrayList> items = new ArrayList<>(4); ChunkHandler(boolean allowExplicitIndex, RestRequest request, Supplier handlerSupplier) { - this.allowExplicitIndex = allowExplicitIndex; this.request = request; - this.defaultIndex = request.param("index"); - this.defaultRouting = request.param("routing"); - this.defaultFetchSourceContext = FetchSourceContext.parseFromRestRequest(request); - this.defaultPipeline = request.param("pipeline"); - this.defaultListExecutedPipelines = request.paramAsBoolean("list_executed_pipelines", false); - this.defaultRequireAlias = request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, false); - this.defaultRequireDataStream = request.paramAsBoolean(DocWriteRequest.REQUIRE_DATA_STREAM, false); - this.parser = new BulkRequestParser(true, request.getRestApiVersion()); this.handlerSupplier = handlerSupplier; + this.parser = new BulkRequestParser(true, request.getRestApiVersion()).incrementalParser( + request.param("index"), + request.param("routing"), + FetchSourceContext.parseFromRestRequest(request), + request.param("pipeline"), + request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, false), + request.paramAsBoolean(DocWriteRequest.REQUIRE_DATA_STREAM, false), + request.paramAsBoolean("list_executed_pipelines", false), + allowExplicitIndex, + request.getXContentType(), + (indexRequest, type) -> items.add(indexRequest), + items::add, + items::add + ); } @Override @@ -210,23 +204,7 @@ public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boo // TODO: Check that the behavior here vs. globalRouting, globalPipeline, globalRequireAlias, globalRequireDatsStream in // BulkRequest#add is fine - bytesConsumed = parser.incrementalParse( - data, - defaultIndex, - defaultRouting, - defaultFetchSourceContext, - defaultPipeline, - defaultRequireAlias, - defaultRequireDataStream, - defaultListExecutedPipelines, - allowExplicitIndex, - request.getXContentType(), - (request, type) -> items.add(request), - items::add, - items::add, - isLast == false, - stringDeduplicator - ); + bytesConsumed = parser.parse(data, isLast); bytesParsed += bytesConsumed; } catch (Exception e) { @@ -253,7 +231,7 @@ public void handleChunk(RestChannel channel, ReleasableBytesReference chunk, boo items.clear(); handler.addItems(toPass, () -> Releasables.close(releasables), () -> request.contentStream().next()); } else { - assert releasables.isEmpty(); + Releasables.close(releasables); request.contentStream().next(); } } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestParserTests.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestParserTests.java index b7f7a02e3b07e..5785d076693e7 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestParserTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestParserTests.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.bulk; +import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.core.RestApiVersion; @@ -23,6 +24,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Stream; +import static org.hamcrest.Matchers.equalTo; + public class BulkRequestParserTests extends ESTestCase { @UpdateForV10(owner = UpdateForV10.Owner.DATA_MANAGEMENT) // Replace with just RestApiVersion.values() when V8 no longer exists @@ -30,6 +33,84 @@ public class BulkRequestParserTests extends ESTestCase { .filter(v -> v.compareTo(RestApiVersion.V_8) > 0) .toList(); + public void testParserCannotBeReusedAfterFailure() { + BytesArray request = new BytesArray(""" + { "index":{ }, "something": "unexpected" } + {} + """); + + BulkRequestParser parser = new BulkRequestParser(randomBoolean(), RestApiVersion.current()); + BulkRequestParser.IncrementalParser incrementalParser = parser.incrementalParser( + null, + null, + null, + null, + null, + null, + null, + false, + XContentType.JSON, + (req, type) -> fail("expected failure before we got this far"), + req -> fail("expected failure before we got this far"), + req -> fail("expected failure before we got this far") + ); + + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> incrementalParser.parse(request, false)); + assertEquals("Malformed action/metadata line [1], expected END_OBJECT but found [FIELD_NAME]", ex.getMessage()); + + BytesArray valid = new BytesArray(""" + { "index":{ "_id": "bar" } } + {} + """); + expectThrows(AssertionError.class, () -> incrementalParser.parse(valid, false)); + } + + public void testIncrementalParsing() throws IOException { + ArrayList> indexRequests = new ArrayList<>(); + ArrayList> updateRequests = new ArrayList<>(); + ArrayList> deleteRequests = new ArrayList<>(); + + BulkRequestParser parser = new BulkRequestParser(randomBoolean(), RestApiVersion.current()); + BulkRequestParser.IncrementalParser incrementalParser = parser.incrementalParser( + null, + null, + null, + null, + null, + null, + null, + false, + XContentType.JSON, + (r, t) -> indexRequests.add(r), + updateRequests::add, + deleteRequests::add + ); + + BytesArray request = new BytesArray(""" + { "index":{ "_id": "bar", "pipeline": "foo" } } + { "field": "value"} + { "index":{ "require_alias": false } } + { "field": "value" } + { "update":{ "_id": "bus", "require_alias": true } } + { "doc": {"field": "value" }} + { "delete":{ "_id": "baz" } } + { "index": { } } + { "field": "value"} + { "delete":{ "_id": "bop" } } + """); + + int consumed = 0; + for (int i = 0; i < request.length() - 1; ++i) { + consumed += incrementalParser.parse(request.slice(consumed, i - consumed + 1), false); + } + consumed += incrementalParser.parse(request.slice(consumed, request.length() - consumed), true); + assertThat(consumed, equalTo(request.length())); + + assertThat(indexRequests.size(), equalTo(3)); + assertThat(updateRequests.size(), equalTo(1)); + assertThat(deleteRequests.size(), equalTo(2)); + } + public void testIndexRequest() throws IOException { BytesArray request = new BytesArray(""" { "index":{ "_id": "bar" } } @@ -133,7 +214,7 @@ public void testUpdateRequest() throws IOException { }, req -> fail()); } - public void testBarfOnLackOfTrailingNewline() { + public void testBarfOnLackOfTrailingNewline() throws IOException { BytesArray request = new BytesArray(""" { "index":{ "_id": "bar" } } {}"""); @@ -157,6 +238,27 @@ public void testBarfOnLackOfTrailingNewline() { ) ); assertEquals("The bulk request must be terminated by a newline [\\n]", e.getMessage()); + + BulkRequestParser.IncrementalParser incrementalParser = parser.incrementalParser( + "foo", + null, + null, + null, + null, + null, + null, + false, + XContentType.JSON, + (req, type) -> {}, + req -> {}, + req -> {} + ); + + // Should not throw because not last + incrementalParser.parse(request, false); + + IllegalArgumentException e2 = expectThrows(IllegalArgumentException.class, () -> incrementalParser.parse(request, true)); + assertEquals("The bulk request must be terminated by a newline [\\n]", e2.getMessage()); } public void testFailOnExplicitIndex() { diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java index 032db4135aab7..b36b3af1ddb86 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java @@ -406,7 +406,7 @@ public void testInvalidDynamicTemplates() { IllegalArgumentException.class, () -> new BulkRequest().add(updateWithDynamicTemplates, null, XContentType.JSON) ); - assertThat(error.getMessage(), equalTo("Update request in line [2] does not accept dynamic_templates")); + assertThat(error.getMessage(), equalTo("Update request in line [1] does not accept dynamic_templates")); BytesArray invalidDynamicTemplates = new BytesArray(""" { "index":{"_index":"test","dynamic_templates":[]} diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java index 25cfd1e56514c..3b6b280565da5 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java @@ -251,6 +251,7 @@ public void lastItems(List> items, Releasable releasable, Act assertTrue(next.get()); next.set(false); assertFalse(isLast.get()); + assertFalse(r1.hasReferences()); ReleasableBytesReference r2 = new ReleasableBytesReference(new BytesArray("{\"field\":1}"), () -> {}); chunkHandler.handleChunk(channel, r2, false); @@ -258,7 +259,6 @@ public void lastItems(List> items, Releasable releasable, Act assertTrue(next.get()); next.set(false); assertFalse(isLast.get()); - assertTrue(r1.hasReferences()); assertTrue(r2.hasReferences()); ReleasableBytesReference r3 = new ReleasableBytesReference(new BytesArray("\n{\"delete\":"), () -> {}); From 583cf23e977db595d9c9dfcc775e12ff8c42ce6f Mon Sep 17 00:00:00 2001 From: Pat Whelan Date: Mon, 4 Nov 2024 17:38:44 -0500 Subject: [PATCH 23/35] [ML] Handle empty percentile results (#116015) Do not add the leaf node if the percentiles bucket is empty. Fix #116013 --- docs/changelog/116015.yaml | 6 ++++++ .../aggregation/AggregationToJsonProcessor.java | 5 ++--- .../aggregation/AggregationToJsonProcessorTests.java | 9 +++++++++ 3 files changed, 17 insertions(+), 3 deletions(-) create mode 100644 docs/changelog/116015.yaml diff --git a/docs/changelog/116015.yaml b/docs/changelog/116015.yaml new file mode 100644 index 0000000000000..693fad639f2fa --- /dev/null +++ b/docs/changelog/116015.yaml @@ -0,0 +1,6 @@ +pr: 116015 +summary: Empty percentile results no longer throw no_such_element_exception in Anomaly Detection jobs +area: Machine Learning +type: bug +issues: + - 116013 diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationToJsonProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationToJsonProcessor.java index 5c9711a6e5d8b..cc997b3804e6b 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationToJsonProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationToJsonProcessor.java @@ -20,7 +20,6 @@ import org.elasticsearch.search.aggregations.metrics.GeoCentroid; import org.elasticsearch.search.aggregations.metrics.Max; import org.elasticsearch.search.aggregations.metrics.NumericMetricsAggregation; -import org.elasticsearch.search.aggregations.metrics.Percentile; import org.elasticsearch.search.aggregations.metrics.Percentiles; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.json.JsonXContent; @@ -408,8 +407,8 @@ private boolean processGeoCentroid(GeoCentroid agg) { } private boolean processPercentiles(Percentiles percentiles) { - Iterator percentileIterator = percentiles.iterator(); - boolean aggregationAdded = addMetricIfFinite(percentiles.getName(), percentileIterator.next().value()); + var percentileIterator = percentiles.iterator(); + var aggregationAdded = percentileIterator.hasNext() && addMetricIfFinite(percentiles.getName(), percentileIterator.next().value()); if (percentileIterator.hasNext()) { throw new IllegalArgumentException("Multi-percentile aggregation [" + percentiles.getName() + "] is not supported"); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationToJsonProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationToJsonProcessorTests.java index fc774a4ee3e48..ab1dee3b91236 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationToJsonProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationToJsonProcessorTests.java @@ -505,6 +505,15 @@ public void testProcessGivenBucketAndLeafAggregationsButBucketNotInFields() thro {"time":4400,"my_value":4.0,"doc_count":7}""")); } + public void testProcessGivenEmptyPercentiles() throws IOException { + var histogramBuckets = List.of(createHistogramBucket(1000L, 4, List.of(createMax("time", 1000), createPercentiles("my_field")))); + + var json = aggToString(Sets.newHashSet("my_field"), histogramBuckets); + + assertThat(json, equalTo(""" + {"time":1000,"doc_count":4}""")); + } + public void testProcessGivenSinglePercentilesPerHistogram() throws IOException { List histogramBuckets = Arrays.asList( createHistogramBucket(1000L, 4, Arrays.asList(createMax("time", 1000), createPercentiles("my_field", 1.0))), From 91474289122fc610b457b2836f6daafb60e44bd7 Mon Sep 17 00:00:00 2001 From: Mikhail Berezovskiy Date: Mon, 4 Nov 2024 18:54:42 -0800 Subject: [PATCH 24/35] Use underlying ByteBuf refCount for ReleasableBytesReference (#116211) --- docs/changelog/116211.yaml | 5 ++ .../netty4/Netty4MessageInboundHandler.java | 36 +-------------- .../transport/netty4/Netty4Utils.java | 46 ++++++++++++++++++- .../transport/netty4/Netty4UtilsTests.java | 38 +++++++++++++++ 4 files changed, 89 insertions(+), 36 deletions(-) create mode 100644 docs/changelog/116211.yaml diff --git a/docs/changelog/116211.yaml b/docs/changelog/116211.yaml new file mode 100644 index 0000000000000..6f55b1b2fef34 --- /dev/null +++ b/docs/changelog/116211.yaml @@ -0,0 +1,5 @@ +pr: 116211 +summary: Use underlying `ByteBuf` `refCount` for `ReleasableBytesReference` +area: Network +type: bug +issues: [] diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4MessageInboundHandler.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4MessageInboundHandler.java index 3bec37c0997db..8fdb7051e2be6 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4MessageInboundHandler.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4MessageInboundHandler.java @@ -14,10 +14,8 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.common.network.ThreadWatchdog; -import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasables; import org.elasticsearch.transport.InboundPipeline; import org.elasticsearch.transport.Transports; @@ -52,9 +50,8 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception final ByteBuf buffer = (ByteBuf) msg; Netty4TcpChannel channel = ctx.channel().attr(Netty4Transport.CHANNEL_KEY).get(); - final BytesReference wrapped = Netty4Utils.toBytesReference(buffer); activityTracker.startActivity(); - try (ReleasableBytesReference reference = new ReleasableBytesReference(wrapped, new ByteBufRefCounted(buffer))) { + try (ReleasableBytesReference reference = Netty4Utils.toReleasableBytesReference(buffer)) { pipeline.handleBytes(channel, reference); } finally { activityTracker.stopActivity(); @@ -81,35 +78,4 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { super.channelInactive(ctx); } - private record ByteBufRefCounted(ByteBuf buffer) implements RefCounted { - - @Override - public void incRef() { - buffer.retain(); - } - - @Override - public boolean tryIncRef() { - if (hasReferences() == false) { - return false; - } - try { - buffer.retain(); - } catch (RuntimeException e) { - assert hasReferences() == false; - return false; - } - return true; - } - - @Override - public boolean decRef() { - return buffer.release(); - } - - @Override - public boolean hasReferences() { - return buffer.refCnt() > 0; - } - } } diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java index f57aa0e680fa1..459b6c77be8c3 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/Netty4Utils.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.Booleans; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.http.HttpBody; import org.elasticsearch.transport.TransportException; @@ -130,8 +131,51 @@ public static BytesReference toBytesReference(final ByteBuf buffer) { } } + /** + * Wrap Netty's {@link ByteBuf} into {@link ReleasableBytesReference} and delegating reference count to ByteBuf. + */ public static ReleasableBytesReference toReleasableBytesReference(final ByteBuf buffer) { - return new ReleasableBytesReference(toBytesReference(buffer), buffer::release); + return new ReleasableBytesReference(toBytesReference(buffer), toRefCounted(buffer)); + } + + static ByteBufRefCounted toRefCounted(final ByteBuf buf) { + return new ByteBufRefCounted(buf); + } + + record ByteBufRefCounted(ByteBuf buffer) implements RefCounted { + + public int refCnt() { + return buffer.refCnt(); + } + + @Override + public void incRef() { + buffer.retain(); + } + + @Override + public boolean tryIncRef() { + if (hasReferences() == false) { + return false; + } + try { + buffer.retain(); + } catch (RuntimeException e) { + assert hasReferences() == false; + return false; + } + return true; + } + + @Override + public boolean decRef() { + return buffer.release(); + } + + @Override + public boolean hasReferences() { + return buffer.refCnt() > 0; + } } public static HttpBody.Full fullHttpBodyFrom(final ByteBuf buf) { diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4UtilsTests.java b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4UtilsTests.java index 5676ef6dfc5ee..3844953628777 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4UtilsTests.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/transport/netty4/Netty4UtilsTests.java @@ -11,6 +11,7 @@ import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.PooledByteBufAllocator; import io.netty.buffer.Unpooled; import org.apache.lucene.util.BytesRef; @@ -68,6 +69,43 @@ public void testToChannelBuffer() throws IOException { assertArrayEquals(BytesReference.toBytes(ref), BytesReference.toBytes(bytesReference)); } + /** + * Test that wrapped reference counted object from netty reflects correct counts in ES RefCounted + */ + public void testToRefCounted() { + var buf = PooledByteBufAllocator.DEFAULT.buffer(1); + assertEquals(1, buf.refCnt()); + + var refCounted = Netty4Utils.toRefCounted(buf); + assertEquals(1, refCounted.refCnt()); + + buf.retain(); + assertEquals(2, refCounted.refCnt()); + + refCounted.incRef(); + assertEquals(3, refCounted.refCnt()); + assertEquals(buf.refCnt(), refCounted.refCnt()); + + refCounted.decRef(); + assertEquals(2, refCounted.refCnt()); + assertEquals(buf.refCnt(), refCounted.refCnt()); + assertTrue(refCounted.hasReferences()); + + refCounted.decRef(); + refCounted.decRef(); + assertFalse(refCounted.hasReferences()); + } + + /** + * Ensures that released ByteBuf cannot be accessed from ReleasableBytesReference + */ + public void testToReleasableBytesReferenceThrowOnByteBufRelease() { + var buf = PooledByteBufAllocator.DEFAULT.buffer(1); + var relBytes = Netty4Utils.toReleasableBytesReference(buf); + buf.release(); + assertThrows(AssertionError.class, () -> relBytes.get(0)); + } + private BytesReference getRandomizedBytesReference(int length) throws IOException { // we know bytes stream output always creates a paged bytes reference, we use it to create randomized content ReleasableBytesStreamOutput out = new ReleasableBytesStreamOutput(length, bigarrays); From 000901a45ef7941631be146f2627c2e14e20f91f Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Tue, 5 Nov 2024 16:55:42 +1100 Subject: [PATCH 25/35] Mute org.elasticsearch.indexing.IndexActionIT testAutoGenerateIdNoDuplicates #115716 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 534a25a402b2c..911187706e8fb 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -279,6 +279,9 @@ tests: - class: org.elasticsearch.search.basic.SearchWithRandomDisconnectsIT method: testSearchWithRandomDisconnects issue: https://github.com/elastic/elasticsearch/issues/116175 +- class: org.elasticsearch.indexing.IndexActionIT + method: testAutoGenerateIdNoDuplicates + issue: https://github.com/elastic/elasticsearch/issues/115716 # Examples: # From 8afe2f4f58a8fe709a192b69011dc447c7454658 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Tue, 5 Nov 2024 17:10:44 +1100 Subject: [PATCH 26/35] Mute org.elasticsearch.xpack.test.rest.XPackRestIT test {p0=ml/start_stop_datafeed/Test start datafeed given index pattern with no matching indices} #116220 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 911187706e8fb..802381d1b34f9 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -282,6 +282,9 @@ tests: - class: org.elasticsearch.indexing.IndexActionIT method: testAutoGenerateIdNoDuplicates issue: https://github.com/elastic/elasticsearch/issues/115716 +- class: org.elasticsearch.xpack.test.rest.XPackRestIT + method: test {p0=ml/start_stop_datafeed/Test start datafeed given index pattern with no matching indices} + issue: https://github.com/elastic/elasticsearch/issues/116220 # Examples: # From b13313f7e71fd0709e9a36da6a9fe86227285fbc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Aur=C3=A9lien=20FOUCRET?= Date: Tue, 5 Nov 2024 09:37:28 +0100 Subject: [PATCH 27/35] KQL plugin - Code cleanup (#116180) --- x-pack/plugin/kql/build.gradle | 2 +- .../kql/licenses/antlr4-runtime-LICENSE.txt | 26 +++++++++++++++++++ .../kql/licenses/antlr4-runtime-NOTICE.txt | 0 .../plugin/kql/src/main/java/module-info.java | 19 ++++++++++++++ .../xpack/kql/parser/ParserUtils.java | 10 +++---- 5 files changed, 51 insertions(+), 6 deletions(-) create mode 100644 x-pack/plugin/kql/licenses/antlr4-runtime-LICENSE.txt create mode 100644 x-pack/plugin/kql/licenses/antlr4-runtime-NOTICE.txt create mode 100644 x-pack/plugin/kql/src/main/java/module-info.java diff --git a/x-pack/plugin/kql/build.gradle b/x-pack/plugin/kql/build.gradle index 198099329c7c0..9d0860346b188 100644 --- a/x-pack/plugin/kql/build.gradle +++ b/x-pack/plugin/kql/build.gradle @@ -17,7 +17,7 @@ base { dependencies { compileOnly project(path: xpackModule('core')) - compileOnly "org.antlr:antlr4-runtime:${versions.antlr4}" + api "org.antlr:antlr4-runtime:${versions.antlr4}" testImplementation "org.antlr:antlr4-runtime:${versions.antlr4}" testImplementation project(':test:framework') diff --git a/x-pack/plugin/kql/licenses/antlr4-runtime-LICENSE.txt b/x-pack/plugin/kql/licenses/antlr4-runtime-LICENSE.txt new file mode 100644 index 0000000000000..95d0a2554f686 --- /dev/null +++ b/x-pack/plugin/kql/licenses/antlr4-runtime-LICENSE.txt @@ -0,0 +1,26 @@ +[The "BSD license"] +Copyright (c) 2015 Terence Parr, Sam Harwell +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions +are met: + + 1. Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + 2. Redistributions in binary form must reproduce the above copyright + notice, this list of conditions and the following disclaimer in the + documentation and/or other materials provided with the distribution. + 3. The name of the author may not be used to endorse or promote products + derived from this software without specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE AUTHOR ``AS IS'' AND ANY EXPRESS OR +IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES +OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE DISCLAIMED. +IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY DIRECT, INDIRECT, +INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT +NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF +THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/x-pack/plugin/kql/licenses/antlr4-runtime-NOTICE.txt b/x-pack/plugin/kql/licenses/antlr4-runtime-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/x-pack/plugin/kql/src/main/java/module-info.java b/x-pack/plugin/kql/src/main/java/module-info.java new file mode 100644 index 0000000000000..c4dd539508f39 --- /dev/null +++ b/x-pack/plugin/kql/src/main/java/module-info.java @@ -0,0 +1,19 @@ +/* + * 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. + */ + +module org.elasticsearch.kql { + requires org.elasticsearch.server; + requires org.elasticsearch.xcontent; + requires org.antlr.antlr4.runtime; + requires org.elasticsearch.base; + requires org.apache.lucene.queryparser; + requires org.elasticsearch.logging; + requires org.apache.lucene.core; + + exports org.elasticsearch.xpack.kql; + exports org.elasticsearch.xpack.kql.parser; +} diff --git a/x-pack/plugin/kql/src/main/java/org/elasticsearch/xpack/kql/parser/ParserUtils.java b/x-pack/plugin/kql/src/main/java/org/elasticsearch/xpack/kql/parser/ParserUtils.java index f996a953ea7f7..3319d920a88ee 100644 --- a/x-pack/plugin/kql/src/main/java/org/elasticsearch/xpack/kql/parser/ParserUtils.java +++ b/x-pack/plugin/kql/src/main/java/org/elasticsearch/xpack/kql/parser/ParserUtils.java @@ -12,11 +12,11 @@ import org.antlr.v4.runtime.tree.ParseTree; import org.antlr.v4.runtime.tree.ParseTreeVisitor; import org.antlr.v4.runtime.tree.TerminalNode; -import org.apache.logging.log4j.util.Strings; import org.apache.lucene.queryparser.classic.QueryParser; import java.util.ArrayList; import java.util.List; +import java.util.Locale; /** * Utility class for parsing and processing KQL expressions. @@ -211,15 +211,15 @@ private static boolean isEscapedKeywordSequence(String input, int startIndex) { if (startIndex + 1 >= input.length()) { return false; } - String remaining = Strings.toRootLowerCase(input.substring(startIndex)); + String remaining = input.substring(startIndex).toLowerCase(Locale.ROOT); return remaining.startsWith("and") || remaining.startsWith("or") || remaining.startsWith("not"); } private static String handleKeywordSequence(String input, int startIndex) { String remaining = input.substring(startIndex); - if (Strings.toRootLowerCase(remaining).startsWith("and")) return remaining.substring(0, 3); - if (Strings.toRootLowerCase(remaining).startsWith("or")) return remaining.substring(0, 2); - if (Strings.toRootLowerCase(remaining).startsWith("not")) return remaining.substring(0, 3); + if (remaining.toLowerCase(Locale.ROOT).startsWith("and")) return remaining.substring(0, 3); + if (remaining.toLowerCase(Locale.ROOT).startsWith("or")) return remaining.substring(0, 2); + if (remaining.toLowerCase(Locale.ROOT).startsWith("not")) return remaining.substring(0, 3); return ""; } From 0c4209b5efbfd26a243711b02c51cd04487bffda Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 5 Nov 2024 09:27:56 +0000 Subject: [PATCH 28/35] Handle status code 0 in S3 CMU response (#116212) A `CompleteMultipartUpload` action may fail after sending the `200 OK` response line. In this case the response body describes the error, and the SDK translates this situation to an exception with status code 0 but with the `ErrorCode` string set appropriately. This commit enhances the exception handling in `S3BlobContainer` to handle this possibility. Closes #102294 Co-authored-by: Pat Patterson --- docs/changelog/116212.yaml | 6 ++++++ .../repositories/s3/S3BlobContainer.java | 9 +++++++-- .../src/main/java/fixture/s3/S3HttpHandler.java | 17 ++++++++++++++++- .../analyze/S3RepositoryAnalysisRestIT.java | 7 ------- 4 files changed, 29 insertions(+), 10 deletions(-) create mode 100644 docs/changelog/116212.yaml diff --git a/docs/changelog/116212.yaml b/docs/changelog/116212.yaml new file mode 100644 index 0000000000000..7c8756f4054cd --- /dev/null +++ b/docs/changelog/116212.yaml @@ -0,0 +1,6 @@ +pr: 116212 +summary: Handle status code 0 in S3 CMU response +area: Snapshot/Restore +type: bug +issues: + - 102294 diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java index 9757d3af861a9..e13cc40dd3e0f 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3BlobContainer.java @@ -897,8 +897,13 @@ public void compareAndExchangeRegister( final var clientReference = blobStore.clientReference(); ActionListener.run(ActionListener.releaseAfter(listener.delegateResponse((delegate, e) -> { logger.trace(() -> Strings.format("[%s]: compareAndExchangeRegister failed", key), e); - if (e instanceof AmazonS3Exception amazonS3Exception && amazonS3Exception.getStatusCode() == 404) { - // an uncaught 404 means that our multipart upload was aborted by a concurrent operation before we could complete it + if (e instanceof AmazonS3Exception amazonS3Exception + && (amazonS3Exception.getStatusCode() == 404 + || amazonS3Exception.getStatusCode() == 0 && "NoSuchUpload".equals(amazonS3Exception.getErrorCode()))) { + // An uncaught 404 means that our multipart upload was aborted by a concurrent operation before we could complete it. + // Also (rarely) S3 can start processing the request during a concurrent abort and this can result in a 200 OK with an + // NoSuchUpload... in the response, which the SDK translates to status code 0. Either way, this means + // that our write encountered contention: delegate.onResponse(OptionalBytesReference.MISSING); } else { delegate.onFailure(e); diff --git a/test/fixtures/s3-fixture/src/main/java/fixture/s3/S3HttpHandler.java b/test/fixtures/s3-fixture/src/main/java/fixture/s3/S3HttpHandler.java index eddce6aae298a..56d3454aa5544 100644 --- a/test/fixtures/s3-fixture/src/main/java/fixture/s3/S3HttpHandler.java +++ b/test/fixtures/s3-fixture/src/main/java/fixture/s3/S3HttpHandler.java @@ -13,6 +13,7 @@ import com.sun.net.httpserver.HttpHandler; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.Randomness; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesReference; @@ -168,7 +169,21 @@ public void handle(final HttpExchange exchange) throws IOException { RestUtils.decodeQueryString(request, request.indexOf('?') + 1, params); final var upload = uploads.remove(params.get("uploadId")); if (upload == null) { - exchange.sendResponseHeaders(RestStatus.NOT_FOUND.getStatus(), -1); + if (Randomness.get().nextBoolean()) { + exchange.sendResponseHeaders(RestStatus.NOT_FOUND.getStatus(), -1); + } else { + byte[] response = (""" + + + NoSuchUpload + No such upload + test-request-id + test-host-id + """).getBytes(StandardCharsets.UTF_8); + exchange.getResponseHeaders().add("Content-Type", "application/xml"); + exchange.sendResponseHeaders(RestStatus.OK.getStatus(), response.length); + exchange.getResponseBody().write(response); + } } else { final var blobContents = upload.complete(extractPartEtags(Streams.readFully(exchange.getRequestBody()))); blobs.put(requestComponents.path, blobContents); diff --git a/x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/S3RepositoryAnalysisRestIT.java b/x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/S3RepositoryAnalysisRestIT.java index 8986cf1059191..c0f2b40f5a10f 100644 --- a/x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/S3RepositoryAnalysisRestIT.java +++ b/x-pack/plugin/snapshot-repo-test-kit/qa/s3/src/javaRestTest/java/org/elasticsearch/repositories/blobstore/testkit/analyze/S3RepositoryAnalysisRestIT.java @@ -31,13 +31,6 @@ public class S3RepositoryAnalysisRestIT extends AbstractRepositoryAnalysisRestTe .setting("s3.client.repo_test_kit.protocol", () -> "http", (n) -> USE_FIXTURE) .setting("s3.client.repo_test_kit.endpoint", s3Fixture::getAddress, (n) -> USE_FIXTURE) .setting("xpack.security.enabled", "false") - // Additional tracing related to investigation into https://github.com/elastic/elasticsearch/issues/102294 - .setting("logger.org.elasticsearch.repositories.s3", "TRACE") - .setting("logger.org.elasticsearch.repositories.blobstore.testkit", "TRACE") - .setting("logger.com.amazonaws.request", "DEBUG") - .setting("logger.org.apache.http.wire", "DEBUG") - // Necessary to permit setting the above two restricted loggers to DEBUG - .jvmArg("-Des.insecure_network_trace_enabled=true") .build(); @ClassRule From 577f324aee985dae2d022dd8659362aafdb308dc Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 5 Nov 2024 10:32:14 +0100 Subject: [PATCH 29/35] Only create MapperService in SyntheticSourceIndexSettingsProvider when required (#116075) In case the index.mapping.source.mode is specified, there is no need to create a mapper service to determine whether synthetic source is used. In case of logsdb/tsdb there is also no reason to create a mapper service. If _source.mode attribute is specified, then it doesn't really matter whether what its value is for the SyntheticSourceIndexSettingsProvider. If it is synthetic, then that is the same as the index mode's default source mode. If it is stored, we just will add set index.mapping.source.mode to stored, which has no effect. And disabled source mode isn't allowed in the case of logsdb and tsdb. Closes #116070 --- .../SyntheticSourceIndexSettingsProvider.java | 17 +++++++++++- ...heticSourceIndexSettingsProviderTests.java | 27 +++++++++++++++---- 2 files changed, 38 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProvider.java b/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProvider.java index e7572d6a646e1..e87f10ec19916 100644 --- a/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProvider.java +++ b/x-pack/plugin/logsdb/src/main/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProvider.java @@ -101,6 +101,20 @@ boolean newIndexHasSyntheticSourceUsage( try { var tmpIndexMetadata = buildIndexMetadataForMapperService(indexName, templateIndexMode, indexTemplateAndCreateRequestSettings); + var indexMode = tmpIndexMetadata.getIndexMode(); + if (SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.exists(tmpIndexMetadata.getSettings()) + || indexMode == IndexMode.LOGSDB + || indexMode == IndexMode.TIME_SERIES) { + // In case when index mode is tsdb or logsdb and only _source.mode mapping attribute is specified, then the default + // could be wrong. However, it doesn't really matter, because if the _source.mode mapping attribute is set to stored, + // then configuring the index.mapping.source.mode setting to stored has no effect. Additionally _source.mode can't be set + // to disabled, because that isn't allowed with logsdb/tsdb. In other words setting index.mapping.source.mode setting to + // stored when _source.mode mapping attribute is stored is fine as it has no effect, but avoids creating MapperService. + var sourceMode = SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.get(tmpIndexMetadata.getSettings()); + return sourceMode == SourceFieldMapper.Mode.SYNTHETIC; + } + + // TODO: remove this when _source.mode attribute has been removed: try (var mapperService = mapperServiceFactory.apply(tmpIndexMetadata)) { // combinedTemplateMappings can be null when creating system indices // combinedTemplateMappings can be empty when creating a normal index that doesn't match any template and without mapping. @@ -112,7 +126,8 @@ boolean newIndexHasSyntheticSourceUsage( } } catch (AssertionError | Exception e) { // In case invalid mappings or setting are provided, then mapper service creation can fail. - // In that case it is ok to return false here. The index creation will fail anyway later, so need to fallback to stored source. + // In that case it is ok to return false here. The index creation will fail anyway later, so no need to fallback to stored + // source. LOGGER.info(() -> Strings.format("unable to create mapper service for index [%s]", indexName), e); return false; } diff --git a/x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProviderTests.java b/x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProviderTests.java index 2ab77b38b3373..2d8723a0d8c25 100644 --- a/x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProviderTests.java +++ b/x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceIndexSettingsProviderTests.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.time.Instant; import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; import static org.elasticsearch.common.settings.Settings.builder; import static org.hamcrest.Matchers.equalTo; @@ -35,6 +36,7 @@ public class SyntheticSourceIndexSettingsProviderTests extends ESTestCase { private SyntheticSourceLicenseService syntheticSourceLicenseService; private SyntheticSourceIndexSettingsProvider provider; + private final AtomicInteger newMapperServiceCounter = new AtomicInteger(); private static LogsdbIndexModeSettingsProvider getLogsdbIndexModeSettingsProvider(boolean enabled) { return new LogsdbIndexModeSettingsProvider(Settings.builder().put("cluster.logsdb.enabled", enabled).build()); @@ -49,11 +51,11 @@ public void setup() { syntheticSourceLicenseService = new SyntheticSourceLicenseService(Settings.EMPTY); syntheticSourceLicenseService.setLicenseState(licenseState); - provider = new SyntheticSourceIndexSettingsProvider( - syntheticSourceLicenseService, - im -> MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), im.getSettings(), im.getIndex().getName()), - getLogsdbIndexModeSettingsProvider(false) - ); + provider = new SyntheticSourceIndexSettingsProvider(syntheticSourceLicenseService, im -> { + newMapperServiceCounter.incrementAndGet(); + return MapperTestUtils.newMapperService(xContentRegistry(), createTempDir(), im.getSettings(), im.getIndex().getName()); + }, getLogsdbIndexModeSettingsProvider(false)); + newMapperServiceCounter.set(0); } public void testNewIndexHasSyntheticSourceUsage() throws IOException { @@ -77,6 +79,7 @@ public void testNewIndexHasSyntheticSourceUsage() throws IOException { """; boolean result = provider.newIndexHasSyntheticSourceUsage(indexName, null, settings, List.of(new CompressedXContent(mapping))); assertTrue(result); + assertThat(newMapperServiceCounter.get(), equalTo(1)); } { String mapping; @@ -110,6 +113,7 @@ public void testNewIndexHasSyntheticSourceUsage() throws IOException { } boolean result = provider.newIndexHasSyntheticSourceUsage(indexName, null, settings, List.of(new CompressedXContent(mapping))); assertFalse(result); + assertThat(newMapperServiceCounter.get(), equalTo(2)); } } @@ -152,15 +156,18 @@ public void testNewIndexHasSyntheticSourceUsageLogsdbIndex() throws IOException Settings settings = Settings.builder().put("index.mode", "logsdb").build(); boolean result = provider.newIndexHasSyntheticSourceUsage(indexName, null, settings, List.of(new CompressedXContent(mapping))); assertTrue(result); + assertThat(newMapperServiceCounter.get(), equalTo(0)); } { Settings settings = Settings.builder().put("index.mode", "logsdb").build(); boolean result = provider.newIndexHasSyntheticSourceUsage(indexName, null, settings, List.of()); assertTrue(result); + assertThat(newMapperServiceCounter.get(), equalTo(0)); } { boolean result = provider.newIndexHasSyntheticSourceUsage(indexName, null, Settings.EMPTY, List.of()); assertFalse(result); + assertThat(newMapperServiceCounter.get(), equalTo(1)); } { boolean result = provider.newIndexHasSyntheticSourceUsage( @@ -170,6 +177,7 @@ public void testNewIndexHasSyntheticSourceUsageLogsdbIndex() throws IOException List.of(new CompressedXContent(mapping)) ); assertFalse(result); + assertThat(newMapperServiceCounter.get(), equalTo(2)); } } @@ -234,6 +242,7 @@ public void testNewIndexHasSyntheticSourceUsage_invalidSettings() throws IOExcep """; boolean result = provider.newIndexHasSyntheticSourceUsage(indexName, null, settings, List.of(new CompressedXContent(mapping))); assertFalse(result); + assertThat(newMapperServiceCounter.get(), equalTo(1)); } { String mapping = """ @@ -249,6 +258,7 @@ public void testNewIndexHasSyntheticSourceUsage_invalidSettings() throws IOExcep """; boolean result = provider.newIndexHasSyntheticSourceUsage(indexName, null, settings, List.of(new CompressedXContent(mapping))); assertFalse(result); + assertThat(newMapperServiceCounter.get(), equalTo(2)); } } @@ -278,6 +288,7 @@ public void testGetAdditionalIndexSettingsDowngradeFromSyntheticSource() throws List.of() ); assertThat(result.size(), equalTo(0)); + assertThat(newMapperServiceCounter.get(), equalTo(0)); syntheticSourceLicenseService.setSyntheticSourceFallback(true); result = provider.getAdditionalIndexSettings( @@ -291,6 +302,7 @@ public void testGetAdditionalIndexSettingsDowngradeFromSyntheticSource() throws ); assertThat(result.size(), equalTo(1)); assertEquals(SourceFieldMapper.Mode.STORED, SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.get(result)); + assertThat(newMapperServiceCounter.get(), equalTo(0)); result = provider.getAdditionalIndexSettings( DataStream.getDefaultBackingIndexName(dataStreamName, 2), @@ -303,6 +315,7 @@ public void testGetAdditionalIndexSettingsDowngradeFromSyntheticSource() throws ); assertThat(result.size(), equalTo(1)); assertEquals(SourceFieldMapper.Mode.STORED, SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.get(result)); + assertThat(newMapperServiceCounter.get(), equalTo(0)); result = provider.getAdditionalIndexSettings( DataStream.getDefaultBackingIndexName(dataStreamName, 2), @@ -315,6 +328,7 @@ public void testGetAdditionalIndexSettingsDowngradeFromSyntheticSource() throws ); assertThat(result.size(), equalTo(1)); assertEquals(SourceFieldMapper.Mode.STORED, SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.get(result)); + assertThat(newMapperServiceCounter.get(), equalTo(0)); } public void testGetAdditionalIndexSettingsDowngradeFromSyntheticSourceFileMatch() throws IOException { @@ -347,6 +361,7 @@ public void testGetAdditionalIndexSettingsDowngradeFromSyntheticSourceFileMatch( List.of() ); assertThat(result.size(), equalTo(0)); + assertThat(newMapperServiceCounter.get(), equalTo(0)); dataStreamName = "logs-app1-0"; mb = Metadata.builder( @@ -371,6 +386,7 @@ public void testGetAdditionalIndexSettingsDowngradeFromSyntheticSourceFileMatch( ); assertThat(result.size(), equalTo(1)); assertEquals(SourceFieldMapper.Mode.STORED, SourceFieldMapper.INDEX_MAPPER_SOURCE_MODE_SETTING.get(result)); + assertThat(newMapperServiceCounter.get(), equalTo(0)); result = provider.getAdditionalIndexSettings( DataStream.getDefaultBackingIndexName(dataStreamName, 2), @@ -382,5 +398,6 @@ public void testGetAdditionalIndexSettingsDowngradeFromSyntheticSourceFileMatch( List.of() ); assertThat(result.size(), equalTo(0)); + assertThat(newMapperServiceCounter.get(), equalTo(0)); } } From eef2ba43f6d4a902880a14258ef9f2fa178412a1 Mon Sep 17 00:00:00 2001 From: Ioana Tagirta Date: Tue, 5 Nov 2024 10:43:38 +0100 Subject: [PATCH 30/35] Fix release tests for semantic_text (#116202) --- .../org/elasticsearch/xpack/esql/core/type/DataType.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java index 81739536c6572..9708a3ea0db85 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/type/DataType.java @@ -370,10 +370,7 @@ public static boolean isUnsupported(DataType from) { } public static boolean isString(DataType t) { - if (EsqlCorePlugin.SEMANTIC_TEXT_FEATURE_FLAG.isEnabled() && t == SEMANTIC_TEXT) { - return true; - } - return t == KEYWORD || t == TEXT; + return t == KEYWORD || t == TEXT || t == SEMANTIC_TEXT; } public static boolean isPrimitiveAndSupported(DataType t) { From b34ca93bd8f2b558080d835c44ba96e88323c7d4 Mon Sep 17 00:00:00 2001 From: Nikolaj Volgushev Date: Tue, 5 Nov 2024 11:07:06 +0100 Subject: [PATCH 31/35] Enable DEBUG logging for `FileSettingsServiceTests` (#116192) https://github.com/elastic/elasticsearch/issues/115280 still mystifies me. It does not reproduce on a buildkite instance either. This PR turns up logging for the file watching service steps (in particular the polled events should be useful) and adds some logging to the test. --- muted-tests.yml | 3 -- .../service/FileSettingsServiceTests.java | 41 +++++++++++++++---- 2 files changed, 32 insertions(+), 12 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 802381d1b34f9..902f4789112d8 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -184,9 +184,6 @@ tests: - class: org.elasticsearch.search.StressSearchServiceReaperIT method: testStressReaper issue: https://github.com/elastic/elasticsearch/issues/115816 -- class: org.elasticsearch.reservedstate.service.FileSettingsServiceTests - method: testProcessFileChanges - issue: https://github.com/elastic/elasticsearch/issues/115280 - class: org.elasticsearch.search.SearchServiceTests method: testWaitOnRefreshTimeout issue: https://github.com/elastic/elasticsearch/issues/115935 diff --git a/server/src/test/java/org/elasticsearch/reservedstate/service/FileSettingsServiceTests.java b/server/src/test/java/org/elasticsearch/reservedstate/service/FileSettingsServiceTests.java index f67d7ddcc7550..3622285478df2 100644 --- a/server/src/test/java/org/elasticsearch/reservedstate/service/FileSettingsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/reservedstate/service/FileSettingsServiceTests.java @@ -9,6 +9,9 @@ package org.elasticsearch.reservedstate.service; +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; @@ -22,6 +25,8 @@ import org.elasticsearch.cluster.routing.RerouteService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.component.Lifecycle; +import org.elasticsearch.common.file.AbstractFileWatchingService; +import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.IOUtils; @@ -67,6 +72,7 @@ import static org.mockito.Mockito.verify; public class FileSettingsServiceTests extends ESTestCase { + private static final Logger logger = LogManager.getLogger(FileSettingsServiceTests.class); private Environment env; private ClusterService clusterService; private ReservedClusterStateService controller; @@ -76,6 +82,8 @@ public class FileSettingsServiceTests extends ESTestCase { @Before public void setUp() throws Exception { super.setUp(); + // TODO remove me once https://github.com/elastic/elasticsearch/issues/115280 is closed + Loggers.setLevel(LogManager.getLogger(AbstractFileWatchingService.class), Level.DEBUG); threadpool = new TestThreadPool("file_settings_service_tests"); @@ -120,16 +128,23 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - if (fileSettingsService.lifecycleState() == Lifecycle.State.STARTED) { - fileSettingsService.stop(); - } - if (fileSettingsService.lifecycleState() == Lifecycle.State.STOPPED) { - fileSettingsService.close(); - } + try { + if (fileSettingsService.lifecycleState() == Lifecycle.State.STARTED) { + logger.info("Stopping file settings service"); + fileSettingsService.stop(); + } + if (fileSettingsService.lifecycleState() == Lifecycle.State.STOPPED) { + logger.info("Closing file settings service"); + fileSettingsService.close(); + } - super.tearDown(); - clusterService.close(); - threadpool.shutdownNow(); + super.tearDown(); + clusterService.close(); + threadpool.shutdownNow(); + } finally { + // TODO remove me once https://github.com/elastic/elasticsearch/issues/115280 is closed + Loggers.setLevel(LogManager.getLogger(AbstractFileWatchingService.class), Level.INFO); + } } public void testStartStop() { @@ -355,17 +370,25 @@ public void testHandleSnapshotRestoreResetsMetadata() throws Exception { private static void writeTestFile(Path path, String contents) { Path tempFile = null; try { + logger.info("Creating settings temp file under [{}]", path.toAbsolutePath()); tempFile = Files.createTempFile(path.getParent(), path.getFileName().toString(), "tmp"); + logger.info("Created settings temp file [{}]", tempFile.getFileName()); Files.writeString(tempFile, contents); try { + logger.info("Moving settings temp file to replace [{}]", tempFile.getFileName()); Files.move(tempFile, path, REPLACE_EXISTING, ATOMIC_MOVE); } catch (AtomicMoveNotSupportedException e) { + logger.info( + "Atomic move was not available. Falling back on non-atomic move for settings temp file to replace [{}]", + tempFile.getFileName() + ); Files.move(tempFile, path, REPLACE_EXISTING); } } catch (final IOException e) { throw new UncheckedIOException(Strings.format("could not write file [%s]", path.toAbsolutePath()), e); } finally { + logger.info("Deleting settings temp file [{}]", tempFile != null ? tempFile.getFileName() : null); // we are ignoring exceptions here, so we do not need handle whether or not tempFile was initialized nor if the file exists IOUtils.deleteFilesIgnoringExceptions(tempFile); } From 89bd58e04b6c000f605d00f6d3836fc13e5cb4d3 Mon Sep 17 00:00:00 2001 From: Kostas Krikellas <131142368+kkrik-es@users.noreply.github.com> Date: Tue, 5 Nov 2024 12:30:42 +0200 Subject: [PATCH 32/35] [TEST] Restore rest compat tests (#116229) * Track source for objects and fields with [synthetic_source_keep:arrays] in arrays as ignored * Update TransportResumeFollowActionTests.java * rest compat fixes * rest compat fixes * update test * Restore rest compat tests --- rest-api-spec/build.gradle | 5 ----- 1 file changed, 5 deletions(-) diff --git a/rest-api-spec/build.gradle b/rest-api-spec/build.gradle index 3532e08e8f659..8e1df37804708 100644 --- a/rest-api-spec/build.gradle +++ b/rest-api-spec/build.gradle @@ -57,11 +57,6 @@ tasks.named("precommit").configure { tasks.named("yamlRestCompatTestTransform").configure ({ task -> task.replaceValueInMatch("profile.shards.0.dfs.knn.0.query.0.description", "DocAndScoreQuery[0,...][0.009673266,...],0.009673266", "dfs knn vector profiling") task.replaceValueInMatch("profile.shards.0.dfs.knn.0.query.0.description", "DocAndScoreQuery[0,...][0.009673266,...],0.009673266", "dfs knn vector profiling with vector_operations_count") - task.skipTest("indices.sort/10_basic/Index Sort", "warning does not exist for compatibility") - task.skipTest("search/330_fetch_fields/Test search rewrite", "warning does not exist for compatibility") - task.skipTest("indices.create/21_synthetic_source_stored/index param - nested array within array - disabled second pass", "temporary until backported") - task.skipTest("indices.create/21_synthetic_source_stored/index param - root arrays", "temporary until backported") - task.skipTest("indices.create/21_synthetic_source_stored/object param - nested object with stored array", "temporary until backported") task.skipTest("cat.aliases/10_basic/Deprecated local parameter", "CAT APIs not covered by compatibility policy") task.skipTest("cat.shards/10_basic/Help", "sync_id is removed in 9.0") }) From b7b1ca041e2d96fe17d9baeaf80d6a575f61082a Mon Sep 17 00:00:00 2001 From: Andrew Wilkins Date: Tue, 5 Nov 2024 19:11:50 +0800 Subject: [PATCH 33/35] [apm-data] Apply lazy rollover on index template creation (#116219) * Apply lazy rollover on index template creation We should trigger a lazy rollover of existing data streams regardless of whether the index template is being created or updated. This ensures that the apm-data plugin will roll over data streams that were previously using the Fleet integration package. * Update docs/changelog/116219.yaml * Update docs/changelog/116219.yaml * Add YAML REST test for template reinstallation * Code review suggestion https://github.com/elastic/elasticsearch/pull/116219#discussion_r1828992554 * Remove wait_for_events from setup This doesn't guarantee the templates are set up, it only increases the chances; and we disable the plugin at the start of the test anyway. --- docs/changelog/116219.yaml | 6 + .../resources/rest-api-spec/test/10_apm.yml | 17 +++ .../rest-api-spec/test/10_rollover.yml | 51 ++++++++ .../RolloverEnabledTestTemplateRegistry.java | 2 +- .../core/template/IndexTemplateRegistry.java | 110 +++++++++++------- .../core/template/YamlTemplateRegistry.java | 2 +- .../template/IndexTemplateRegistryTests.java | 7 +- .../TestRegistryWithCustomPlugin.java | 2 +- 8 files changed, 146 insertions(+), 51 deletions(-) create mode 100644 docs/changelog/116219.yaml create mode 100644 x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/10_rollover.yml diff --git a/docs/changelog/116219.yaml b/docs/changelog/116219.yaml new file mode 100644 index 0000000000000..aeeea68570e77 --- /dev/null +++ b/docs/changelog/116219.yaml @@ -0,0 +1,6 @@ +pr: 116219 +summary: "[apm-data] Apply lazy rollover on index template creation" +area: Data streams +type: bug +issues: + - 116230 diff --git a/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/10_apm.yml b/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/10_apm.yml index c591668f3549d..64e7c12caeef3 100644 --- a/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/10_apm.yml +++ b/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/10_apm.yml @@ -53,6 +53,23 @@ setup: - contains: {index_templates: {name: logs-apm.app@template}} - contains: {index_templates: {name: logs-apm.error@template}} +--- +"Test template reinstallation": + - skip: + reason: contains is a newly added assertion + features: contains + - do: + indices.delete_index_template: + name: traces-apm@template + - do: + cluster.health: + wait_for_events: languid + - do: + indices.get_index_template: + name: traces-apm@template + - length: {index_templates: 1} + - contains: {index_templates: {name: traces-apm@template}} + --- "Test traces-apm-* data stream indexing": - skip: diff --git a/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/10_rollover.yml b/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/10_rollover.yml new file mode 100644 index 0000000000000..95586cd1fd665 --- /dev/null +++ b/x-pack/plugin/apm-data/src/yamlRestTest/resources/rest-api-spec/test/10_rollover.yml @@ -0,0 +1,51 @@ +--- +setup: + - do: + indices.put_index_template: + name: traces-low-prio + body: + data_stream: {} + index_patterns: ["traces-*"] + priority: 1 + +--- +"Test data stream rollover on template installation": + - skip: + awaits_fix: "https://github.com/elastic/elasticsearch/issues/102360" + + # Disable the apm-data plugin and delete the traces-apm@template index + # template so traces-low-prio takes effect. + - do: + cluster.put_settings: + body: + transient: + xpack.apm_data.registry.enabled: false + - do: + indices.delete_index_template: + name: traces-apm@template + - do: + indices.create_data_stream: + name: traces-apm-testing + - do: + indices.get_data_stream: + name: traces-apm-testing + - match: {data_streams.0.template: traces-low-prio} + + # Re-enable the apm-data plugin, after which the traces-apm@template + # index template should be recreated and trigger a lazy rollover on + # the traces-apm-testing data stream. + - do: + cluster.put_settings: + body: + transient: + xpack.apm_data.registry.enabled: true + - do: + cluster.health: + wait_for_events: languid + - do: + indices.get_data_stream: + name: traces-apm-testing + - length: {data_streams: 1} + - match: {data_streams.0.template: traces-apm@template} + - match: {data_streams.0.rollover_on_write: true} + diff --git a/x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/core/template/RolloverEnabledTestTemplateRegistry.java b/x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/core/template/RolloverEnabledTestTemplateRegistry.java index 819b0e01ac4de..442ad9a68dfc4 100644 --- a/x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/core/template/RolloverEnabledTestTemplateRegistry.java +++ b/x-pack/plugin/core/src/internalClusterTest/java/org/elasticsearch/xpack/core/template/RolloverEnabledTestTemplateRegistry.java @@ -53,7 +53,7 @@ protected Map getComposableTemplateConfigs() { } @Override - protected boolean applyRolloverAfterTemplateV2Upgrade() { + protected boolean applyRolloverAfterTemplateV2Update() { return true; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistry.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistry.java index 8849377e6ad7e..05f4e560b73c1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistry.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistry.java @@ -401,7 +401,7 @@ private void addComposableTemplatesIfMissing(ClusterState state) { } } else if (Objects.isNull(currentTemplate)) { logger.debug("adding composable template [{}] for [{}], because it doesn't exist", templateName, getOrigin()); - putComposableTemplate(state, templateName, newTemplate.getValue(), creationCheck, false); + putComposableTemplate(state, templateName, newTemplate.getValue(), creationCheck); } else if (Objects.isNull(currentTemplate.version()) || newTemplate.getValue().version() > currentTemplate.version()) { // IndexTemplateConfig now enforces templates contain a `version` property, so if the template doesn't have one we can // safely assume it's an old version of the template. @@ -412,7 +412,7 @@ private void addComposableTemplatesIfMissing(ClusterState state) { currentTemplate.version(), newTemplate.getValue().version() ); - putComposableTemplate(state, templateName, newTemplate.getValue(), creationCheck, true); + putComposableTemplate(state, templateName, newTemplate.getValue(), creationCheck); } else { creationCheck.set(false); logger.trace( @@ -434,11 +434,11 @@ private void addComposableTemplatesIfMissing(ClusterState state) { /** * Returns true if the cluster state contains all of the component templates needed by the composable template. If this registry - * requires automatic rollover after index template upgrades (see {@link #applyRolloverAfterTemplateV2Upgrade()}), this method also + * requires automatic rollover after index template upgrades (see {@link #applyRolloverAfterTemplateV2Update()}), this method also * verifies that the installed components templates are of the right version. */ private boolean componentTemplatesInstalled(ClusterState state, ComposableIndexTemplate indexTemplate) { - if (applyRolloverAfterTemplateV2Upgrade() == false) { + if (applyRolloverAfterTemplateV2Update() == false) { // component templates and index templates can be updated independently, we only need to know that the required component // templates are available return state.metadata().componentTemplates().keySet().containsAll(indexTemplate.getRequiredComponentTemplates()); @@ -534,8 +534,7 @@ private void putComposableTemplate( ClusterState state, final String templateName, final ComposableIndexTemplate indexTemplate, - final AtomicBoolean creationCheck, - final boolean isUpgrade + final AtomicBoolean creationCheck ) { final Executor executor = threadPool.generic(); executor.execute(() -> { @@ -550,8 +549,8 @@ private void putComposableTemplate( @Override public void onResponse(AcknowledgedResponse response) { if (response.isAcknowledged()) { - if (isUpgrade && applyRolloverAfterTemplateV2Upgrade()) { - invokeRollover(state, templateName, indexTemplate, creationCheck); + if (applyRolloverAfterTemplateV2Update()) { + invokeRollover(state, templateName, indexTemplate, () -> creationCheck.set((false))); } else { creationCheck.set(false); } @@ -765,12 +764,13 @@ public void onFailure(Exception e) { /** * Allows registries to opt-in for automatic rollover of "relevant" data streams immediately after a composable index template gets - * upgraded. If set to {@code true}, then every time a composable index template is being upgraded, all data streams of which name - * matches this template's index patterns AND of all matching templates the upgraded one has the highest priority, will be rolled over. + * updated, including its initial installation. If set to {@code true}, then every time a composable index template is being updated, + * all data streams of which name matches this template's index patterns AND of all matching templates the upgraded one has the highest + * priority, will be rolled over. * * @return {@code true} if this registry wants to apply automatic rollovers after template V2 upgrades */ - protected boolean applyRolloverAfterTemplateV2Upgrade() { + protected boolean applyRolloverAfterTemplateV2Update() { return false; } @@ -784,50 +784,56 @@ protected void onPutPipelineFailure(String pipelineId, Exception e) { logger.error(() -> format("error adding ingest pipeline template [%s] for [%s]", pipelineId, getOrigin()), e); } + /** + * invokeRollover rolls over any data streams matching the index template, + * and then invokes runAfter. + */ private void invokeRollover( final ClusterState state, final String templateName, final ComposableIndexTemplate indexTemplate, - final AtomicBoolean creationCheck + final Runnable runAfter ) { final Executor executor = threadPool.generic(); executor.execute(() -> { List rolloverTargets = findRolloverTargetDataStreams(state, templateName, indexTemplate); - if (rolloverTargets.isEmpty() == false) { - GroupedActionListener groupedActionListener = new GroupedActionListener<>( - rolloverTargets.size(), - new ActionListener<>() { - @Override - public void onResponse(Collection rolloverResponses) { - creationCheck.set(false); - onRolloversBulkResponse(rolloverResponses); - } + if (rolloverTargets.isEmpty()) { + runAfter.run(); + return; + } + GroupedActionListener groupedActionListener = new GroupedActionListener<>( + rolloverTargets.size(), + new ActionListener<>() { + @Override + public void onResponse(Collection rolloverResponses) { + runAfter.run(); + onRolloversBulkResponse(rolloverResponses); + } - @Override - public void onFailure(Exception e) { - creationCheck.set(false); - onRolloverFailure(e); - } + @Override + public void onFailure(Exception e) { + runAfter.run(); + onRolloverFailure(e); } - ); - for (String rolloverTarget : rolloverTargets) { - logger.info( - "rolling over data stream [{}] lazily as a followup to the upgrade of the [{}] index template [{}]", - rolloverTarget, - getOrigin(), - templateName - ); - RolloverRequest request = new RolloverRequest(rolloverTarget, null); - request.lazy(true); - request.masterNodeTimeout(TimeValue.MAX_VALUE); - executeAsyncWithOrigin( - client.threadPool().getThreadContext(), - getOrigin(), - request, - groupedActionListener, - (req, listener) -> client.execute(RolloverAction.INSTANCE, req, listener) - ); } + ); + for (String rolloverTarget : rolloverTargets) { + logger.info( + "rolling over data stream [{}] lazily as a followup to the upgrade of the [{}] index template [{}]", + rolloverTarget, + getOrigin(), + templateName + ); + RolloverRequest request = new RolloverRequest(rolloverTarget, null); + request.lazy(true); + request.masterNodeTimeout(TimeValue.MAX_VALUE); + executeAsyncWithOrigin( + client.threadPool().getThreadContext(), + getOrigin(), + request, + groupedActionListener, + (req, listener) -> client.execute(RolloverAction.INSTANCE, req, listener) + ); } }); } @@ -867,7 +873,21 @@ static List findRolloverTargetDataStreams(ClusterState state, String tem .stream() // Limit to checking data streams that match any of the index template's index patterns .filter(ds -> indexTemplate.indexPatterns().stream().anyMatch(pattern -> Regex.simpleMatch(pattern, ds.getName()))) - .filter(ds -> templateName.equals(MetadataIndexTemplateService.findV2Template(metadata, ds.getName(), ds.isHidden()))) + .filter(ds -> { + final String dsTemplateName = MetadataIndexTemplateService.findV2Template(metadata, ds.getName(), ds.isHidden()); + if (templateName.equals(dsTemplateName)) { + return true; + } + // findV2Template did not match templateName, which implies one of two things: + // - indexTemplate has a lower priority than the index template matching for ds, OR + // - indexTemplate does not yet exist in cluster state (i.e. because it's in the process of being + // installed or updated) + // + // Because of the second case, we must check if indexTemplate's priority is greater than the matching + // index template, in case it would take precedence after installation/update. + final ComposableIndexTemplate dsTemplate = metadata.templatesV2().get(dsTemplateName); + return dsTemplate == null || indexTemplate.priorityOrZero() > dsTemplate.priorityOrZero(); + }) .map(DataStream::getName) .collect(Collectors.toList()); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlTemplateRegistry.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlTemplateRegistry.java index 183251f39a029..c8ddd46c5912f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlTemplateRegistry.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/YamlTemplateRegistry.java @@ -227,7 +227,7 @@ private IngestPipelineConfig loadIngestPipeline(String name, int version, @Nulla } @Override - protected boolean applyRolloverAfterTemplateV2Upgrade() { + protected boolean applyRolloverAfterTemplateV2Update() { return true; } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistryTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistryTests.java index b0127c0005323..8d8aa9fd3c634 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistryTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/template/IndexTemplateRegistryTests.java @@ -435,7 +435,7 @@ public void testAutomaticRollover() throws Exception { assertThat(suppressed[0].getMessage(), startsWith("Failed to rollover logs-my_app-")); } - public void testNoRolloverForFreshInstalledIndexTemplate() throws Exception { + public void testRolloverForFreshInstalledIndexTemplate() throws Exception { DiscoveryNode node = DiscoveryNodeUtils.create("node"); DiscoveryNodes nodes = DiscoveryNodes.builder().localNodeId("node").masterNodeId("node").add(node).build(); @@ -473,9 +473,10 @@ public void testNoRolloverForFreshInstalledIndexTemplate() throws Exception { registry.setApplyRollover(true); registry.clusterChanged(event); assertBusy(() -> assertThat(putIndexTemplateCounter.get(), equalTo(1))); - // the index component is first installed, not upgraded, therefore rollover should not be triggered + // rollover should be triggered even for the first installation, since the template + // may now take precedence over a data stream's existing index template Thread.sleep(100L); - assertThat(rolloverCounter.get(), equalTo(0)); + assertThat(rolloverCounter.get(), equalTo(2)); } public void testThatTemplatesAreNotUpgradedWhenNotNeeded() throws Exception { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/template/TestRegistryWithCustomPlugin.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/template/TestRegistryWithCustomPlugin.java index 349fdfe1259c9..2ef0c7f5301ec 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/template/TestRegistryWithCustomPlugin.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/template/TestRegistryWithCustomPlugin.java @@ -118,7 +118,7 @@ public void setPolicyUpgradeRequired(boolean policyUpgradeRequired) { } @Override - protected boolean applyRolloverAfterTemplateV2Upgrade() { + protected boolean applyRolloverAfterTemplateV2Update() { return applyRollover.get(); } From 60f67530bb0cc79d2ae6925e7b5af126271c59b1 Mon Sep 17 00:00:00 2001 From: Alexey Ivanov Date: Tue, 5 Nov 2024 11:21:17 +0000 Subject: [PATCH 34/35] [CI] JvmStatsTests testJvmStats failing (115711) (#116197) Fix and unmute test JvmStatsTests.testJvmStats Fixes #115711 --- muted-tests.yml | 3 --- .../test/java/org/elasticsearch/monitor/jvm/JvmStatsTests.java | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 902f4789112d8..18cb70bfd5fda 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -193,9 +193,6 @@ tests: - class: org.elasticsearch.xpack.test.rest.XPackRestIT method: test {p0=ml/inference_crud/Test delete given model referenced by pipeline} issue: https://github.com/elastic/elasticsearch/issues/115970 -- class: org.elasticsearch.monitor.jvm.JvmStatsTests - method: testJvmStats - issue: https://github.com/elastic/elasticsearch/issues/115711 - class: org.elasticsearch.repositories.s3.S3ServiceTests method: testRetryOn403RetryPolicy issue: https://github.com/elastic/elasticsearch/issues/115986 diff --git a/server/src/test/java/org/elasticsearch/monitor/jvm/JvmStatsTests.java b/server/src/test/java/org/elasticsearch/monitor/jvm/JvmStatsTests.java index 28976d803ff53..7956d67c83c3b 100644 --- a/server/src/test/java/org/elasticsearch/monitor/jvm/JvmStatsTests.java +++ b/server/src/test/java/org/elasticsearch/monitor/jvm/JvmStatsTests.java @@ -53,7 +53,7 @@ public void testJvmStats() { assertThat(memoryPools, hasKey("Metaspace")); assertThat(memoryPools.keySet(), hasSize(greaterThan(3))); for (JvmStats.MemoryPool memoryPool : memoryPools.values()) { - assertThat(memoryPool.getUsed().getBytes(), greaterThan(0L)); + assertThat("Memory pool: " + memoryPool.getName(), memoryPool.getUsed().getBytes(), greaterThanOrEqualTo(0L)); } // Threads From 8f551685773476cb67d2488a58a4a936ea7def0b Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 5 Nov 2024 12:27:43 +0100 Subject: [PATCH 35/35] [TEST] Adjust PartialHitCountCollectorTests#testCollectedHitCount (#116181) Split the test in two, one to verify behaviour with threashold greather than 1. Wrote a specific test for the edge case of threshold set to 1. Added a comment that explains the nuance around the behaviour and what influences it. Closes #106647 --- .../query/PartialHitCountCollectorTests.java | 35 +++++++++++++++++-- 1 file changed, 32 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/query/PartialHitCountCollectorTests.java b/server/src/test/java/org/elasticsearch/search/query/PartialHitCountCollectorTests.java index 961d5200b6e0d..62cd3bf2f308c 100644 --- a/server/src/test/java/org/elasticsearch/search/query/PartialHitCountCollectorTests.java +++ b/server/src/test/java/org/elasticsearch/search/query/PartialHitCountCollectorTests.java @@ -15,13 +15,17 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.Term; +import org.apache.lucene.search.BulkScorer; +import org.apache.lucene.search.CollectionTerminatedException; import org.apache.lucene.search.CollectorManager; +import org.apache.lucene.search.DocIdSetIterator; import org.apache.lucene.search.FilterLeafCollector; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Weight; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; import org.elasticsearch.test.ESTestCase; @@ -121,15 +125,40 @@ public void testHitCountFromWeightDoesNotEarlyTerminate() throws IOException { public void testCollectedHitCount() throws Exception { Query query = new NonCountingTermQuery(new Term("string", "a1")); - int threshold = randomIntBetween(1, 10000); - assumeTrue("bug with single collection & single segment: https://github.com/elastic/elasticsearch/issues/106647", threshold > 1); - // there's one doc matching the query: any totalHitsThreshold greater than or equal to 1 will not cause early termination + int threshold = randomIntBetween(2, 10000); + // there's one doc matching the query: any totalHitsThreshold greater than 1 will not cause early termination CollectorManager collectorManager = createCollectorManager(new HitsThresholdChecker(threshold)); Result result = searcher.search(query, collectorManager); assertEquals(1, result.totalHits); assertFalse(result.terminatedAfter); } + public void testThresholdOne() throws Exception { + Query query = new NonCountingTermQuery(new Term("string", "a1")); + Weight weight = query.createWeight(searcher, ScoreMode.COMPLETE, 0f); + CollectorManager collectorManager = createCollectorManager(new HitsThresholdChecker(1)); + // threshold 1 behaves differently depending on whether there is a single segment (no early termination) or multiple segments. + // With inter-segment concurrency the behaviour is not deterministic and depends on the timing of the different threads. + // Without inter-segment concurrency the behaviour depends on which segment holds the matching document. + // This is because the check for early termination is performed every time a leaf collector is pulled for a segment, as well + // as for every collected doc. + PartialHitCountCollector partialHitCountCollector = collectorManager.newCollector(); + int i = 0; + while (partialHitCountCollector.getTotalHits() == 0 && i < searcher.getLeafContexts().size()) { + LeafReaderContext ctx = searcher.getLeafContexts().get(i++); + LeafCollector leafCollector = partialHitCountCollector.getLeafCollector(ctx); + BulkScorer bulkScorer = weight.bulkScorer(ctx); + bulkScorer.score(leafCollector, ctx.reader().getLiveDocs(), 0, DocIdSetIterator.NO_MORE_DOCS); + } + assertEquals(1, partialHitCountCollector.getTotalHits()); + assertFalse(partialHitCountCollector.hasEarlyTerminated()); + expectThrows( + CollectionTerminatedException.class, + () -> partialHitCountCollector.getLeafCollector(randomFrom(searcher.getLeafContexts())) + ); + assertTrue(partialHitCountCollector.hasEarlyTerminated()); + } + public void testCollectedHitCountEarlyTerminated() throws Exception { Query query = new NonCountingTermQuery(new Term("string", "foo")); // there's three docs matching the query: any totalHitsThreshold lower than 3 will trigger early termination