From e34ea3be3739283f56a429fb8b87b075686542ef Mon Sep 17 00:00:00 2001 From: David Roberts Date: Wed, 8 Nov 2023 11:22:38 +0000 Subject: [PATCH 001/513] [ML] Fix serverless downscale decision when HA prevents it (#101902) In serverless we require up to 3 nodes to be in the cluster when models have more than one allocation. This is to ensure that models can remain allocated during rolling restarts. This PR fixes a bug where the downscaling code was not taking this policy into account and was suggesting a downscale when models would fit on fewer nodes, but not with the desired level of high availability. --- .../MlAutoscalingResourceTracker.java | 2 + .../MlAutoscalingResourceTrackerTests.java | 103 ++++++++++++++++++ 2 files changed, 105 insertions(+) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/autoscaling/MlAutoscalingResourceTracker.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/autoscaling/MlAutoscalingResourceTracker.java index 4b925f678602a..ac6f3914b8f40 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/autoscaling/MlAutoscalingResourceTracker.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/autoscaling/MlAutoscalingResourceTracker.java @@ -270,11 +270,13 @@ static void getMemoryAndProcessors( // - modelMemory on nodes is available // - no jobs wait for assignment // - the total memory usage is less than memory usage after taking away 1 node + // - the current number of nodes is greater than the minimum number of nodes if (perNodeMemoryInBytes > 0 && perNodeAvailableModelMemoryInBytes > 0 && extraModelMemoryInBytes == 0 && extraProcessors == 0 && modelMemoryBytesSum <= perNodeMemoryInBytes * (numberMlNodes - 1) + && minNodes < numberMlNodes && (perNodeModelMemoryInBytes.size() < numberMlNodes // a node has no assigned jobs || checkIfOneNodeCouldBeRemoved( perNodeModelMemoryInBytes, diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/autoscaling/MlAutoscalingResourceTrackerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/autoscaling/MlAutoscalingResourceTrackerTests.java index 7ea63cf7945f0..0028c66dd9659 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/autoscaling/MlAutoscalingResourceTrackerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/autoscaling/MlAutoscalingResourceTrackerTests.java @@ -9,11 +9,14 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.LatchedActionListener; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodeUtils; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.persistent.PersistentTasksCustomMetadata; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.ml.MachineLearningField; import org.elasticsearch.xpack.core.ml.MlTasks; import org.elasticsearch.xpack.core.ml.action.OpenJobAction; import org.elasticsearch.xpack.core.ml.action.StartTrainedModelDeploymentAction; @@ -26,6 +29,7 @@ import org.elasticsearch.xpack.core.ml.job.config.JobTaskState; import org.elasticsearch.xpack.ml.MachineLearning; import org.elasticsearch.xpack.ml.process.MlMemoryTracker; +import org.elasticsearch.xpack.ml.utils.NativeMemoryCalculator; import java.net.InetAddress; import java.util.Collections; @@ -1076,6 +1080,105 @@ public void testGetMemoryAndProcessorsScaleDown() throws InterruptedException { ); } + // scenario: 3 ml nodes, could scale down purely considering memory but high availability needs prevent it + public void testGetMemoryAndProcessorsScaleDownPreventedByMinNodes() throws InterruptedException { + long memory = 8589934592L; + Map nodeAttr = Map.of( + MachineLearning.MACHINE_MEMORY_NODE_ATTR, + Long.toString(memory), + MachineLearning.MAX_JVM_SIZE_NODE_ATTR, + "3435134976", + MachineLearning.ALLOCATED_PROCESSORS_NODE_ATTR, + "4.0", + MachineLearning.ML_CONFIG_VERSION_NODE_ATTR, + "11.0.0" + ); + Settings settings = Settings.builder().put(MachineLearningField.USE_AUTO_MACHINE_MEMORY_PERCENT.getKey(), true).build(); + DiscoveryNode firstNode = DiscoveryNodeUtils.builder("ml-node-1") + .name("ml-node-name-1") + .address(new TransportAddress(InetAddress.getLoopbackAddress(), 9300)) + .attributes(nodeAttr) + .roles(Set.of(DiscoveryNodeRole.ML_ROLE)) + .build(); + MlAutoscalingContext mlAutoscalingContext = new MlAutoscalingContext( + List.of(), + List.of(), + List.of(), + Map.of( + ".elser_model_2_linux-x86_64", + TrainedModelAssignment.Builder.empty( + new StartTrainedModelDeploymentAction.TaskParams( + ".elser_model_2_linux-x86_64", + ".elser_model_2_linux-x86_64", + 274756282, + 4, + 2, + 100, + null, + Priority.NORMAL, + 0L, + 0L + ) + ) + .addRoutingEntry("ml-node-1", new RoutingInfo(2, 2, RoutingState.STARTED, "")) + .addRoutingEntry("ml-node-2", new RoutingInfo(2, 2, RoutingState.STARTED, "")) + .build(), + "intfloat__multilingual-e5-base", + TrainedModelAssignment.Builder.empty( + new StartTrainedModelDeploymentAction.TaskParams( + "intfloat__multilingual-e5-base", + "intfloat__multilingual-e5-base", + 1109885608, + 1, + 1, + 100, + null, + Priority.NORMAL, + 0L, + 0L + ) + ).addRoutingEntry("ml-node-3", new RoutingInfo(1, 1, RoutingState.STARTED, "")).build() + ), + List.of( + firstNode, + DiscoveryNodeUtils.builder("ml-node-2") + .name("ml-node-name-2") + .address(new TransportAddress(InetAddress.getLoopbackAddress(), 9300)) + .attributes(nodeAttr) + .roles(Set.of(DiscoveryNodeRole.ML_ROLE)) + .build(), + DiscoveryNodeUtils.builder("ml-node-3") + .name("ml-node-name-3") + .address(new TransportAddress(InetAddress.getLoopbackAddress(), 9300)) + .attributes(nodeAttr) + .roles(Set.of(DiscoveryNodeRole.ML_ROLE)) + .build() + ), + PersistentTasksCustomMetadata.builder().build() + ); + MlMemoryTracker mockTracker = mock(MlMemoryTracker.class); + + this.assertAsync( + listener -> MlAutoscalingResourceTracker.getMemoryAndProcessors( + mlAutoscalingContext, + mockTracker, + Map.of("ml-node-1", memory, "ml-node-2", memory, "ml-node-3", memory), + NativeMemoryCalculator.allowedBytesForMl(firstNode, settings).getAsLong(), + 4, + MachineLearning.DEFAULT_MAX_OPEN_JOBS_PER_NODE, + listener + ), + stats -> { + assertEquals(memory, stats.perNodeMemoryInBytes()); + assertEquals(3, stats.nodes()); + assertEquals(3, stats.minNodes()); + assertEquals(0, stats.extraSingleNodeProcessors()); + assertEquals(0, stats.removeNodeMemoryInBytes()); + assertEquals(MachineLearning.NATIVE_EXECUTABLE_CODE_OVERHEAD.getBytes(), stats.perNodeMemoryOverheadInBytes()); + } + ); + } + private void assertAsync(Consumer> function, Consumer furtherTests) throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); AtomicBoolean listenerCalled = new AtomicBoolean(false); From cb6a570c49387ffa75098cf02f3df75ceaa61c3e Mon Sep 17 00:00:00 2001 From: Ievgen Degtiarenko Date: Wed, 8 Nov 2023 13:13:00 +0100 Subject: [PATCH 002/513] Extract shard size estimation into a separate component (#101854) This commit is a prerequisite for the future improvements of the initializing shard size estimation. It moves the initializing shard size estimation into a separate component and covers it with additional tests. --- .../routing/ExpectedShardSizeEstimator.java | 86 +++++++++ .../allocator/BalancedShardsAllocator.java | 14 +- .../allocator/DesiredBalanceReconciler.java | 11 +- .../decider/DiskThresholdDecider.java | 63 +------ .../ExpectedShardSizeEstimatorTests.java | 149 +++++++++++++++ .../ExpectedShardSizeAllocationTests.java | 169 +++++++++++++++--- .../DiskThresholdDeciderUnitTests.java | 27 +-- .../ReactiveStorageDeciderService.java | 3 +- .../SearchableSnapshotAllocator.java | 4 +- 9 files changed, 408 insertions(+), 118 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimator.java create mode 100644 server/src/test/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimatorTests.java diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimator.java b/server/src/main/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimator.java new file mode 100644 index 0000000000000..05c0876669732 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimator.java @@ -0,0 +1,86 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.cluster.routing; + +import org.elasticsearch.cluster.ClusterInfo; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.snapshots.SnapshotShardSizeInfo; + +import java.util.Set; + +public class ExpectedShardSizeEstimator { + + public static long getExpectedShardSize(ShardRouting shardRouting, long defaultSize, RoutingAllocation allocation) { + return getExpectedShardSize( + shardRouting, + defaultSize, + allocation.clusterInfo(), + allocation.snapshotShardSizeInfo(), + allocation.metadata(), + allocation.routingTable() + ); + } + + /** + * Returns the expected shard size for the given shard or the default value provided if not enough information are available + * to estimate the shards size. + */ + public static long getExpectedShardSize( + ShardRouting shard, + long defaultValue, + ClusterInfo clusterInfo, + SnapshotShardSizeInfo snapshotShardSizeInfo, + Metadata metadata, + RoutingTable routingTable + ) { + final IndexMetadata indexMetadata = metadata.getIndexSafe(shard.index()); + if (indexMetadata.getResizeSourceIndex() != null + && shard.active() == false + && shard.recoverySource().getType() == RecoverySource.Type.LOCAL_SHARDS) { + return getExpectedSizeOfResizedShard(shard, defaultValue, indexMetadata, clusterInfo, metadata, routingTable); + } else if (shard.unassigned() && shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) { + return snapshotShardSizeInfo.getShardSize(shard, defaultValue); + } else { + return clusterInfo.getShardSize(shard, defaultValue); + } + } + + private static long getExpectedSizeOfResizedShard( + ShardRouting shard, + long defaultValue, + IndexMetadata indexMetadata, + ClusterInfo clusterInfo, + Metadata metadata, + RoutingTable routingTable + ) { + // in the shrink index case we sum up the source index shards since we basically make a copy of the shard in the worst case + long targetShardSize = 0; + final Index mergeSourceIndex = indexMetadata.getResizeSourceIndex(); + final IndexMetadata sourceIndexMetadata = metadata.index(mergeSourceIndex); + if (sourceIndexMetadata != null) { + final Set shardIds = IndexMetadata.selectRecoverFromShards( + shard.id(), + sourceIndexMetadata, + indexMetadata.getNumberOfShards() + ); + final IndexRoutingTable indexRoutingTable = routingTable.index(mergeSourceIndex.getName()); + for (int i = 0; i < indexRoutingTable.size(); i++) { + IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(i); + if (shardIds.contains(shardRoutingTable.shardId())) { + targetShardSize += clusterInfo.getShardSize(shardRoutingTable.primaryShard(), 0); + } + } + } + return targetShardSize == 0 ? defaultValue : targetShardSize; + } +} 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 8d336d2147e11..438c81b5fbb98 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 @@ -31,7 +31,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type; -import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.logging.DeprecationCategory; import org.elasticsearch.common.logging.DeprecationLogger; @@ -57,6 +56,7 @@ import java.util.stream.StreamSupport; import static org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata.Type.REPLACE; +import static org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator.getExpectedShardSize; import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING; import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings; @@ -1037,11 +1037,7 @@ private void allocateUnassigned() { logger.trace("Assigned shard [{}] to [{}]", shard, minNode.getNodeId()); } - final long shardSize = DiskThresholdDecider.getExpectedShardSize( - shard, - ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, - allocation - ); + final long shardSize = getExpectedShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation); shard = routingNodes.initializeShard(shard, minNode.getNodeId(), null, shardSize, allocation.changes()); minNode.addShard(shard); if (shard.primary() == false) { @@ -1064,11 +1060,7 @@ private void allocateUnassigned() { if (minNode != null) { // throttle decision scenario assert allocationDecision.getAllocationStatus() == AllocationStatus.DECIDERS_THROTTLED; - final long shardSize = DiskThresholdDecider.getExpectedShardSize( - shard, - ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, - allocation - ); + final long shardSize = getExpectedShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation); minNode.addShard(shard.initialize(minNode.getNodeId(), null, shardSize)); } else { if (logger.isTraceEnabled()) { 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 048ade3ef86c5..6fac97e34d022 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,7 +21,6 @@ import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; -import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -40,6 +39,7 @@ import java.util.stream.IntStream; import static org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata.Type.REPLACE; +import static org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator.getExpectedShardSize; /** * Given the current allocation of shards and the desired balance, performs the next (legal) shard movements towards the goal. @@ -271,14 +271,7 @@ private void allocateUnassigned() { switch (decision.type()) { case YES -> { logger.debug("Assigning shard [{}] to {} [{}]", shard, nodeIdsIterator.source, nodeId); - final long shardSize = DiskThresholdDecider.getExpectedShardSize( - shard, - ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, - allocation.clusterInfo(), - allocation.snapshotShardSizeInfo(), - allocation.metadata(), - allocation.routingTable() - ); + long shardSize = getExpectedShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation); routingNodes.initializeShard(shard, nodeId, null, shardSize, allocation.changes()); allocationOrdering.recordAllocation(nodeId); if (shard.primary() == false) { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java index d916aa7638786..fe001480e5f46 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java @@ -15,8 +15,6 @@ import org.elasticsearch.cluster.DiskUsage; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; -import org.elasticsearch.cluster.routing.IndexRoutingTable; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingTable; @@ -29,12 +27,10 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.snapshots.SnapshotShardSizeInfo; import java.util.Map; -import java.util.Set; + +import static org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator.getExpectedShardSize; /** * The {@link DiskThresholdDecider} checks that the node a shard is potentially @@ -541,61 +537,6 @@ private Decision earlyTerminate(Map usages) { return null; } - public static long getExpectedShardSize(ShardRouting shardRouting, long defaultSize, RoutingAllocation allocation) { - return DiskThresholdDecider.getExpectedShardSize( - shardRouting, - defaultSize, - allocation.clusterInfo(), - allocation.snapshotShardSizeInfo(), - allocation.metadata(), - allocation.routingTable() - ); - } - - /** - * Returns the expected shard size for the given shard or the default value provided if not enough information are available - * to estimate the shards size. - */ - public static long getExpectedShardSize( - ShardRouting shard, - long defaultValue, - ClusterInfo clusterInfo, - SnapshotShardSizeInfo snapshotShardSizeInfo, - Metadata metadata, - RoutingTable routingTable - ) { - final IndexMetadata indexMetadata = metadata.getIndexSafe(shard.index()); - if (indexMetadata.getResizeSourceIndex() != null - && shard.active() == false - && shard.recoverySource().getType() == RecoverySource.Type.LOCAL_SHARDS) { - // in the shrink index case we sum up the source index shards since we basically make a copy of the shard in - // the worst case - long targetShardSize = 0; - final Index mergeSourceIndex = indexMetadata.getResizeSourceIndex(); - final IndexMetadata sourceIndexMeta = metadata.index(mergeSourceIndex); - if (sourceIndexMeta != null) { - final Set shardIds = IndexMetadata.selectRecoverFromShards( - shard.id(), - sourceIndexMeta, - indexMetadata.getNumberOfShards() - ); - final IndexRoutingTable indexRoutingTable = routingTable.index(mergeSourceIndex.getName()); - for (int i = 0; i < indexRoutingTable.size(); i++) { - IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(i); - if (shardIds.contains(shardRoutingTable.shardId())) { - targetShardSize += clusterInfo.getShardSize(shardRoutingTable.primaryShard(), 0); - } - } - } - return targetShardSize == 0 ? defaultValue : targetShardSize; - } else { - if (shard.unassigned() && shard.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) { - return snapshotShardSizeInfo.getShardSize(shard, defaultValue); - } - return clusterInfo.getShardSize(shard, defaultValue); - } - } - record DiskUsageWithRelocations(DiskUsage diskUsage, long relocatingShardSize) { double getFreeDiskAsPercentage() { diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimatorTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimatorTests.java new file mode 100644 index 0000000000000..c894585edd776 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/cluster/routing/ExpectedShardSizeEstimatorTests.java @@ -0,0 +1,149 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.cluster.routing; + +import org.elasticsearch.cluster.ClusterInfo; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ESAllocationTestCase; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; +import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.InternalSnapshotsInfoService; +import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotShardSizeInfo; + +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_RESIZE_SOURCE_NAME_KEY; +import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_RESIZE_SOURCE_UUID_KEY; +import static org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator.getExpectedShardSize; +import static org.elasticsearch.cluster.routing.TestShardRouting.newShardRouting; +import static org.hamcrest.Matchers.equalTo; + +public class ExpectedShardSizeEstimatorTests extends ESAllocationTestCase { + + private final long defaultValue = randomLongBetween(-1, 0); + + public void testShouldFallbackToDefaultValue() { + + var state = ClusterState.builder(ClusterName.DEFAULT).metadata(metadata(index("my-index"))).build(); + var shard = newShardRouting("my-index", 0, randomIdentifier(), true, ShardRoutingState.INITIALIZING); + + var allocation = createRoutingAllocation(state, ClusterInfo.EMPTY, SnapshotShardSizeInfo.EMPTY); + + assertThat(getExpectedShardSize(shard, defaultValue, allocation), equalTo(defaultValue)); + } + + public void testShouldReadExpectedSizeFromClusterInfo() { + + var shardSize = randomLongBetween(100, 1000); + var state = ClusterState.builder(ClusterName.DEFAULT).metadata(metadata(index("my-index"))).build(); + var shard = newShardRouting("my-index", 0, randomIdentifier(), true, ShardRoutingState.INITIALIZING); + + var clusterInfo = createClusterInfo(shard, shardSize); + var allocation = createRoutingAllocation(state, clusterInfo, SnapshotShardSizeInfo.EMPTY); + + assertThat(getExpectedShardSize(shard, defaultValue, allocation), equalTo(shardSize)); + } + + public void testShouldReadExpectedSizeWhenInitializingFromSnapshot() { + + var snapshotShardSize = randomLongBetween(100, 1000); + var state = ClusterState.builder(ClusterName.DEFAULT).metadata(metadata(index("my-index"))).build(); + + var snapshot = new Snapshot("repository", new SnapshotId("snapshot-1", "na")); + var indexId = new IndexId("my-index", "_na_"); + + var shard = newShardRouting( + new ShardId("my-index", "_na_", 0), + null, + true, + ShardRoutingState.UNASSIGNED, + new RecoverySource.SnapshotRecoverySource(randomUUID(), snapshot, IndexVersion.current(), indexId) + ); + + var snapshotShardSizeInfo = new SnapshotShardSizeInfo( + Map.of(new InternalSnapshotsInfoService.SnapshotShard(snapshot, indexId, shard.shardId()), snapshotShardSize) + ); + var allocation = createRoutingAllocation(state, ClusterInfo.EMPTY, snapshotShardSizeInfo); + + assertThat(getExpectedShardSize(shard, defaultValue, allocation), equalTo(snapshotShardSize)); + } + + public void testShouldReadSizeFromClonedShard() { + + var sourceShardSize = randomLongBetween(100, 1000); + var source = newShardRouting(new ShardId("source", "_na_", 0), randomIdentifier(), true, ShardRoutingState.STARTED); + var target = newShardRouting( + new ShardId("target", "_na_", 0), + randomIdentifier(), + true, + ShardRoutingState.INITIALIZING, + RecoverySource.LocalShardsRecoverySource.INSTANCE + ); + + var state = ClusterState.builder(ClusterName.DEFAULT) + .metadata( + metadata( + IndexMetadata.builder("source").settings(indexSettings(IndexVersion.current(), 2, 0)), + IndexMetadata.builder("target") + .settings( + indexSettings(IndexVersion.current(), 1, 0) // + .put(INDEX_RESIZE_SOURCE_NAME_KEY, "source") // + .put(INDEX_RESIZE_SOURCE_UUID_KEY, "_na_") + ) + ) + ) + .routingTable(RoutingTable.builder().add(IndexRoutingTable.builder(source.index()).addShard(source))) + .build(); + + var clusterInfo = createClusterInfo(source, sourceShardSize); + var allocation = createRoutingAllocation(state, clusterInfo, SnapshotShardSizeInfo.EMPTY); + + assertThat(getExpectedShardSize(target, defaultValue, allocation), equalTo(sourceShardSize)); + } + + private static RoutingAllocation createRoutingAllocation( + ClusterState state, + ClusterInfo clusterInfo, + SnapshotShardSizeInfo snapshotShardSizeInfo + ) { + return new RoutingAllocation(new AllocationDeciders(List.of()), state, clusterInfo, snapshotShardSizeInfo, 0); + } + + private static IndexMetadata.Builder index(String name) { + return IndexMetadata.builder(name).settings(indexSettings(IndexVersion.current(), 1, 0)); + } + + private static Metadata metadata(IndexMetadata.Builder... indices) { + var builder = Metadata.builder(); + for (IndexMetadata.Builder index : indices) { + builder.put(index.build(), false); + } + return builder.build(); + } + + private static ClusterInfo createClusterInfo(ShardRouting shard, Long size) { + return new ClusterInfo( + Map.of(), + Map.of(), + Map.of(ClusterInfo.shardIdentifierFromRouting(shard), size), + Map.of(), + Map.of(), + Map.of() + ); + } +} diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ExpectedShardSizeAllocationTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ExpectedShardSizeAllocationTests.java index a99c77b0be3cb..9ccc80ae8d1b4 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ExpectedShardSizeAllocationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/ExpectedShardSizeAllocationTests.java @@ -12,37 +12,170 @@ import org.elasticsearch.cluster.ClusterInfo; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.DiskUsage; import org.elasticsearch.cluster.ESAllocationTestCase; +import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.TestShardRoutingRoleStrategies; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.InternalSnapshotsInfoService; +import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotShardSizeInfo; +import org.elasticsearch.test.gateway.TestGatewayAllocator; +import java.util.Collection; +import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import static java.util.stream.Collectors.toMap; import static org.elasticsearch.cluster.routing.RoutingNodesHelper.shardsWithState; +import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; public class ExpectedShardSizeAllocationTests extends ESAllocationTestCase { + + public void testAllocateToCorrectNodeUsingShardSizeFromClusterInfo() { + + var indexMetadata = IndexMetadata.builder("test").settings(indexSettings(IndexVersion.current(), 1, 0)).build(); + + var clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(newNode("node-1")).add(newNode("node-2")).add(newNode("node-3"))) + .metadata(Metadata.builder().put(indexMetadata, false)) + .routingTable(RoutingTable.builder(TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY).addAsNew(indexMetadata)) + .build(); + var dataNodeIds = clusterState.nodes().getDataNodes().keySet(); + + long shardSize = ByteSizeValue.ofGb(1).getBytes(); + long diskSize = ByteSizeValue.ofGb(5).getBytes(); + long headRoom = diskSize / 10; + var expectedNodeId = randomFrom(dataNodeIds); + var clusterInfo = createClusterInfo( + createDiskUsage( + dataNodeIds, + nodeId -> createDiskUsage(nodeId, diskSize, headRoom + shardSize + (Objects.equals(nodeId, expectedNodeId) ? +1 : -1)) + ), + Map.of(ClusterInfo.shardIdentifierFromRouting(new ShardId(indexMetadata.getIndex(), 0), true), shardSize) + ); + + AllocationService service = createAllocationService(Settings.EMPTY, () -> clusterInfo); + clusterState = service.reroute(clusterState, "reroute", ActionListener.noop()); + + assertThatShard( + clusterState.routingTable().index(indexMetadata.getIndex()).shard(0).primaryShard(), + INITIALIZING, + expectedNodeId, + shardSize + ); + } + + public void testAllocateToCorrectNodeAccordingToSnapshotShardInfo() { + + var snapshot = new Snapshot("repository", new SnapshotId("snapshot-1", "na")); + var indexId = new IndexId("my-index", "_na_"); + var restoreId = "restore-id"; + + var indexMetadata = IndexMetadata.builder("test") + .settings(indexSettings(IndexVersion.current(), 1, 0)) + .putInSyncAllocationIds(0, Set.of(randomUUID())) + .build(); + + var clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(newNode("node-1")).add(newNode("node-2")).add(newNode("node-3"))) + .metadata(Metadata.builder().put(indexMetadata, false)) + .routingTable( + RoutingTable.builder(TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY) + .addAsRestore( + indexMetadata, + new RecoverySource.SnapshotRecoverySource(restoreId, snapshot, IndexVersion.current(), indexId) + ) + ) + .customs( + Map.of( + RestoreInProgress.TYPE, + new RestoreInProgress.Builder().add( + new RestoreInProgress.Entry( + restoreId, + snapshot, + RestoreInProgress.State.STARTED, + false, + List.of(indexMetadata.getIndex().getName()), + Map.of(new ShardId(indexMetadata.getIndex(), 0), new RestoreInProgress.ShardRestoreStatus(randomIdentifier())) + ) + ).build() + ) + ) + .build(); + var dataNodeIds = clusterState.nodes().getDataNodes().keySet(); + + long shardSize = ByteSizeValue.ofGb(1).getBytes(); + long diskSize = ByteSizeValue.ofGb(5).getBytes(); + long headRoom = diskSize / 10; + var expectedNodeId = randomFrom(dataNodeIds); + var clusterInfo = createClusterInfo( + createDiskUsage( + dataNodeIds, + nodeId -> createDiskUsage(nodeId, diskSize, headRoom + shardSize + (Objects.equals(nodeId, expectedNodeId) ? +1 : -1)) + ), + Map.of() + ); + var snapshotShardSizeInfo = new SnapshotShardSizeInfo( + Map.of(new InternalSnapshotsInfoService.SnapshotShard(snapshot, indexId, new ShardId(indexMetadata.getIndex(), 0)), shardSize) + ); + + AllocationService service = createAllocationService( + Settings.EMPTY, + new TestGatewayAllocator(), + () -> clusterInfo, + () -> snapshotShardSizeInfo + ); + clusterState = service.reroute(clusterState, "reroute", ActionListener.noop()); + + assertThatShard( + clusterState.routingTable().index(indexMetadata.getIndex()).shard(0).primaryShard(), + INITIALIZING, + expectedNodeId, + shardSize + ); + } + + private static void assertThatShard(ShardRouting shard, ShardRoutingState state, String nodeId, long expectedShardSize) { + assertThat(shard.state(), equalTo(state)); + assertThat(shard.currentNodeId(), equalTo(nodeId)); + assertThat(shard.getExpectedShardSize(), equalTo(expectedShardSize)); + } + + private static Map createDiskUsage(Collection nodeIds, Function diskUsageCreator) { + return nodeIds.stream().collect(toMap(Function.identity(), diskUsageCreator)); + } + + private static DiskUsage createDiskUsage(String nodeId, long totalBytes, long freeBytes) { + return new DiskUsage(nodeId, nodeId, "/data", totalBytes, freeBytes); + } + public void testInitializingHasExpectedSize() { final long byteSize = randomIntBetween(0, Integer.MAX_VALUE); final ClusterInfo clusterInfo = createClusterInfoWith(new ShardId("test", "_na_", 0), byteSize); AllocationService strategy = createAllocationService(Settings.EMPTY, () -> clusterInfo); logger.info("Building initial routing table"); - var indexMetadata = IndexMetadata.builder("test") - .settings(settings(IndexVersion.current())) - .numberOfShards(1) - .numberOfReplicas(1) - .build(); + var indexMetadata = IndexMetadata.builder("test").settings(indexSettings(IndexVersion.current(), 1, 1)).build(); ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) .metadata(Metadata.builder().put(indexMetadata, false)) @@ -52,11 +185,8 @@ public void testInitializingHasExpectedSize() { logger.info("Adding one node and performing rerouting"); clusterState = strategy.reroute(clusterState, "reroute", ActionListener.noop()); - assertEquals(1, clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING)); - assertEquals( - byteSize, - shardsWithState(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING).get(0).getExpectedShardSize() - ); + assertEquals(1, clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(INITIALIZING)); + assertEquals(byteSize, shardsWithState(clusterState.getRoutingNodes(), INITIALIZING).get(0).getExpectedShardSize()); logger.info("Start the primary shard"); clusterState = startInitializingShardsAndReroute(strategy, clusterState); @@ -67,11 +197,8 @@ public void testInitializingHasExpectedSize() { clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build(); clusterState = strategy.reroute(clusterState, "reroute", ActionListener.noop()); - assertEquals(1, clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING)); - assertEquals( - byteSize, - shardsWithState(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING).get(0).getExpectedShardSize() - ); + assertEquals(1, clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(INITIALIZING)); + assertEquals(byteSize, shardsWithState(clusterState.getRoutingNodes(), INITIALIZING).get(0).getExpectedShardSize()); } public void testExpectedSizeOnMove() { @@ -79,11 +206,7 @@ public void testExpectedSizeOnMove() { final ClusterInfo clusterInfo = createClusterInfoWith(new ShardId("test", "_na_", 0), byteSize); final AllocationService allocation = createAllocationService(Settings.EMPTY, () -> clusterInfo); logger.info("creating an index with 1 shard, no replica"); - var indexMetadata = IndexMetadata.builder("test") - .settings(settings(IndexVersion.current())) - .numberOfShards(1) - .numberOfReplicas(0) - .build(); + var indexMetadata = IndexMetadata.builder("test").settings(indexSettings(IndexVersion.current(), 1, 0)).build(); ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) .metadata(Metadata.builder().put(indexMetadata, false)) .routingTable(RoutingTable.builder(TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY).addAsNew(indexMetadata)) @@ -111,7 +234,7 @@ public void testExpectedSizeOnMove() { assertThat(commandsResult.clusterState(), not(equalTo(clusterState))); clusterState = commandsResult.clusterState(); assertEquals(clusterState.getRoutingNodes().node(existingNodeId).iterator().next().state(), ShardRoutingState.RELOCATING); - assertEquals(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(), ShardRoutingState.INITIALIZING); + assertEquals(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(), INITIALIZING); assertEquals(clusterState.getRoutingNodes().node(existingNodeId).iterator().next().getExpectedShardSize(), byteSize); assertEquals(clusterState.getRoutingNodes().node(toNodeId).iterator().next().getExpectedShardSize(), byteSize); @@ -137,4 +260,8 @@ private static ClusterInfo createClusterInfoWith(ShardId shardId, long size) { Map.of() ); } + + private static ClusterInfo createClusterInfo(Map diskUsage, Map shardSizes) { + return new ClusterInfo(diskUsage, diskUsage, shardSizes, Map.of(), Map.of(), Map.of()); + } } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java index 41af35629790d..88c7dc24b4089 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java @@ -51,6 +51,7 @@ import java.util.Map; import static java.util.Collections.emptySet; +import static org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator.getExpectedShardSize; import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE; import static org.hamcrest.Matchers.containsString; @@ -459,9 +460,9 @@ public void testShardSizeAndRelocatingSize() { test_2 = ShardRoutingHelper.initialize(test_2, "node1"); test_2 = ShardRoutingHelper.moveToStarted(test_2); - assertEquals(1000L, DiskThresholdDecider.getExpectedShardSize(test_2, 0L, allocation)); - assertEquals(100L, DiskThresholdDecider.getExpectedShardSize(test_1, 0L, allocation)); - assertEquals(10L, DiskThresholdDecider.getExpectedShardSize(test_0, 0L, allocation)); + assertEquals(1000L, getExpectedShardSize(test_2, 0L, allocation)); + assertEquals(100L, getExpectedShardSize(test_1, 0L, allocation)); + assertEquals(10L, getExpectedShardSize(test_0, 0L, allocation)); RoutingNode node = RoutingNodesHelper.routingNode( "node1", @@ -484,7 +485,7 @@ public void testShardSizeAndRelocatingSize() { ); test_3 = ShardRoutingHelper.initialize(test_3, "node1"); test_3 = ShardRoutingHelper.moveToStarted(test_3); - assertEquals(0L, DiskThresholdDecider.getExpectedShardSize(test_3, 0L, allocation)); + assertEquals(0L, getExpectedShardSize(test_3, 0L, allocation)); boolean primary = randomBoolean(); ShardRouting other_0 = ShardRouting.newUnassigned( @@ -725,10 +726,10 @@ public void testSizeShrinkIndex() { ShardRouting.Role.DEFAULT ); test_3 = ShardRoutingHelper.initialize(test_3, "node1"); - assertEquals(500L, DiskThresholdDecider.getExpectedShardSize(test_3, 0L, allocation)); - assertEquals(500L, DiskThresholdDecider.getExpectedShardSize(test_2, 0L, allocation)); - assertEquals(100L, DiskThresholdDecider.getExpectedShardSize(test_1, 0L, allocation)); - assertEquals(10L, DiskThresholdDecider.getExpectedShardSize(test_0, 0L, allocation)); + assertEquals(500L, getExpectedShardSize(test_3, 0L, allocation)); + assertEquals(500L, getExpectedShardSize(test_2, 0L, allocation)); + assertEquals(100L, getExpectedShardSize(test_1, 0L, allocation)); + assertEquals(10L, getExpectedShardSize(test_0, 0L, allocation)); ShardRouting target = ShardRouting.newUnassigned( new ShardId(new Index("target", "5678"), 0), @@ -737,7 +738,7 @@ public void testSizeShrinkIndex() { new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"), ShardRouting.Role.DEFAULT ); - assertEquals(1110L, DiskThresholdDecider.getExpectedShardSize(target, 0L, allocation)); + assertEquals(1110L, getExpectedShardSize(target, 0L, allocation)); ShardRouting target2 = ShardRouting.newUnassigned( new ShardId(new Index("target2", "9101112"), 0), @@ -746,7 +747,7 @@ public void testSizeShrinkIndex() { new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"), ShardRouting.Role.DEFAULT ); - assertEquals(110L, DiskThresholdDecider.getExpectedShardSize(target2, 0L, allocation)); + assertEquals(110L, getExpectedShardSize(target2, 0L, allocation)); target2 = ShardRouting.newUnassigned( new ShardId(new Index("target2", "9101112"), 1), @@ -755,7 +756,7 @@ public void testSizeShrinkIndex() { new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"), ShardRouting.Role.DEFAULT ); - assertEquals(1000L, DiskThresholdDecider.getExpectedShardSize(target2, 0L, allocation)); + assertEquals(1000L, getExpectedShardSize(target2, 0L, allocation)); // check that the DiskThresholdDecider still works even if the source index has been deleted ClusterState clusterStateWithMissingSourceIndex = ClusterState.builder(clusterState) @@ -765,8 +766,8 @@ public void testSizeShrinkIndex() { allocationService.reroute(clusterState, "foo", ActionListener.noop()); RoutingAllocation allocationWithMissingSourceIndex = new RoutingAllocation(null, clusterStateWithMissingSourceIndex, info, null, 0); - assertEquals(42L, DiskThresholdDecider.getExpectedShardSize(target, 42L, allocationWithMissingSourceIndex)); - assertEquals(42L, DiskThresholdDecider.getExpectedShardSize(target2, 42L, allocationWithMissingSourceIndex)); + assertEquals(42L, getExpectedShardSize(target, 42L, allocationWithMissingSourceIndex)); + assertEquals(42L, getExpectedShardSize(target2, 42L, allocationWithMissingSourceIndex)); } public void testDiskUsageWithRelocations() { diff --git a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java index fc8eedbe1ca75..7eb3cca18efd0 100644 --- a/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java +++ b/x-pack/plugin/autoscaling/src/main/java/org/elasticsearch/xpack/autoscaling/storage/ReactiveStorageDeciderService.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodeFilters; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RoutingNode; @@ -670,7 +671,7 @@ long sizeOf(ShardRouting shard) { } private long getExpectedShardSize(ShardRouting shard) { - return DiskThresholdDecider.getExpectedShardSize(shard, 0L, info, shardSizeInfo, state.metadata(), state.routingTable()); + return ExpectedShardSizeEstimator.getExpectedShardSize(shard, 0L, info, shardSizeInfo, state.metadata(), state.routingTable()); } long unmovableSize(String nodeId, Collection shards) { diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java index f95b44f1d2e22..fbac3d339e902 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotAllocator.java @@ -30,7 +30,6 @@ import org.elasticsearch.cluster.routing.allocation.NodeAllocationResult; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; -import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; @@ -61,6 +60,7 @@ import java.util.concurrent.ConcurrentMap; import static java.util.stream.Collectors.toSet; +import static org.elasticsearch.cluster.routing.ExpectedShardSizeEstimator.getExpectedShardSize; import static org.elasticsearch.gateway.ReplicaShardAllocator.augmentExplanationsWithStoreInfo; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SNAPSHOT_PARTIAL_SETTING; import static org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots.SNAPSHOT_INDEX_ID_SETTING; @@ -198,7 +198,7 @@ public void allocateUnassigned( unassignedAllocationHandler.initialize( allocateUnassignedDecision.getTargetNode().getId(), allocateUnassignedDecision.getAllocationId(), - DiskThresholdDecider.getExpectedShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation), + getExpectedShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation), allocation.changes() ); } else { From 11cb81a09ad82004f726cb984a34b138ec34f06a Mon Sep 17 00:00:00 2001 From: Fabio Busatto <52658645+bytebilly@users.noreply.github.com> Date: Wed, 8 Nov 2023 13:39:49 +0100 Subject: [PATCH 003/513] Update SVG reference for starts_with ESQL docs (#101909) --- docs/reference/esql/functions/starts_with.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/esql/functions/starts_with.asciidoc b/docs/reference/esql/functions/starts_with.asciidoc index 38cee79ea63f8..f98a76ef68206 100644 --- a/docs/reference/esql/functions/starts_with.asciidoc +++ b/docs/reference/esql/functions/starts_with.asciidoc @@ -2,7 +2,7 @@ [[esql-starts_with]] === `STARTS_WITH` [.text-center] -image::esql/functions/signature/ends_with.svg[Embedded,opts=inline] +image::esql/functions/signature/starts_with.svg[Embedded,opts=inline] Returns a boolean that indicates whether a keyword string starts with another string: From 40993874410ddfc101806d04ddd2640be8517cb0 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 8 Nov 2023 15:13:36 +0000 Subject: [PATCH 004/513] Longer timeout & more logging in testClusterHealthRestCancellation (#101911) Relates #100062 --- .../http/ClusterHealthRestCancellationIT.java | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/ClusterHealthRestCancellationIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/ClusterHealthRestCancellationIT.java index cf76d86c9298f..755bbce93c95b 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/ClusterHealthRestCancellationIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/ClusterHealthRestCancellationIT.java @@ -18,9 +18,11 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; +import org.elasticsearch.test.junit.annotations.TestIssueLogging; import java.util.concurrent.CancellationException; import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.TimeUnit; import static org.elasticsearch.action.support.ActionTestUtils.wrapAsRestResponseListener; import static org.elasticsearch.test.TaskAssertions.assertAllCancellableTasksAreCancelled; @@ -28,6 +30,10 @@ public class ClusterHealthRestCancellationIT extends HttpSmokeTestCase { + @TestIssueLogging( + issueUrl = "https://github.com/elastic/elasticsearch/issues/100062", + value = "org.elasticsearch.test.TaskAssertions:TRACE" + ) public void testClusterHealthRestCancellation() throws Exception { final var barrier = new CyclicBarrier(2); @@ -37,7 +43,18 @@ public void testClusterHealthRestCancellation() throws Exception { @Override public ClusterState execute(ClusterState currentState) { safeAwait(barrier); - safeAwait(barrier); + // safeAwait(barrier); + + // temporarily lengthen timeout on safeAwait while investigating #100062 + try { + barrier.await(60, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new AssertionError("unexpected", e); + } catch (Exception e) { + throw new AssertionError("unexpected", e); + } + return currentState; } From 772474d52abca4f0dc6a9c202f43102c94e5e3c6 Mon Sep 17 00:00:00 2001 From: John Verwolf Date: Wed, 8 Nov 2023 07:17:06 -0800 Subject: [PATCH 005/513] Add metrics to the shared blob cache (#101577) Adds pattern for metrics in the shared blob cache. --- docs/changelog/101577.yaml | 5 ++ .../blobcache/BlobCacheMetrics.java | 48 ++++++++++++ .../shared/SharedBlobCacheService.java | 36 +++++++-- .../shared/SharedBlobCacheServiceTests.java | 73 ++++++++++++++++--- .../SearchableSnapshots.java | 4 +- .../AbstractSearchableSnapshotsTestCase.java | 10 ++- .../store/input/FrozenIndexInputTests.java | 4 +- 7 files changed, 160 insertions(+), 20 deletions(-) create mode 100644 docs/changelog/101577.yaml create mode 100644 x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/BlobCacheMetrics.java diff --git a/docs/changelog/101577.yaml b/docs/changelog/101577.yaml new file mode 100644 index 0000000000000..e485fd3811cb6 --- /dev/null +++ b/docs/changelog/101577.yaml @@ -0,0 +1,5 @@ +pr: 101577 +summary: Add metrics to the shared blob cache +area: Search +type: enhancement +issues: [] diff --git a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/BlobCacheMetrics.java b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/BlobCacheMetrics.java new file mode 100644 index 0000000000000..b4830ca97938f --- /dev/null +++ b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/BlobCacheMetrics.java @@ -0,0 +1,48 @@ +/* + * 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.blobcache; + +import org.elasticsearch.telemetry.TelemetryProvider; +import org.elasticsearch.telemetry.metric.LongCounter; +import org.elasticsearch.telemetry.metric.LongHistogram; +import org.elasticsearch.telemetry.metric.MeterRegistry; + +public class BlobCacheMetrics { + private final LongCounter cacheMissCounter; + private final LongHistogram cacheMissLoadTimes; + + public BlobCacheMetrics(MeterRegistry meterRegistry) { + this( + meterRegistry.registerLongCounter( + "elasticsearch.blob_cache.miss_that_triggered_read", + "The number of times there was a cache miss that triggered a read from the blob store", + "count" + ), + meterRegistry.registerLongHistogram( + "elasticsearch.blob_cache.cache_miss_load_times", + "The timing data for populating entries in the blob store resulting from a cache miss.", + "count" + ) + ); + } + + BlobCacheMetrics(LongCounter cacheMissCounter, LongHistogram cacheMissLoadTimes) { + this.cacheMissCounter = cacheMissCounter; + this.cacheMissLoadTimes = cacheMissLoadTimes; + } + + public static BlobCacheMetrics NOOP = new BlobCacheMetrics(TelemetryProvider.NOOP.getMeterRegistry()); + + public LongCounter getCacheMissCounter() { + return cacheMissCounter; + } + + public LongHistogram getCacheMissLoadTimes() { + return cacheMissLoadTimes; + } +} diff --git a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java index 2538ee613b96f..7740e500344f2 100644 --- a/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java +++ b/x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java @@ -13,6 +13,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.RefCountingListener; +import org.elasticsearch.blobcache.BlobCacheMetrics; import org.elasticsearch.blobcache.BlobCacheUtils; import org.elasticsearch.blobcache.common.ByteRange; import org.elasticsearch.blobcache.common.SparseFileTracker; @@ -298,8 +299,16 @@ private CacheEntry(T chunk) { private final LongAdder evictCount = new LongAdder(); - public SharedBlobCacheService(NodeEnvironment environment, Settings settings, ThreadPool threadPool, String ioExecutor) { - this(environment, settings, threadPool, ioExecutor, ioExecutor); + private final BlobCacheMetrics blobCacheMetrics; + + public SharedBlobCacheService( + NodeEnvironment environment, + Settings settings, + ThreadPool threadPool, + String ioExecutor, + BlobCacheMetrics blobCacheMetrics + ) { + this(environment, settings, threadPool, ioExecutor, ioExecutor, blobCacheMetrics); } public SharedBlobCacheService( @@ -307,7 +316,8 @@ public SharedBlobCacheService( Settings settings, ThreadPool threadPool, String ioExecutor, - String bulkExecutor + String bulkExecutor, + BlobCacheMetrics blobCacheMetrics ) { this.threadPool = threadPool; this.ioExecutor = threadPool.executor(ioExecutor); @@ -347,6 +357,8 @@ public SharedBlobCacheService( this.rangeSize = SHARED_CACHE_RANGE_SIZE_SETTING.get(settings); this.recoveryRangeSize = SHARED_CACHE_RECOVERY_RANGE_SIZE_SETTING.get(settings); + + this.blobCacheMetrics = blobCacheMetrics; } public static long calculateCacheSize(Settings settings, long totalFsSize) { @@ -795,6 +807,20 @@ public int populateAndRead( final RangeAvailableHandler reader, final RangeMissingHandler writer ) throws Exception { + // We are interested in the total time that the system spends when fetching a result (including time spent queuing), so we start + // our measurement here. + final long startTime = threadPool.relativeTimeInMillis(); + RangeMissingHandler writerInstrumentationDecorator = ( + SharedBytes.IO channel, + int channelPos, + int relativePos, + int length, + IntConsumer progressUpdater) -> { + writer.fillCacheRange(channel, channelPos, relativePos, length, progressUpdater); + var elapsedTime = threadPool.relativeTimeInMillis() - startTime; + SharedBlobCacheService.this.blobCacheMetrics.getCacheMissLoadTimes().record(elapsedTime); + SharedBlobCacheService.this.blobCacheMetrics.getCacheMissCounter().increment(); + }; if (rangeToRead.isEmpty()) { // nothing to read, skip return 0; @@ -802,9 +828,9 @@ public int populateAndRead( final int startRegion = getRegion(rangeToWrite.start()); final int endRegion = getEndingRegion(rangeToWrite.end()); if (startRegion == endRegion) { - return readSingleRegion(rangeToWrite, rangeToRead, reader, writer, startRegion); + return readSingleRegion(rangeToWrite, rangeToRead, reader, writerInstrumentationDecorator, startRegion); } - return readMultiRegions(rangeToWrite, rangeToRead, reader, writer, startRegion, endRegion); + return readMultiRegions(rangeToWrite, rangeToRead, reader, writerInstrumentationDecorator, startRegion, endRegion); } private int readSingleRegion( diff --git a/x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java b/x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java index 300f9f1d8efa9..cd9bb5b5934c8 100644 --- a/x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java +++ b/x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java @@ -10,6 +10,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.blobcache.BlobCacheMetrics; import org.elasticsearch.blobcache.common.ByteRange; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.common.settings.Setting; @@ -66,7 +67,13 @@ public void testBasicEviction() throws IOException { final DeterministicTaskQueue taskQueue = new DeterministicTaskQueue(); try ( NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings)); - var cacheService = new SharedBlobCacheService<>(environment, settings, taskQueue.getThreadPool(), ThreadPool.Names.GENERIC) + var cacheService = new SharedBlobCacheService<>( + environment, + settings, + taskQueue.getThreadPool(), + ThreadPool.Names.GENERIC, + BlobCacheMetrics.NOOP + ) ) { final var cacheKey = generateCacheKey(); assertEquals(5, cacheService.freeRegionCount()); @@ -126,7 +133,13 @@ public void testAutoEviction() throws IOException { final DeterministicTaskQueue taskQueue = new DeterministicTaskQueue(); try ( NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings)); - var cacheService = new SharedBlobCacheService<>(environment, settings, taskQueue.getThreadPool(), ThreadPool.Names.GENERIC) + var cacheService = new SharedBlobCacheService<>( + environment, + settings, + taskQueue.getThreadPool(), + ThreadPool.Names.GENERIC, + BlobCacheMetrics.NOOP + ) ) { final var cacheKey = generateCacheKey(); assertEquals(2, cacheService.freeRegionCount()); @@ -164,7 +177,13 @@ public void testForceEviction() throws IOException { final DeterministicTaskQueue taskQueue = new DeterministicTaskQueue(); try ( NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings)); - var cacheService = new SharedBlobCacheService<>(environment, settings, taskQueue.getThreadPool(), ThreadPool.Names.GENERIC) + var cacheService = new SharedBlobCacheService<>( + environment, + settings, + taskQueue.getThreadPool(), + ThreadPool.Names.GENERIC, + BlobCacheMetrics.NOOP + ) ) { final var cacheKey1 = generateCacheKey(); final var cacheKey2 = generateCacheKey(); @@ -192,7 +211,13 @@ public void testForceEvictResponse() throws IOException { final DeterministicTaskQueue taskQueue = new DeterministicTaskQueue(); try ( NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings)); - var cacheService = new SharedBlobCacheService<>(environment, settings, taskQueue.getThreadPool(), ThreadPool.Names.GENERIC) + var cacheService = new SharedBlobCacheService<>( + environment, + settings, + taskQueue.getThreadPool(), + ThreadPool.Names.GENERIC, + BlobCacheMetrics.NOOP + ) ) { final var cacheKey1 = generateCacheKey(); final var cacheKey2 = generateCacheKey(); @@ -219,7 +244,13 @@ public void testDecay() throws IOException { final DeterministicTaskQueue taskQueue = new DeterministicTaskQueue(); try ( NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings)); - var cacheService = new SharedBlobCacheService<>(environment, settings, taskQueue.getThreadPool(), ThreadPool.Names.GENERIC) + var cacheService = new SharedBlobCacheService<>( + environment, + settings, + taskQueue.getThreadPool(), + ThreadPool.Names.GENERIC, + BlobCacheMetrics.NOOP + ) ) { final var cacheKey1 = generateCacheKey(); final var cacheKey2 = generateCacheKey(); @@ -284,7 +315,13 @@ public void testGetMultiThreaded() throws IOException { Set files = randomSet(1, 10, () -> randomAlphaOfLength(5)); try ( NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings)); - var cacheService = new SharedBlobCacheService(environment, settings, threadPool, ThreadPool.Names.GENERIC) + var cacheService = new SharedBlobCacheService( + environment, + settings, + threadPool, + ThreadPool.Names.GENERIC, + BlobCacheMetrics.NOOP + ) ) { CyclicBarrier ready = new CyclicBarrier(threads); List threadList = IntStream.range(0, threads).mapToObj(no -> { @@ -364,7 +401,14 @@ public void execute(Runnable command) { try ( NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings)); - var cacheService = new SharedBlobCacheService<>(environment, settings, threadPool, ThreadPool.Names.GENERIC, "bulk") + var cacheService = new SharedBlobCacheService<>( + environment, + settings, + threadPool, + ThreadPool.Names.GENERIC, + "bulk", + BlobCacheMetrics.NOOP + ) ) { { final var cacheKey = generateCacheKey(); @@ -418,7 +462,14 @@ public ExecutorService executor(String name) { try ( NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings)); - var cacheService = new SharedBlobCacheService<>(environment, settings, threadPool, ThreadPool.Names.GENERIC, "bulk") + var cacheService = new SharedBlobCacheService<>( + environment, + settings, + threadPool, + ThreadPool.Names.GENERIC, + "bulk", + BlobCacheMetrics.NOOP + ) ) { final long size = size(randomIntBetween(1, 100)); @@ -620,7 +671,8 @@ public void testCacheSizeChanges() throws IOException { environment, settings, taskQueue.getThreadPool(), - ThreadPool.Names.GENERIC + ThreadPool.Names.GENERIC, + BlobCacheMetrics.NOOP ) ) { assertEquals(val1.getBytes(), cacheService.getStats().size()); @@ -637,7 +689,8 @@ public void testCacheSizeChanges() throws IOException { environment, settings, taskQueue.getThreadPool(), - ThreadPool.Names.GENERIC + ThreadPool.Names.GENERIC, + BlobCacheMetrics.NOOP ) ) { assertEquals(val2.getBytes(), cacheService.getStats().size()); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java index 98f6da9ba6a58..45c38e52ad9c3 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java @@ -12,6 +12,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.blobcache.BlobCacheMetrics; import org.elasticsearch.blobcache.shared.SharedBlobCacheService; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -328,7 +329,8 @@ public Collection createComponents(PluginServices services) { nodeEnvironment, settings, threadPool, - SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME + SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME, + new BlobCacheMetrics(services.telemetryProvider().getMeterRegistry()) ); this.frozenCacheService.set(sharedBlobCacheService); components.add(cacheService); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/AbstractSearchableSnapshotsTestCase.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/AbstractSearchableSnapshotsTestCase.java index 0bc339f17e190..a599aeaeada71 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/AbstractSearchableSnapshotsTestCase.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/AbstractSearchableSnapshotsTestCase.java @@ -14,6 +14,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; +import org.elasticsearch.blobcache.BlobCacheMetrics; import org.elasticsearch.blobcache.common.ByteRange; import org.elasticsearch.blobcache.shared.SharedBlobCacheService; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -143,7 +144,8 @@ protected SharedBlobCacheService defaultFrozenCacheService() { nodeEnvironment, Settings.EMPTY, threadPool, - SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME + SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME, + BlobCacheMetrics.NOOP ); } @@ -165,7 +167,8 @@ protected SharedBlobCacheService randomFrozenCacheService() { singlePathNodeEnvironment, cacheSettings.build(), threadPool, - SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME + SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME, + BlobCacheMetrics.NOOP ); } @@ -189,7 +192,8 @@ protected SharedBlobCacheService createFrozenCacheService(final ByteSi .put(SharedBlobCacheService.SHARED_CACHE_RANGE_SIZE_SETTING.getKey(), cacheRangeSize) .build(), threadPool, - SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME + SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME, + BlobCacheMetrics.NOOP ); } diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInputTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInputTests.java index f6f494cd46099..ca4ad51ecb819 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInputTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInputTests.java @@ -9,6 +9,7 @@ import org.apache.lucene.store.IndexInput; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.blobcache.BlobCacheMetrics; import org.elasticsearch.blobcache.shared.SharedBlobCacheService; import org.elasticsearch.blobcache.shared.SharedBytes; import org.elasticsearch.common.settings.Settings; @@ -108,7 +109,8 @@ public void testRandomReads() throws IOException { nodeEnvironment, settings, threadPool, - SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME + SearchableSnapshots.CACHE_FETCH_ASYNC_THREAD_POOL_NAME, + BlobCacheMetrics.NOOP ); CacheService cacheService = randomCacheService(); TestSearchableSnapshotDirectory directory = new TestSearchableSnapshotDirectory( From 94287eee1976f9a81e0fc1f3b70e4d972f5cfc53 Mon Sep 17 00:00:00 2001 From: Nastasha Solomon <79124755+nastasha-solomon@users.noreply.github.com> Date: Wed, 8 Nov 2023 11:04:55 -0500 Subject: [PATCH 006/513] [DOCS] Adds AI assistant functionality to Using ES|QL page (#101918) Co-authored-by: Abdon Pijpelink --- docs/reference/esql/esql-using.asciidoc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/reference/esql/esql-using.asciidoc b/docs/reference/esql/esql-using.asciidoc index dbab521ead4d1..235c7defe559b 100644 --- a/docs/reference/esql/esql-using.asciidoc +++ b/docs/reference/esql/esql-using.asciidoc @@ -9,8 +9,8 @@ Using {esql} in {kib} to query and aggregate your data, create visualizations, and set up alerts. <>:: -Using {esql} in {elastic-sec} to investigate events in Timeline and create -detection rules. +Using {esql} in {elastic-sec} to investigate events in Timeline, create +detection rules, and build {esql} queries using Elastic AI Assistant. <>:: Using the <> to list and cancel {esql} queries. @@ -18,4 +18,4 @@ Using the <> to list and cancel {esql} queries. include::esql-rest.asciidoc[] include::esql-kibana.asciidoc[] include::esql-security-solution.asciidoc[] -include::task-management.asciidoc[] \ No newline at end of file +include::task-management.asciidoc[] From 60c7a661bf484597e37e17108ac886d27834c655 Mon Sep 17 00:00:00 2001 From: Przemyslaw Gomulka Date: Wed, 8 Nov 2023 17:38:22 +0100 Subject: [PATCH 007/513] Add Tracing APM integration test (#101887) this commit adds an integration test where ES is run with apm-java-agent and sending traces. Traces are then received by a fake http server and could be asserted on. --- .../{ApmIT.java => MetricsApmIT.java} | 4 +- .../apmintegration/RecordingApmServer.java | 2 - .../test/apmintegration/TracesApmIT.java | 146 ++++++++++++++++++ 3 files changed, 149 insertions(+), 3 deletions(-) rename test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/{ApmIT.java => MetricsApmIT.java} (97%) create mode 100644 test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/TracesApmIT.java diff --git a/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/ApmIT.java b/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/MetricsApmIT.java similarity index 97% rename from test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/ApmIT.java rename to test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/MetricsApmIT.java index b75ce619e54ea..0c33cd4984d86 100644 --- a/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/ApmIT.java +++ b/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/MetricsApmIT.java @@ -35,7 +35,7 @@ import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.equalTo; -public class ApmIT extends ESRestTestCase { +public class MetricsApmIT extends ESRestTestCase { private static final XContentProvider.FormatProvider XCONTENT = XContentProvider.provider().getJsonXContent(); @ClassRule @@ -83,6 +83,8 @@ public void testApmIntegration() throws Exception { Consumer messageConsumer = (String message) -> { var apmMessage = parseMap(message); if (isElasticsearchMetric(apmMessage)) { + logger.info("Apm metric message received: " + message); + var metricset = (Map) apmMessage.get("metricset"); var samples = (Map) metricset.get("samples"); diff --git a/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/RecordingApmServer.java b/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/RecordingApmServer.java index c3a8df2c4b150..542f8a8d7d2f2 100644 --- a/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/RecordingApmServer.java +++ b/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/RecordingApmServer.java @@ -60,7 +60,6 @@ private Thread consumerThread() { try { String msg = received.poll(1L, TimeUnit.SECONDS); if (msg != null && msg.isEmpty() == false) { - logger.info("APM server received: " + msg); consumer.accept(msg); } @@ -84,7 +83,6 @@ private void handle(HttpExchange exchange) throws IOException { try (InputStream requestBody = exchange.getRequestBody()) { if (requestBody != null) { var read = readJsonMessages(requestBody); - read.forEach(s -> logger.debug(s)); received.addAll(read); } } diff --git a/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/TracesApmIT.java b/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/TracesApmIT.java new file mode 100644 index 0000000000000..79816114cc38f --- /dev/null +++ b/test/external-modules/apm-integration/src/javaRestTest/java/org/elasticsearch/test/apmintegration/TracesApmIT.java @@ -0,0 +1,146 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.test.apmintegration; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.cluster.local.distribution.DistributionType; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.spi.XContentProvider; +import org.hamcrest.Matcher; +import org.hamcrest.StringDescription; +import org.junit.ClassRule; +import org.junit.Rule; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.equalTo; + +public class TracesApmIT extends ESRestTestCase { + private static final XContentProvider.FormatProvider XCONTENT = XContentProvider.provider().getJsonXContent(); + final String traceIdValue = "0af7651916cd43dd8448eb211c80319c"; + final String traceParentValue = "00-" + traceIdValue + "-b7ad6b7169203331-01"; + + @ClassRule + public static RecordingApmServer mockApmServer = new RecordingApmServer(); + + @Rule + public ElasticsearchCluster cluster = ElasticsearchCluster.local() + .distribution(DistributionType.INTEG_TEST) + .module("test-apm-integration") + .module("apm") + .setting("telemetry.metrics.enabled", "false") + .setting("tracing.apm.enabled", "true") + .setting("tracing.apm.agent.metrics_interval", "1s") + .setting("tracing.apm.agent.server_url", "http://127.0.0.1:" + mockApmServer.getPort()) + .build(); + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + @SuppressWarnings("unchecked") + public void testApmIntegration() throws Exception { + Set>> assertions = new HashSet<>( + Set.of(allTrue(transactionValue("name", equalTo("GET /_nodes/stats")), transactionValue("trace_id", equalTo(traceIdValue)))) + ); + + CountDownLatch finished = new CountDownLatch(1); + + // a consumer that will remove the assertions from a map once it matched + Consumer messageConsumer = (String message) -> { + var apmMessage = parseMap(message); + if (isTransactionTraceMessage(apmMessage)) { + logger.info("Apm transaction message received: " + message); + assertions.removeIf(e -> e.test(apmMessage)); + } + + if (assertions.isEmpty()) { + finished.countDown(); + } + }; + + mockApmServer.addMessageConsumer(messageConsumer); + + Request nodeStatsRequest = new Request("GET", "/_nodes/stats"); + + nodeStatsRequest.setOptions(RequestOptions.DEFAULT.toBuilder().addHeader(Task.TRACE_PARENT_HTTP_HEADER, traceParentValue).build()); + + client().performRequest(nodeStatsRequest); + + finished.await(30, TimeUnit.SECONDS); + assertThat(assertions, equalTo(Collections.emptySet())); + } + + private boolean isTransactionTraceMessage(Map apmMessage) { + return apmMessage.containsKey("transaction"); + } + + @SuppressWarnings("unchecked") + private Predicate> allTrue(Predicate>... predicates) { + var allTrueTest = Arrays.stream(predicates).reduce(v -> true, Predicate::and); + return new Predicate<>() { + @Override + public boolean test(Map map) { + return allTrueTest.test(map); + } + + @Override + public String toString() { + return Arrays.stream(predicates).map(p -> p.toString()).collect(Collectors.joining(" and ")); + } + }; + } + + @SuppressWarnings("unchecked") + private Predicate> transactionValue(String path, Matcher expected) { + + return new Predicate<>() { + @Override + public boolean test(Map map) { + var transaction = (Map) map.get("transaction"); + var value = XContentMapValues.extractValue(path, transaction); + return expected.matches((T) value); + } + + @Override + public String toString() { + StringDescription matcherDescription = new StringDescription(); + expected.describeTo(matcherDescription); + return path + " " + matcherDescription; + } + }; + } + + private Map parseMap(String message) { + try (XContentParser parser = XCONTENT.XContent().createParser(XContentParserConfiguration.EMPTY, message)) { + return parser.map(); + } catch (IOException e) { + fail(e); + return Collections.emptyMap(); + } + } + +} From 481ebd2e2122a312ab7c8d9a6a2e82aa64a24799 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Istv=C3=A1n=20Zolt=C3=A1n=20Szab=C3=B3?= Date: Wed, 8 Nov 2023 17:57:57 +0100 Subject: [PATCH 008/513] [DOCS] Improves readability of PUT trained models API docs page (#101880) * [DOCS] Improves readability of PUT trained models API docs page. * [DOCS] Fixes URLs. --- .../apis/put-trained-models.asciidoc | 1244 +++-------------- 1 file changed, 213 insertions(+), 1031 deletions(-) diff --git a/docs/reference/ml/trained-models/apis/put-trained-models.asciidoc b/docs/reference/ml/trained-models/apis/put-trained-models.asciidoc index 7da46e13a8ce4..5696a032b165c 100644 --- a/docs/reference/ml/trained-models/apis/put-trained-models.asciidoc +++ b/docs/reference/ml/trained-models/apis/put-trained-models.asciidoc @@ -443,121 +443,8 @@ include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] + -.Properties of tokenization -[%collapsible%open] -====== -`bert`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert] -+ -.Properties of bert -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-with-special-tokens] -======= -`roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta] -+ -.Properties of roberta -[%collapsible%open] -======= -`add_prefix_space`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-add-prefix-space] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`mpnet`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet] -+ -.Properties of mpnet -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet-with-special-tokens] -======= -`xlm_roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-xlm-roberta] -+ -.Properties of xlm_roberta -[%collapsible%open] -======= -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`bert_ja`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja] -+ -.Properties of bert_ja -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja-with-special-tokens] -======= -====== +Refer to <> to review the properties of the +`tokenization` object. ===== `ner`::: @@ -582,121 +469,8 @@ include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] + -.Properties of tokenization -[%collapsible%open] -====== -`bert`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert] -+ -.Properties of bert -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-with-special-tokens] -======= -`roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta] -+ -.Properties of roberta -[%collapsible%open] -======= -`add_prefix_space`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-add-prefix-space] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`mpnet`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet] -+ -.Properties of mpnet -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet-with-special-tokens] -======= -`xlm_roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-xlm-roberta] -+ -.Properties of xlm_roberta -[%collapsible%open] -======= -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`bert_ja`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja] -+ -.Properties of bert_ja -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja-with-special-tokens] -======= -====== +Refer to <> to review the +properties of the `tokenization` object. ===== `pass_through`::: @@ -714,738 +488,121 @@ include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] + -.Properties of tokenization -[%collapsible%open] -====== -`bert`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert] -+ -.Properties of bert -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] +Refer to <> to review the properties of the +`tokenization` object. +===== -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-with-special-tokens] -======= -`roberta`:::: +`question_answering`::: (Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-question-answering] + -.Properties of roberta +.Properties of question_answering inference [%collapsible%open] -======= -`add_prefix_space`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-add-prefix-space] - -`max_sequence_length`:::: +===== +`max_answer_length`:::: (Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] +The maximum amount of words in the answer. Defaults to `15`. -`truncate`:::: +`results_field`:::: (Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`mpnet`:::: +`tokenization`:::: (Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] + -.Properties of mpnet -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet-with-special-tokens] -======= -`xlm_roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-xlm-roberta] +Recommended to set `max_sentence_length` to `386` with `128` of `span` and set +`truncate` to `none`. + -.Properties of xlm_roberta -[%collapsible%open] -======= -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] +Refer to <> to review the properties of the +`tokenization` object. +===== -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`bert_ja`:::: +`regression`::: (Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja] -+ -.Properties of bert_ja -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja-with-special-tokens] -======= -====== -===== - -`question_answering`::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-question-answering] -+ -.Properties of question_answering inference -[%collapsible%open] -===== -`max_answer_length`:::: -(Optional, integer) -The maximum amount of words in the answer. Defaults to `15`. - -`results_field`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] - -`tokenization`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] -+ -Recommended to set `max_sentence_length` to `386` with `128` of `span` and set -`truncate` to `none`. -+ -.Properties of tokenization -[%collapsible%open] -====== -`bert`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert] -+ -.Properties of bert -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-with-special-tokens] -======= -`roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta] -+ -.Properties of roberta -[%collapsible%open] -======= -`add_prefix_space`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-add-prefix-space] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`mpnet`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet] -+ -.Properties of mpnet -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet-with-special-tokens] -======= -`xlm_roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-xlm-roberta] -+ -.Properties of xlm_roberta -[%collapsible%open] -======= -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`bert_ja`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja] -+ -.Properties of bert_ja -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja-with-special-tokens] -======= -====== -===== - -`regression`::: -(Optional, object) -Regression configuration for inference. +Regression configuration for inference. + .Properties of regression inference -[%collapsible%open] -===== -`num_top_feature_importance_values`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-regression-num-top-feature-importance-values] - -`results_field`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] -===== - -`text_classification`::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-classification] -+ -.Properties of text_classification inference -[%collapsible%open] -===== -`classification_labels`:::: -(Optional, string) An array of classification labels. - -`num_top_classes`:::: -(Optional, integer) -Specifies the number of top class predictions to return. Defaults to all classes (-1). - -`results_field`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] - -`tokenization`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] -+ -.Properties of tokenization -[%collapsible%open] -====== -`bert`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert] -+ -.Properties of bert -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-with-special-tokens] -======= -`roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta] -+ -.Properties of roberta -[%collapsible%open] -======= -`add_prefix_space`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-add-prefix-space] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`mpnet`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet] -+ -.Properties of mpnet -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet-with-special-tokens] -======= -`xlm_roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-xlm-roberta] -+ -.Properties of xlm_roberta -[%collapsible%open] -======= -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`bert_ja`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja] -+ -.Properties of bert_ja -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja-with-special-tokens] -======= -====== -===== -`text_embedding`::: -(Object, optional) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-embedding] -+ -.Properties of text_embedding inference -[%collapsible%open] -===== -`embedding_size`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-embedding-size] - -`results_field`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] - -`tokenization`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] -+ -.Properties of tokenization -[%collapsible%open] -====== -`bert`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert] -+ -.Properties of bert -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-with-special-tokens] -======= -`roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta] -+ -.Properties of roberta -[%collapsible%open] -======= -`add_prefix_space`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-add-prefix-space] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`mpnet`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet] -+ -.Properties of mpnet -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet-with-special-tokens] -======= -`xlm_roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-xlm-roberta] -+ -.Properties of xlm_roberta -[%collapsible%open] -======= -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`bert_ja`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja] -+ -.Properties of bert_ja -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja-with-special-tokens] -======= -====== -===== -`text_similarity`:::: -(Object, optional) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-similarity] -+ -.Properties of text_similarity inference -[%collapsible%open] -===== -`span_score_combination_function`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-similarity-span-score-func] - -`tokenization`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] -+ -.Properties of tokenization -[%collapsible%open] -====== -`bert`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert] -+ -.Properties of bert -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-with-special-tokens] -======= -`roberta`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta] -+ -.Properties of roberta -[%collapsible%open] -======= -`add_prefix_space`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-add-prefix-space] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] - -`truncate`:::: -(Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] - -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`mpnet`:::: -(Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet] -+ -.Properties of mpnet -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] - -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] - -`span`:::: +[%collapsible%open] +===== +`num_top_feature_importance_values`:::: (Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-regression-num-top-feature-importance-values] -`truncate`:::: +`results_field`:::: (Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] +===== -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet-with-special-tokens] -======= -`xlm_roberta`:::: +`text_classification`::: (Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-xlm-roberta] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-classification] + -.Properties of xlm_roberta +.Properties of text_classification inference [%collapsible%open] -======= -`max_sequence_length`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] +===== +`classification_labels`:::: +(Optional, string) An array of classification labels. -`span`:::: +`num_top_classes`:::: (Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] +Specifies the number of top class predictions to return. Defaults to all classes +(-1). -`truncate`:::: +`results_field`:::: (Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`bert_ja`:::: +`tokenization`:::: (Optional, object) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] + -.Properties of bert_ja -[%collapsible%open] -======= -`do_lower_case`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] +Refer to <> to review the properties of the +`tokenization` object. +===== -`max_sequence_length`:::: +`text_embedding`::: +(Object, optional) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-embedding] ++ +.Properties of text_embedding inference +[%collapsible%open] +===== +`embedding_size`:::: (Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-embedding-size] -`span`:::: -(Optional, integer) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] +`results_field`:::: +(Optional, string) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] + +`tokenization`:::: +(Optional, object) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] ++ +Refer to <> to review the properties of the +`tokenization` object. +===== -`truncate`:::: +`text_similarity`:::: +(Object, optional) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-similarity] ++ +.Properties of text_similarity inference +[%collapsible%open] +===== +`span_score_combination_function`:::: (Optional, string) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-text-similarity-span-score-func] -`with_special_tokens`:::: -(Optional, boolean) -include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja-with-special-tokens] -======= -====== +`tokenization`:::: +(Optional, object) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] ++ +Refer to <> to review the properties of the +`tokenization` object. ===== + `zero_shot_classification`::: (Object, optional) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-zero-shot-classification] @@ -1477,190 +634,215 @@ include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-results-field] (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization] + -.Properties of tokenization +Refer to <> to review the properties of the +`tokenization` object. +===== +==== +//End of inference_config + +//Begin input +`input`:: +(Required, object) +The input field names for the model definition. ++ +.Properties of `input` [%collapsible%open] -====== -`bert`:::: +==== +`field_names`::: +(Required, string) +An array of input field names for the model. +==== +//End input + +// Begin location +`location`:: +(Optional, object) +The model definition location. If the `definition` or `compressed_definition` +are not specified, the `location` is required. ++ +.Properties of `location` +[%collapsible%open] +==== +`index`::: +(Required, object) +Indicates that the model definition is stored in an index. This object must be +empty as the index for storing model definitions is configured automatically. +==== +// End location + +`metadata`:: +(Optional, object) +An object map that contains metadata about the model. + +`model_size_bytes`:: +(Optional, integer) +The estimated memory usage in bytes to keep the trained model in memory. This +property is supported only if `defer_definition_decompression` is `true` or the +model definition is not supplied. + +`model_type`:: +(Optional, string) +The created model type. By default the model type is `tree_ensemble`. +Appropriate types are: ++ +-- +* `tree_ensemble`: The model definition is an ensemble model of decision trees. +* `lang_ident`: A special type reserved for language identification models. +* `pytorch`: The stored definition is a PyTorch (specifically a TorchScript) model. Currently only +NLP models are supported. For more information, refer to {ml-docs}/ml-nlp.html[{nlp-cap}]. +-- +`platform_architecture`:: +(Optional, string) +If the model only works on one platform, because it is heavily +optimized for a particular processor architecture and OS combination, +then this field specifies which. The format of the string must match +the platform identifiers used by Elasticsearch, so one of, `linux-x86_64`, +`linux-aarch64`, `darwin-x86_64`, `darwin-aarch64`, or `windows-x86_64`. +For portable models (those that work independent of processor architecture or +OS features), leave this field unset. + + +`tags`:: +(Optional, string) +An array of tags to organize the model. + + +[[tokenization-properties]] +=== Properties of `tokenizaton` + +The `tokenization` object has the following properties. + +`bert`:: (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert] + .Properties of bert [%collapsible%open] -======= -`do_lower_case`:::: +==== +`do_lower_case`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] -`max_sequence_length`:::: +`max_sequence_length`::: (Optional, integer) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] -`truncate`:::: +`span`::: +(Optional, integer) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] + +`truncate`::: (Optional, string) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] -`with_special_tokens`:::: +`with_special_tokens`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-with-special-tokens] -======= -`roberta`:::: +==== +`roberta`:: (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta] + .Properties of roberta [%collapsible%open] -======= -`add_prefix_space`:::: +==== +`add_prefix_space`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-add-prefix-space] -`max_sequence_length`:::: +`max_sequence_length`::: (Optional, integer) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] -`truncate`:::: +`span`::: +(Optional, integer) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] + +`truncate`::: (Optional, string) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] -`with_special_tokens`:::: +`with_special_tokens`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`mpnet`:::: +==== +`mpnet`:: (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet] + .Properties of mpnet [%collapsible%open] -======= -`do_lower_case`:::: +==== +`do_lower_case`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] -`max_sequence_length`:::: +`max_sequence_length`::: (Optional, integer) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] -`truncate`:::: +`span`::: +(Optional, integer) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] + +`truncate`::: (Optional, string) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] -`with_special_tokens`:::: +`with_special_tokens`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-mpnet-with-special-tokens] -======= -`xlm_roberta`:::: +==== +`xlm_roberta`:: (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-xlm-roberta] + .Properties of xlm_roberta [%collapsible%open] -======= -`max_sequence_length`:::: +==== +`max_sequence_length`::: (Optional, integer) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] -`truncate`:::: +`span`::: +(Optional, integer) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] + +`truncate`::: (Optional, string) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] -`with_special_tokens`:::: +`with_special_tokens`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-roberta-with-special-tokens] -======= -`bert_ja`:::: +==== +`bert_ja`:: (Optional, object) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja] + .Properties of bert_ja [%collapsible%open] -======= -`do_lower_case`:::: +==== +`do_lower_case`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-do-lower-case] -`max_sequence_length`:::: +`max_sequence_length`::: (Optional, integer) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-max-sequence-length] -`truncate`:::: +`span`::: +(Optional, integer) +include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-span] + +`truncate`::: (Optional, string) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-truncate] -`with_special_tokens`:::: +`with_special_tokens`::: (Optional, boolean) include::{es-repo-dir}/ml/ml-shared.asciidoc[tag=inference-config-nlp-tokenization-bert-ja-with-special-tokens] -======= -====== -===== -==== -//End of inference_config - -//Begin input -`input`:: -(Required, object) -The input field names for the model definition. -+ -.Properties of `input` -[%collapsible%open] -==== -`field_names`::: -(Required, string) -An array of input field names for the model. -==== -//End input - -// Begin location -`location`:: -(Optional, object) -The model definition location. If the `definition` or `compressed_definition` -are not specified, the `location` is required. -+ -.Properties of `location` -[%collapsible%open] -==== -`index`::: -(Required, object) -Indicates that the model definition is stored in an index. This object must be -empty as the index for storing model definitions is configured automatically. ==== -// End location - -`metadata`:: -(Optional, object) -An object map that contains metadata about the model. - -`model_size_bytes`:: -(Optional, integer) -The estimated memory usage in bytes to keep the trained model in memory. This -property is supported only if `defer_definition_decompression` is `true` or the -model definition is not supplied. - -`model_type`:: -(Optional, string) -The created model type. By default the model type is `tree_ensemble`. -Appropriate types are: -+ --- -* `tree_ensemble`: The model definition is an ensemble model of decision trees. -* `lang_ident`: A special type reserved for language identification models. -* `pytorch`: The stored definition is a PyTorch (specifically a TorchScript) model. Currently only -NLP models are supported. For more information, refer to {ml-docs}/ml-nlp.html[{nlp-cap}]. --- -`platform_architecture`:: -(Optional, string) -If the model only works on one platform, because it is heavily -optimized for a particular processor architecture and OS combination, -then this field specifies which. The format of the string must match -the platform identifiers used by Elasticsearch, so one of, `linux-x86_64`, -`linux-aarch64`, `darwin-x86_64`, `darwin-aarch64`, or `windows-x86_64`. -For portable models (those that work independent of processor architecture or -OS features), leave this field unset. - - -`tags`:: -(Optional, string) -An array of tags to organize the model. [[ml-put-trained-models-example]] From e791cb286b210fd5e2df64568974a5c0c7d5e4ce Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 8 Nov 2023 19:15:36 +0100 Subject: [PATCH 009/513] Dry up AsyncTaskIndexService memory management and fix inefficient circuit breaker use (#101892) This does two things. For one, this dries up a couple of spots where we were allocating the buffers for "base64ed" search responses. This sets up future improvements to the memory use of this functionality (and generally makes the memory usage easier to follow IMO). But also, one fix is already made in the error handling in `updateResponse`. With this change we directly release buffers when this method fails instead of waiting for the subsequent persistence of the exception to complete. This was a serious bug when the expection to be persisted was a circuit breaker exception because we would need yet more bytes from the `BigArrays` instance that's already at capacity, to be able to release the bytes that we acquired up until the breaker exception. --- docs/changelog/101892.yaml | 6 + .../core/async/AsyncTaskIndexService.java | 155 +++++++----------- 2 files changed, 69 insertions(+), 92 deletions(-) create mode 100644 docs/changelog/101892.yaml diff --git a/docs/changelog/101892.yaml b/docs/changelog/101892.yaml new file mode 100644 index 0000000000000..175871de83d1a --- /dev/null +++ b/docs/changelog/101892.yaml @@ -0,0 +1,6 @@ +pr: 101892 +summary: Dry up `AsyncTaskIndexService` memory management and fix inefficient circuit + breaker use +area: Search +type: bug +issues: [] diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java index 6f6cc6c259e34..746172fef18f3 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/async/AsyncTaskIndexService.java @@ -43,6 +43,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParserUtils; +import org.elasticsearch.core.Releasables; import org.elasticsearch.core.Streams; import org.elasticsearch.index.engine.DocumentMissingException; import org.elasticsearch.index.engine.VersionConflictEngineException; @@ -63,7 +64,6 @@ import java.io.IOException; import java.io.InputStream; -import java.io.OutputStream; import java.io.UncheckedIOException; import java.nio.CharBuffer; import java.util.Base64; @@ -215,54 +215,15 @@ public SecurityContext getSecurityContext() { * TODO: add limit for stored async response in EQL, and instead of this method use createResponse */ public void createResponseForEQL(String docId, Map headers, R response, ActionListener listener) { - try { - final ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutput(0, bigArrays.withCircuitBreaking()); - final XContentBuilder source = XContentFactory.jsonBuilder(buffer); - listener = ActionListener.runBefore(listener, buffer::close); - source.startObject() - .field(HEADERS_FIELD, headers) - .field(EXPIRATION_TIME_FIELD, response.getExpirationTime()) - .directFieldAsBase64(RESULT_FIELD, os -> writeResponse(response, os)) - .endObject(); - - // do not close the buffer or the XContentBuilder until the IndexRequest is completed (i.e., listener is notified); - // otherwise, we underestimate the memory usage in case the circuit breaker does not use the real memory usage. - source.flush(); - final IndexRequest indexRequest = new IndexRequest(index).create(true).id(docId).source(buffer.bytes(), source.contentType()); - clientWithOrigin.index(indexRequest, listener); - } catch (Exception e) { - listener.onFailure(e); - } + indexResponse(docId, headers, response, false, listener); } /** * Stores the initial response with the original headers of the authenticated user * and the expected expiration time. */ - public void createResponse(String docId, Map headers, R response, ActionListener listener) - throws IOException { - try { - final ReleasableBytesStreamOutput buffer = new ReleasableBytesStreamOutputWithLimit( - 0, - bigArrays.withCircuitBreaking(), - maxResponseSize - ); - final XContentBuilder source = XContentFactory.jsonBuilder(buffer); - listener = ActionListener.runBefore(listener, buffer::close); - source.startObject() - .field(HEADERS_FIELD, headers) - .field(EXPIRATION_TIME_FIELD, response.getExpirationTime()) - .directFieldAsBase64(RESULT_FIELD, os -> writeResponse(response, os)) - .endObject(); - - // do not close the buffer or the XContentBuilder until the IndexRequest is completed (i.e., listener is notified); - // otherwise, we underestimate the memory usage in case the circuit breaker does not use the real memory usage. - source.flush(); - final IndexRequest indexRequest = new IndexRequest(index).create(true).id(docId).source(buffer.bytes(), source.contentType()); - clientWithOrigin.index(indexRequest, listener); - } catch (Exception e) { - listener.onFailure(e); - } + public void createResponse(String docId, Map headers, R response, ActionListener listener) { + indexResponse(docId, headers, response, true, listener); } public void updateResponse( @@ -274,6 +235,27 @@ public void updateResponse( updateResponse(docId, responseHeaders, response, listener, false); } + private void indexResponse( + String docId, + Map headers, + R response, + boolean limitToMaxResponseSize, + ActionListener listener + ) { + try { + var buffer = allocateBuffer(limitToMaxResponseSize); + listener = ActionListener.runBefore(listener, buffer::close); + final XContentBuilder source = XContentFactory.jsonBuilder(buffer) + .startObject() + .field(HEADERS_FIELD, headers) + .field(EXPIRATION_TIME_FIELD, response.getExpirationTime()); + addResultFieldAndFinish(response, source); + clientWithOrigin.index(new IndexRequest(index).create(true).id(docId).source(buffer.bytes(), source.contentType()), listener); + } catch (Exception e) { + listener.onFailure(e); + } + } + /** * Stores the final response if the place-holder document is still present (update). */ @@ -284,25 +266,18 @@ private void updateResponse( ActionListener listener, boolean isFailure ) { + ReleasableBytesStreamOutput buffer = null; try { - final ReleasableBytesStreamOutput buffer = isFailure - ? new ReleasableBytesStreamOutput(0, bigArrays.withCircuitBreaking()) - : new ReleasableBytesStreamOutputWithLimit(0, bigArrays.withCircuitBreaking(), maxResponseSize); - final XContentBuilder source = XContentFactory.jsonBuilder(buffer); - listener = ActionListener.runBefore(listener, buffer::close); - source.startObject() - .field(RESPONSE_HEADERS_FIELD, responseHeaders) - .directFieldAsBase64(RESULT_FIELD, os -> writeResponse(response, os)) - .endObject(); - // do not close the buffer or the XContentBuilder until the UpdateRequest is completed (i.e., listener is notified); - // otherwise, we underestimate the memory usage in case the circuit breaker does not use the real memory usage. - source.flush(); - final UpdateRequest request = new UpdateRequest().index(index) - .id(docId) - .doc(buffer.bytes(), source.contentType()) - .retryOnConflict(5); - clientWithOrigin.update(request, listener); + buffer = allocateBuffer(isFailure == false); + final XContentBuilder source = XContentFactory.jsonBuilder(buffer).startObject().field(RESPONSE_HEADERS_FIELD, responseHeaders); + addResultFieldAndFinish(response, source); + clientWithOrigin.update( + new UpdateRequest().index(index).id(docId).doc(buffer.bytes(), source.contentType()).retryOnConflict(5), + ActionListener.runBefore(listener, buffer::close) + ); } catch (Exception e) { + // release buffer right away to save memory, particularly in case the exception came from the circuit breaker + Releasables.close(buffer); // even if we expect updating with a failure always succeed // this is just an extra precaution not to create infinite loops if (isFailure) { @@ -311,14 +286,13 @@ private void updateResponse( Throwable cause = ExceptionsHelper.unwrapCause(e); if (cause instanceof DocumentMissingException == false && cause instanceof VersionConflictEngineException == false) { logger.error(() -> "failed to store async-search [" + docId + "]", e); - ActionListener newListener = listener; - updateStoredResponseWithFailure( + // at end, we should report a failure to the listener + updateResponse( docId, responseHeaders, - response, - e, - // at end, we should report a failure to the listener - ActionListener.running(() -> newListener.onFailure(e)) + response.convertToFailure(e), + ActionListener.running(() -> listener.onFailure(e)), + true ); } else { listener.onFailure(e); @@ -327,18 +301,29 @@ private void updateResponse( } } - /** - * Update the initial stored response with a failure - */ - private void updateStoredResponseWithFailure( - String docId, - Map> responseHeaders, - R response, - Exception updateException, - ActionListener listener - ) { - R failureResponse = response.convertToFailure(updateException); - updateResponse(docId, responseHeaders, failureResponse, listener, true); + private ReleasableBytesStreamOutput allocateBuffer(boolean limitToMaxResponseSize) { + return limitToMaxResponseSize + ? new ReleasableBytesStreamOutputWithLimit(0, bigArrays.withCircuitBreaking(), maxResponseSize) + : new ReleasableBytesStreamOutput(0, bigArrays.withCircuitBreaking()); + } + + private void addResultFieldAndFinish(Writeable response, XContentBuilder source) throws IOException { + source.directFieldAsBase64(RESULT_FIELD, os -> { + // do not close the output + os = Streams.noCloseStream(os); + TransportVersion minNodeVersion = clusterService.state().getMinTransportVersion(); + TransportVersion.writeVersion(minNodeVersion, new OutputStreamStreamOutput(os)); + if (minNodeVersion.onOrAfter(TransportVersions.V_7_15_0)) { + os = CompressorFactory.COMPRESSOR.threadLocalOutputStream(os); + } + try (OutputStreamStreamOutput out = new OutputStreamStreamOutput(os)) { + out.setTransportVersion(minNodeVersion); + response.writeTo(out); + } + }).endObject(); + // do not close the buffer or the XContentBuilder until the request is completed (i.e., listener is notified); + // otherwise, we underestimate the memory usage in case the circuit breaker does not use the real memory usage. + source.flush(); } /** @@ -565,20 +550,6 @@ void ensureAuthenticatedUserCanDeleteFromIndex(AsyncExecutionId executionId, Act }, exc -> listener.onFailure(new ResourceNotFoundException(executionId.getEncoded())))); } - private void writeResponse(R response, OutputStream os) throws IOException { - // do not close the output - os = Streams.noCloseStream(os); - TransportVersion minNodeVersion = clusterService.state().getMinTransportVersion(); - TransportVersion.writeVersion(minNodeVersion, new OutputStreamStreamOutput(os)); - if (minNodeVersion.onOrAfter(TransportVersions.V_7_15_0)) { - os = CompressorFactory.COMPRESSOR.threadLocalOutputStream(os); - } - try (OutputStreamStreamOutput out = new OutputStreamStreamOutput(os)) { - out.setTransportVersion(minNodeVersion); - response.writeTo(out); - } - } - /** * Decode the provided base-64 bytes into a {@link AsyncSearchResponse}. */ From 379e53190ac185d4ad2a4b71935281eeb9b08296 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 8 Nov 2023 14:24:16 -0500 Subject: [PATCH 010/513] Update 8.11.0.asciidoc forward port of: https://github.com/elastic/elasticsearch/pull/101908/files --- docs/reference/release-notes/8.11.0.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/release-notes/8.11.0.asciidoc b/docs/reference/release-notes/8.11.0.asciidoc index 08ddaf5667845..16ff5edd6d91a 100644 --- a/docs/reference/release-notes/8.11.0.asciidoc +++ b/docs/reference/release-notes/8.11.0.asciidoc @@ -297,8 +297,8 @@ Transform:: * Add accessors required to recreate `TransformStats` object from the fields {es-pull}98844[#98844] Vector Search:: -* Add new max_inner_product vector similarity function {es-pull}99445[#99445] -* Adds `nested` support for indexed `dense_vector` fields {es-pull}99532[#99532] +* Add new max_inner_product vector similarity function {es-pull}99527[#99527] +* Adds `nested` support for indexed `dense_vector` fields {es-pull}99763[#99763] * Dense vector field types are indexed by default {es-pull}98268[#98268] * Increase the max vector dims to 4096 {es-pull}99682[#99682] From fd819ad2475ace982e521cfef619adfe9cd9c64d Mon Sep 17 00:00:00 2001 From: Chris Cowan Date: Wed, 8 Nov 2023 12:59:49 -0700 Subject: [PATCH 011/513] Add manage_enrich cluster privilege to kibana_system role (#101682) * Add manage_enrich cluster privilege to kibana_system role * Adding change log entry * fixing typo * Adding test for manage_enrich privilege for kibana_system role * Adding delete and stats --- docs/changelog/101682.yaml | 5 +++++ .../authz/store/KibanaOwnedReservedRoleDescriptors.java | 2 ++ .../core/security/authz/store/ReservedRolesStoreTests.java | 7 +++++++ 3 files changed, 14 insertions(+) create mode 100644 docs/changelog/101682.yaml diff --git a/docs/changelog/101682.yaml b/docs/changelog/101682.yaml new file mode 100644 index 0000000000000..e512006057581 --- /dev/null +++ b/docs/changelog/101682.yaml @@ -0,0 +1,5 @@ +pr: 101682 +summary: "Add manage_enrich cluster privilege to kibana_system role" +area: Authentication +type: enhancement +issues: [] diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/KibanaOwnedReservedRoleDescriptors.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/KibanaOwnedReservedRoleDescriptors.java index c5fc72bfeb2ab..33c8dbdf27bf9 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/KibanaOwnedReservedRoleDescriptors.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/store/KibanaOwnedReservedRoleDescriptors.java @@ -64,6 +64,8 @@ static RoleDescriptor kibanaSystem(String name) { "manage_saml", "manage_token", "manage_oidc", + // For SLO to install enrich policy + "manage_enrich", // For Fleet package upgrade "manage_pipeline", "manage_ilm", diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java index 477a5e014c105..5028066f67ad9 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java @@ -428,6 +428,13 @@ public void testKibanaSystemRole() { assertThat(kibanaRole.cluster().check(ClusterUpdateSettingsAction.NAME, request, authentication), is(false)); assertThat(kibanaRole.cluster().check(MonitoringBulkAction.NAME, request, authentication), is(true)); + // Enrich + assertThat(kibanaRole.cluster().check("cluster:admin/xpack/enrich/put", request, authentication), is(true)); + assertThat(kibanaRole.cluster().check("cluster:admin/xpack/enrich/execute", request, authentication), is(true)); + assertThat(kibanaRole.cluster().check("cluster:admin/xpack/enrich/get", request, authentication), is(true)); + assertThat(kibanaRole.cluster().check("cluster:admin/xpack/enrich/delete", request, authentication), is(true)); + assertThat(kibanaRole.cluster().check("cluster:admin/xpack/enrich/stats", request, authentication), is(true)); + // SAML and token assertThat(kibanaRole.cluster().check(SamlPrepareAuthenticationAction.NAME, request, authentication), is(true)); assertThat(kibanaRole.cluster().check(SamlAuthenticateAction.NAME, request, authentication), is(true)); From 879efb21b87366a73f90bdc65b94a63124feabe8 Mon Sep 17 00:00:00 2001 From: William Brafford Date: Wed, 8 Nov 2023 15:59:59 -0500 Subject: [PATCH 012/513] Mute failing test in MlAssignmentPlannerUpgradeIT (#101931) Awaits fix: https://github.com/elastic/elasticsearch/issues/101926 --- .../org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java index fc78bf36c72fb..48d7bd918255d 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java @@ -61,6 +61,7 @@ public class MlAssignmentPlannerUpgradeIT extends AbstractUpgradeTestCase { RAW_MODEL_SIZE = Base64.getDecoder().decode(BASE_64_ENCODED_MODEL).length; } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/101926") public void testMlAssignmentPlannerUpgrade() throws Exception { assumeTrue("NLP model deployments added in 8.0", isOriginalClusterVersionAtLeast(Version.V_8_0_0)); From a3ccc8811e2fe2f6b74576afa9e58d17f674c3d6 Mon Sep 17 00:00:00 2001 From: William Brafford Date: Wed, 8 Nov 2023 16:25:31 -0500 Subject: [PATCH 013/513] Mute failing test in SearchQueryThenFetchAsyncActionTests (#101933) Awaits fix issue: #101932 --- .../action/search/SearchQueryThenFetchAsyncActionTests.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 2ed24e95c16d5..b0364209a811a 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -237,6 +237,7 @@ public void run() { assertThat(((FieldDoc) phase.sortedTopDocs().scoreDocs()[0]).fields[0], equalTo(0)); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/101932") public void testMinimumVersionSameAsNewVersion() throws Exception { Version newVersion = Version.CURRENT; Version oldVersion = VersionUtils.randomVersionBetween( @@ -354,6 +355,7 @@ public void onResponse(SearchResponse response) { ); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/101932") public void testMinimumVersionSameAsOldVersion() throws Exception { Version newVersion = Version.CURRENT; Version oldVersion = VersionUtils.randomVersionBetween( @@ -500,6 +502,7 @@ public void run() { assertThat(phase.totalHits().relation, equalTo(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO)); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/101932") public void testMinimumVersionShardDuringPhaseExecution() throws Exception { Version newVersion = Version.CURRENT; Version oldVersion = VersionUtils.randomVersionBetween( From aa274fed51bd8f4bce44901001785c24b37a82b5 Mon Sep 17 00:00:00 2001 From: James Baiera Date: Wed, 8 Nov 2023 17:11:20 -0500 Subject: [PATCH 014/513] Remove usage of Version numbers from some Monitoring code (#101890) --- .../core/monitoring/exporter/MonitoringTemplateUtils.java | 3 +-- .../xpack/monitoring/MonitoringTemplateRegistry.java | 5 ++--- .../xpack/monitoring/exporter/ClusterAlertsUtil.java | 3 +-- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/exporter/MonitoringTemplateUtils.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/exporter/MonitoringTemplateUtils.java index cca9acddd6d86..b600fec54b29b 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/exporter/MonitoringTemplateUtils.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/monitoring/exporter/MonitoringTemplateUtils.java @@ -6,7 +6,6 @@ */ package org.elasticsearch.xpack.core.monitoring.exporter; -import org.elasticsearch.Version; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.xpack.core.monitoring.MonitoredSystem; @@ -19,7 +18,7 @@ public final class MonitoringTemplateUtils { *

* It may be possible for this to diverge between templates and pipelines, but for now they're the same. */ - public static final int LAST_UPDATED_VERSION = Version.V_8_8_0.id; + public static final int LAST_UPDATED_VERSION = 8_08_00_99; /** * Current version of templates used in their name to differentiate from breaking changes (separate from product version). diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringTemplateRegistry.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringTemplateRegistry.java index a7720ce30e940..b054095499a30 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringTemplateRegistry.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/MonitoringTemplateRegistry.java @@ -9,7 +9,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.Version; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; import org.elasticsearch.cluster.service.ClusterService; @@ -50,7 +49,7 @@ public class MonitoringTemplateRegistry extends IndexTemplateRegistry { * continue to use the release version number in this registry, even though this is not standard practice for template * registries. */ - public static final int REGISTRY_VERSION = Version.V_8_8_0.id; + public static final int REGISTRY_VERSION = 8_08_00_99; private static final String REGISTRY_VERSION_VARIABLE = "xpack.monitoring.template.release.version"; /** @@ -78,7 +77,7 @@ public class MonitoringTemplateRegistry extends IndexTemplateRegistry { * writes monitoring data in ECS format as of 8.0. These templates define the ECS schema as well as alias fields for the old monitoring * mappings that point to the corresponding ECS fields. */ - public static final int STACK_MONITORING_REGISTRY_VERSION = Version.V_8_0_0.id + 9; + public static final int STACK_MONITORING_REGISTRY_VERSION = 8_00_00_99 + 9; private static final String STACK_MONITORING_REGISTRY_VERSION_VARIABLE = "xpack.stack.monitoring.template.release.version"; private static final String STACK_TEMPLATE_VERSION = "8"; private static final String STACK_TEMPLATE_VERSION_VARIABLE = "xpack.stack.monitoring.template.version"; diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/ClusterAlertsUtil.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/ClusterAlertsUtil.java index 3173af96cfd7d..29452b6bbf28d 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/ClusterAlertsUtil.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/ClusterAlertsUtil.java @@ -6,7 +6,6 @@ */ package org.elasticsearch.xpack.monitoring.exporter; -import org.elasticsearch.Version; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; @@ -58,7 +57,7 @@ public class ClusterAlertsUtil { * The last time that all watches were updated. For now, all watches have been updated in the same version and should all be replaced * together. */ - public static final int LAST_UPDATED_VERSION = Version.V_7_5_0.id; + public static final int LAST_UPDATED_VERSION = 7_05_00_99; /** * An unsorted list of Watch IDs representing resource files for Monitoring Cluster Alerts. From 57acc1c17114f5f7fc9ab9e50a13a596d56b02fa Mon Sep 17 00:00:00 2001 From: William Brafford Date: Wed, 8 Nov 2023 17:24:19 -0500 Subject: [PATCH 015/513] Mute failing test in APMYamlTestSuiteIT (#101937) Awaits fix: https://github.com/elastic/elasticsearch/issues/101929 --- .../org/elasticsearch/xpack/apmdata/APMYamlTestSuiteIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/apm-data/src/yamlRestTest/java/org/elasticsearch/xpack/apmdata/APMYamlTestSuiteIT.java b/x-pack/plugin/apm-data/src/yamlRestTest/java/org/elasticsearch/xpack/apmdata/APMYamlTestSuiteIT.java index 5835a41479a68..77cac16a4e90c 100644 --- a/x-pack/plugin/apm-data/src/yamlRestTest/java/org/elasticsearch/xpack/apmdata/APMYamlTestSuiteIT.java +++ b/x-pack/plugin/apm-data/src/yamlRestTest/java/org/elasticsearch/xpack/apmdata/APMYamlTestSuiteIT.java @@ -10,11 +10,13 @@ import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.apache.lucene.tests.util.LuceneTestCase; import org.elasticsearch.test.cluster.ElasticsearchCluster; import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate; import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase; import org.junit.ClassRule; +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/101929") public class APMYamlTestSuiteIT extends ESClientYamlSuiteTestCase { @ClassRule From 2ebd08421fcaa989bb93292bf81e012ec91ba89c Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 8 Nov 2023 16:02:27 -0700 Subject: [PATCH 016/513] Revert "Mark legacy stack templates as deprecated (#101476)" (#101935) * Revert "Mark legacy stack templates as deprecated (#101476)" This reverts commit 16cce1fc1a335caa61a764aa843faaffb3e53cbc. * Remove test that was part of revert conflict --------- Co-authored-by: Elastic Machine --- docs/changelog/101476.yaml | 5 -- .../xpack/apmdata/ResourceUtils.java | 7 +- .../apmdata/YamlIngestPipelineConfig.java | 2 +- .../core/template/IngestPipelineConfig.java | 14 --- .../template/JsonIngestPipelineConfig.java | 14 +-- .../xpack/core/template/TemplateUtils.java | 2 +- .../main/resources/180-days@lifecycle.json | 3 +- .../src/main/resources/30-days@lifecycle.json | 3 +- .../main/resources/365-days@lifecycle.json | 3 +- .../src/main/resources/7-days@lifecycle.json | 3 +- .../src/main/resources/90-days@lifecycle.json | 3 +- .../main/resources/data-streams@mappings.json | 3 +- .../src/main/resources/ecs@mappings.json | 3 +- .../resources/kibana-reporting@template.json | 3 +- .../main/resources/logs@default-pipeline.json | 3 +- .../main/resources/logs@json-pipeline.json | 3 +- .../src/main/resources/logs@lifecycle.json | 3 +- .../src/main/resources/logs@mappings.json | 3 +- .../src/main/resources/logs@settings.json | 3 +- .../src/main/resources/logs@template.json | 3 +- .../src/main/resources/metrics@lifecycle.json | 3 +- .../src/main/resources/metrics@mappings.json | 3 +- .../src/main/resources/metrics@settings.json | 3 +- .../src/main/resources/metrics@template.json | 3 +- .../main/resources/metrics@tsdb-settings.json | 3 +- .../main/resources/synthetics@lifecycle.json | 3 +- .../main/resources/synthetics@mappings.json | 3 +- .../main/resources/synthetics@settings.json | 3 +- .../main/resources/synthetics@template.json | 3 +- .../stack/LegacyStackTemplateRegistry.java | 63 ++++--------- .../xpack/stack/StackTemplateRegistry.java | 90 +++++-------------- .../LegacyStackTemplateRegistryTests.java | 63 ------------- .../stack/StackTemplateRegistryTests.java | 17 ---- 33 files changed, 69 insertions(+), 277 deletions(-) delete mode 100644 docs/changelog/101476.yaml delete mode 100644 x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/LegacyStackTemplateRegistryTests.java diff --git a/docs/changelog/101476.yaml b/docs/changelog/101476.yaml deleted file mode 100644 index ee4cd9b1e4b1a..0000000000000 --- a/docs/changelog/101476.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 101476 -summary: Mark legacy stack templates as deprecated -area: Indices APIs -type: enhancement -issues: [] diff --git a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/ResourceUtils.java b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/ResourceUtils.java index 1e6a9a9998a82..b9a6edfb958f3 100644 --- a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/ResourceUtils.java +++ b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/ResourceUtils.java @@ -13,20 +13,15 @@ import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; -import java.util.Map; public class ResourceUtils { public static final String APM_TEMPLATE_VERSION_VARIABLE = "xpack.apmdata.template.version"; static byte[] loadVersionedResourceUTF8(String name, int version) { - return loadVersionedResourceUTF8(name, version, Map.of()); - } - - static byte[] loadVersionedResourceUTF8(String name, int version, Map variables) { try { String content = loadResource(name); - content = TemplateUtils.replaceVariables(content, String.valueOf(version), APM_TEMPLATE_VERSION_VARIABLE, variables); + content = TemplateUtils.replaceVariable(content, APM_TEMPLATE_VERSION_VARIABLE, String.valueOf(version)); return content.getBytes(StandardCharsets.UTF_8); } catch (IOException e) { throw new RuntimeException(e); diff --git a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/YamlIngestPipelineConfig.java b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/YamlIngestPipelineConfig.java index de1b715dd138d..938fd69f80abe 100644 --- a/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/YamlIngestPipelineConfig.java +++ b/x-pack/plugin/apm-data/src/main/java/org/elasticsearch/xpack/apmdata/YamlIngestPipelineConfig.java @@ -31,6 +31,6 @@ public XContentType getXContentType() { @Override public BytesReference loadConfig() { - return new BytesArray(loadVersionedResourceUTF8("/ingest-pipelines/" + id + ".yaml", version, variables)); + return new BytesArray(loadVersionedResourceUTF8("/ingest-pipelines/" + id + ".yaml", version)); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IngestPipelineConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IngestPipelineConfig.java index 2768355183687..a216030f1c2e0 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IngestPipelineConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/IngestPipelineConfig.java @@ -12,7 +12,6 @@ import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Objects; /** @@ -23,7 +22,6 @@ public abstract class IngestPipelineConfig { protected final String resource; protected final int version; protected final String versionProperty; - protected final Map variables; /** * A list of this pipeline's dependencies, for example - such referred to through a pipeline processor. @@ -37,23 +35,11 @@ public IngestPipelineConfig(String id, String resource, int version, String vers } public IngestPipelineConfig(String id, String resource, int version, String versionProperty, List dependencies) { - this(id, resource, version, versionProperty, dependencies, Map.of()); - } - - public IngestPipelineConfig( - String id, - String resource, - int version, - String versionProperty, - List dependencies, - Map variables - ) { this.id = Objects.requireNonNull(id); this.resource = Objects.requireNonNull(resource); this.version = version; this.versionProperty = Objects.requireNonNull(versionProperty); this.dependencies = dependencies; - this.variables = Objects.requireNonNull(variables); } public String getId() { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/JsonIngestPipelineConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/JsonIngestPipelineConfig.java index 05a27de40aadc..fc2ca7cbce186 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/JsonIngestPipelineConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/JsonIngestPipelineConfig.java @@ -12,7 +12,6 @@ import org.elasticsearch.xcontent.XContentType; import java.util.List; -import java.util.Map; public class JsonIngestPipelineConfig extends IngestPipelineConfig { public JsonIngestPipelineConfig(String id, String resource, int version, String versionProperty) { @@ -23,17 +22,6 @@ public JsonIngestPipelineConfig(String id, String resource, int version, String super(id, resource, version, versionProperty, dependencies); } - public JsonIngestPipelineConfig( - String id, - String resource, - int version, - String versionProperty, - List dependencies, - Map variables - ) { - super(id, resource, version, versionProperty, dependencies, variables); - } - @Override public XContentType getXContentType() { return XContentType.JSON; @@ -41,6 +29,6 @@ public XContentType getXContentType() { @Override public BytesReference loadConfig() { - return new BytesArray(TemplateUtils.loadTemplate(resource, String.valueOf(version), versionProperty, variables)); + return new BytesArray(TemplateUtils.loadTemplate(resource, String.valueOf(version), versionProperty)); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/TemplateUtils.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/TemplateUtils.java index d0be0ad9cb697..ad27607e47c5e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/TemplateUtils.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/template/TemplateUtils.java @@ -98,7 +98,7 @@ public static void validate(String source) { } } - public static String replaceVariables(String input, String version, String versionProperty, Map variables) { + private static String replaceVariables(String input, String version, String versionProperty, Map variables) { String template = replaceVariable(input, versionProperty, version); for (Map.Entry variable : variables.entrySet()) { template = replaceVariable(template, variable.getKey(), variable.getValue()); diff --git a/x-pack/plugin/core/template-resources/src/main/resources/180-days@lifecycle.json b/x-pack/plugin/core/template-resources/src/main/resources/180-days@lifecycle.json index 0fcaddb9a02ce..7929d4cb5594c 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/180-days@lifecycle.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/180-days@lifecycle.json @@ -33,6 +33,5 @@ "_meta": { "description": "built-in ILM policy using the hot, warm, and cold phases with a retention of 180 days", "managed": true - }, - "deprecated": ${xpack.stack.template.deprecated} + } } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/30-days@lifecycle.json b/x-pack/plugin/core/template-resources/src/main/resources/30-days@lifecycle.json index 5764b75299ced..6d5a12b39762d 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/30-days@lifecycle.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/30-days@lifecycle.json @@ -29,6 +29,5 @@ "_meta": { "description": "built-in ILM policy using the hot and warm phases with a retention of 30 days", "managed": true - }, - "deprecated": ${xpack.stack.template.deprecated} + } } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/365-days@lifecycle.json b/x-pack/plugin/core/template-resources/src/main/resources/365-days@lifecycle.json index 4398b14387dec..3d2340245f117 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/365-days@lifecycle.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/365-days@lifecycle.json @@ -33,6 +33,5 @@ "_meta": { "description": "built-in ILM policy using the hot, warm, and cold phases with a retention of 365 days", "managed": true - }, - "deprecated": ${xpack.stack.template.deprecated} + } } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/7-days@lifecycle.json b/x-pack/plugin/core/template-resources/src/main/resources/7-days@lifecycle.json index 1a1f74beac516..2c5778e5af1db 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/7-days@lifecycle.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/7-days@lifecycle.json @@ -29,6 +29,5 @@ "_meta": { "description": "built-in ILM policy using the hot and warm phases with a retention of 7 days", "managed": true - }, - "deprecated": ${xpack.stack.template.deprecated} + } } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/90-days@lifecycle.json b/x-pack/plugin/core/template-resources/src/main/resources/90-days@lifecycle.json index e0d2487c8961a..cae4e7c83a064 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/90-days@lifecycle.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/90-days@lifecycle.json @@ -33,6 +33,5 @@ "_meta": { "description": "built-in ILM policy using the hot, warm, and cold phases with a retention of 90 days", "managed": true - }, - "deprecated": ${xpack.stack.template.deprecated} + } } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/data-streams@mappings.json b/x-pack/plugin/core/template-resources/src/main/resources/data-streams@mappings.json index 96bbeca8f7ac8..f87c0e79b7c45 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/data-streams@mappings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/data-streams@mappings.json @@ -63,6 +63,5 @@ "description": "general mapping conventions for data streams", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/ecs@mappings.json b/x-pack/plugin/core/template-resources/src/main/resources/ecs@mappings.json index f1d03531e4b6b..fc29fc98dca96 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/ecs@mappings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/ecs@mappings.json @@ -190,6 +190,5 @@ "description": "dynamic mappings based on ECS, installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/kibana-reporting@template.json b/x-pack/plugin/core/template-resources/src/main/resources/kibana-reporting@template.json index b92942ff010d6..a4388d671eb0d 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/kibana-reporting@template.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/kibana-reporting@template.json @@ -173,6 +173,5 @@ "description": "default kibana reporting template installed by elasticsearch", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/logs@default-pipeline.json b/x-pack/plugin/core/template-resources/src/main/resources/logs@default-pipeline.json index d8dc9cca5ea7c..518ff3cece752 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/logs@default-pipeline.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/logs@default-pipeline.json @@ -20,6 +20,5 @@ "description": "default pipeline for the logs index template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/logs@json-pipeline.json b/x-pack/plugin/core/template-resources/src/main/resources/logs@json-pipeline.json index e3b0f85642a46..cebeccd344324 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/logs@json-pipeline.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/logs@json-pipeline.json @@ -44,6 +44,5 @@ "description": "automatic parsing of JSON log messages", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/logs@lifecycle.json b/x-pack/plugin/core/template-resources/src/main/resources/logs@lifecycle.json index 5b58c138d785f..6bce19aaaab49 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/logs@lifecycle.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/logs@lifecycle.json @@ -12,6 +12,5 @@ "_meta": { "description": "default policy for the logs index template installed by x-pack", "managed": true - }, - "deprecated": ${xpack.stack.template.deprecated} + } } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/logs@mappings.json b/x-pack/plugin/core/template-resources/src/main/resources/logs@mappings.json index 82cbf7e478a27..7417d4809559d 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/logs@mappings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/logs@mappings.json @@ -23,6 +23,5 @@ "description": "default mappings for the logs index template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/logs@settings.json b/x-pack/plugin/core/template-resources/src/main/resources/logs@settings.json index 908b01027e833..cc61f195402fe 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/logs@settings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/logs@settings.json @@ -20,6 +20,5 @@ "description": "default settings for the logs index template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/logs@template.json b/x-pack/plugin/core/template-resources/src/main/resources/logs@template.json index f9b945d75f4f8..b41b2d0453c89 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/logs@template.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/logs@template.json @@ -14,6 +14,5 @@ "description": "default logs template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/metrics@lifecycle.json b/x-pack/plugin/core/template-resources/src/main/resources/metrics@lifecycle.json index daa07659e559e..3c37e8db4a7da 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/metrics@lifecycle.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/metrics@lifecycle.json @@ -12,6 +12,5 @@ "_meta": { "description": "default policy for the metrics index template installed by x-pack", "managed": true - }, - "deprecated": ${xpack.stack.template.deprecated} + } } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/metrics@mappings.json b/x-pack/plugin/core/template-resources/src/main/resources/metrics@mappings.json index 4e48f6b7adaed..5741b441256f9 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/metrics@mappings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/metrics@mappings.json @@ -53,6 +53,5 @@ "description": "default mappings for the metrics index template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/metrics@settings.json b/x-pack/plugin/core/template-resources/src/main/resources/metrics@settings.json index 3a0e6feeaede4..1a13139bb18a4 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/metrics@settings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/metrics@settings.json @@ -16,6 +16,5 @@ "description": "default settings for the metrics index template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/metrics@template.json b/x-pack/plugin/core/template-resources/src/main/resources/metrics@template.json index 464df09ffe2ce..a596314bc9e8c 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/metrics@template.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/metrics@template.json @@ -12,6 +12,5 @@ "description": "default metrics template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/metrics@tsdb-settings.json b/x-pack/plugin/core/template-resources/src/main/resources/metrics@tsdb-settings.json index 6a64ff9be5473..cbcad39ef78d0 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/metrics@tsdb-settings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/metrics@tsdb-settings.json @@ -15,6 +15,5 @@ "description": "default settings for the metrics index template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/synthetics@lifecycle.json b/x-pack/plugin/core/template-resources/src/main/resources/synthetics@lifecycle.json index aa2cf5489b45f..1e4220725177d 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/synthetics@lifecycle.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/synthetics@lifecycle.json @@ -12,6 +12,5 @@ "_meta": { "description": "default policy for the synthetics index template installed by x-pack", "managed": true - }, - "deprecated": ${xpack.stack.template.deprecated} + } } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/synthetics@mappings.json b/x-pack/plugin/core/template-resources/src/main/resources/synthetics@mappings.json index 81b85285450c7..9e3e56e3261d0 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/synthetics@mappings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/synthetics@mappings.json @@ -17,6 +17,5 @@ "description": "default mappings for the synthetics index template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/synthetics@settings.json b/x-pack/plugin/core/template-resources/src/main/resources/synthetics@settings.json index 04d68d083bf9f..27ced96be36e3 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/synthetics@settings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/synthetics@settings.json @@ -13,6 +13,5 @@ "description": "default settings for the synthetics index template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/core/template-resources/src/main/resources/synthetics@template.json b/x-pack/plugin/core/template-resources/src/main/resources/synthetics@template.json index 344426541b8c5..6369bd5a82c15 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/synthetics@template.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/synthetics@template.json @@ -12,6 +12,5 @@ "description": "default synthetics template installed by x-pack", "managed": true }, - "version": ${xpack.stack.template.version}, - "deprecated": ${xpack.stack.template.deprecated} + "version": ${xpack.stack.template.version} } diff --git a/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/LegacyStackTemplateRegistry.java b/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/LegacyStackTemplateRegistry.java index f186d8e20f874..9fb33db74964a 100644 --- a/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/LegacyStackTemplateRegistry.java +++ b/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/LegacyStackTemplateRegistry.java @@ -52,8 +52,6 @@ public class LegacyStackTemplateRegistry extends IndexTemplateRegistry { private final ClusterService clusterService; private volatile boolean stackTemplateEnabled; - private static final Map ADDITIONAL_TEMPLATE_VARIABLES = Map.of("xpack.stack.template.deprecated", "true"); - // General mappings conventions for any data that ends up in a data stream public static final String DATA_STREAMS_MAPPINGS_COMPONENT_TEMPLATE_NAME = "data-streams-mappings"; @@ -123,14 +121,14 @@ private void updateEnabledSetting(boolean newValue) { } private static final List LIFECYCLE_POLICY_CONFIGS = List.of( - new LifecyclePolicyConfig(LOGS_ILM_POLICY_NAME, "/logs@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(METRICS_ILM_POLICY_NAME, "/metrics@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(SYNTHETICS_ILM_POLICY_NAME, "/synthetics@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_7_DAYS_POLICY_NAME, "/7-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_30_DAYS_POLICY_NAME, "/30-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_90_DAYS_POLICY_NAME, "/90-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_180_DAYS_POLICY_NAME, "/180-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_365_DAYS_POLICY_NAME, "/365-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES) + new LifecyclePolicyConfig(LOGS_ILM_POLICY_NAME, "/logs@lifecycle.json"), + new LifecyclePolicyConfig(METRICS_ILM_POLICY_NAME, "/metrics@lifecycle.json"), + new LifecyclePolicyConfig(SYNTHETICS_ILM_POLICY_NAME, "/synthetics@lifecycle.json"), + new LifecyclePolicyConfig(ILM_7_DAYS_POLICY_NAME, "/7-days@lifecycle.json"), + new LifecyclePolicyConfig(ILM_30_DAYS_POLICY_NAME, "/30-days@lifecycle.json"), + new LifecyclePolicyConfig(ILM_90_DAYS_POLICY_NAME, "/90-days@lifecycle.json"), + new LifecyclePolicyConfig(ILM_180_DAYS_POLICY_NAME, "/180-days@lifecycle.json"), + new LifecyclePolicyConfig(ILM_365_DAYS_POLICY_NAME, "/365-days@lifecycle.json") ); @Override @@ -156,64 +154,55 @@ protected List getLifecyclePolicies() { DATA_STREAMS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/data-streams@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( LOGS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/logs@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( ECS_DYNAMIC_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/ecs@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( LOGS_SETTINGS_COMPONENT_TEMPLATE_NAME, "/logs@settings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( METRICS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/metrics@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( METRICS_SETTINGS_COMPONENT_TEMPLATE_NAME, "/metrics@settings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( METRICS_TSDB_SETTINGS_COMPONENT_TEMPLATE_NAME, "/metrics@tsdb-settings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( SYNTHETICS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/synthetics@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( SYNTHETICS_SETTINGS_COMPONENT_TEMPLATE_NAME, "/synthetics@settings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ) )) { try { @@ -243,22 +232,8 @@ protected Map getComposableTemplateConfigs() { } private static final List INGEST_PIPELINE_CONFIGS = List.of( - new JsonIngestPipelineConfig( - "logs@json-message", - "/logs@json-pipeline.json", - REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - List.of(), - ADDITIONAL_TEMPLATE_VARIABLES - ), - new JsonIngestPipelineConfig( - "logs-default-pipeline", - "/logs@default-pipeline.json", - REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - List.of(), - ADDITIONAL_TEMPLATE_VARIABLES - ) + new JsonIngestPipelineConfig("logs@json-message", "/logs@json-pipeline.json", REGISTRY_VERSION, TEMPLATE_VERSION_VARIABLE), + new JsonIngestPipelineConfig("logs-default-pipeline", "/logs@default-pipeline.json", REGISTRY_VERSION, TEMPLATE_VERSION_VARIABLE) ); @Override diff --git a/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java b/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java index af5dce0ee927c..3471d312d9df8 100644 --- a/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java +++ b/x-pack/plugin/stack/src/main/java/org/elasticsearch/xpack/stack/StackTemplateRegistry.java @@ -55,8 +55,6 @@ public class StackTemplateRegistry extends IndexTemplateRegistry { private final ClusterService clusterService; private volatile boolean stackTemplateEnabled; - private static final Map ADDITIONAL_TEMPLATE_VARIABLES = Map.of("xpack.stack.template.deprecated", "false"); - // General mappings conventions for any data that ends up in a data stream public static final String DATA_STREAMS_MAPPINGS_COMPONENT_TEMPLATE_NAME = "data-streams@mappings"; @@ -134,14 +132,14 @@ private void updateEnabledSetting(boolean newValue) { } private static final List LIFECYCLE_POLICY_CONFIGS = List.of( - new LifecyclePolicyConfig(LOGS_ILM_POLICY_NAME, "/logs@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(METRICS_ILM_POLICY_NAME, "/metrics@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(SYNTHETICS_ILM_POLICY_NAME, "/synthetics@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_7_DAYS_POLICY_NAME, "/7-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_30_DAYS_POLICY_NAME, "/30-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_90_DAYS_POLICY_NAME, "/90-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_180_DAYS_POLICY_NAME, "/180-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES), - new LifecyclePolicyConfig(ILM_365_DAYS_POLICY_NAME, "/365-days@lifecycle.json", ADDITIONAL_TEMPLATE_VARIABLES) + new LifecyclePolicyConfig(LOGS_ILM_POLICY_NAME, "/logs@lifecycle.json"), + new LifecyclePolicyConfig(METRICS_ILM_POLICY_NAME, "/metrics@lifecycle.json"), + new LifecyclePolicyConfig(SYNTHETICS_ILM_POLICY_NAME, "/synthetics@lifecycle.json"), + new LifecyclePolicyConfig(ILM_7_DAYS_POLICY_NAME, "/7-days@lifecycle.json"), + new LifecyclePolicyConfig(ILM_30_DAYS_POLICY_NAME, "/30-days@lifecycle.json"), + new LifecyclePolicyConfig(ILM_90_DAYS_POLICY_NAME, "/90-days@lifecycle.json"), + new LifecyclePolicyConfig(ILM_180_DAYS_POLICY_NAME, "/180-days@lifecycle.json"), + new LifecyclePolicyConfig(ILM_365_DAYS_POLICY_NAME, "/365-days@lifecycle.json") ); @Override @@ -163,64 +161,55 @@ protected List getLifecyclePolicies() { DATA_STREAMS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/data-streams@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( LOGS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/logs@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( ECS_DYNAMIC_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/ecs@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( LOGS_SETTINGS_COMPONENT_TEMPLATE_NAME, "/logs@settings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( METRICS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/metrics@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( METRICS_SETTINGS_COMPONENT_TEMPLATE_NAME, "/metrics@settings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( METRICS_TSDB_SETTINGS_COMPONENT_TEMPLATE_NAME, "/metrics@tsdb-settings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( SYNTHETICS_MAPPINGS_COMPONENT_TEMPLATE_NAME, "/synthetics@mappings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ), new IndexTemplateConfig( SYNTHETICS_SETTINGS_COMPONENT_TEMPLATE_NAME, "/synthetics@settings.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ) )) { try { @@ -241,33 +230,14 @@ protected Map getComponentTemplateConfigs() { } private static final Map COMPOSABLE_INDEX_TEMPLATE_CONFIGS = parseComposableTemplates( - new IndexTemplateConfig( - LOGS_INDEX_TEMPLATE_NAME, - "/logs@template.json", - REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES - ), - new IndexTemplateConfig( - METRICS_INDEX_TEMPLATE_NAME, - "/metrics@template.json", - REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES - ), - new IndexTemplateConfig( - SYNTHETICS_INDEX_TEMPLATE_NAME, - "/synthetics@template.json", - REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES - ), + new IndexTemplateConfig(LOGS_INDEX_TEMPLATE_NAME, "/logs@template.json", REGISTRY_VERSION, TEMPLATE_VERSION_VARIABLE), + new IndexTemplateConfig(METRICS_INDEX_TEMPLATE_NAME, "/metrics@template.json", REGISTRY_VERSION, TEMPLATE_VERSION_VARIABLE), + new IndexTemplateConfig(SYNTHETICS_INDEX_TEMPLATE_NAME, "/synthetics@template.json", REGISTRY_VERSION, TEMPLATE_VERSION_VARIABLE), new IndexTemplateConfig( KIBANA_REPORTING_INDEX_TEMPLATE_NAME, "/kibana-reporting@template.json", REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - ADDITIONAL_TEMPLATE_VARIABLES + TEMPLATE_VERSION_VARIABLE ) ); @@ -281,22 +251,8 @@ protected Map getComposableTemplateConfigs() { } private static final List INGEST_PIPELINE_CONFIGS = List.of( - new JsonIngestPipelineConfig( - "logs@json-pipeline", - "/logs@json-pipeline.json", - REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - List.of(), - ADDITIONAL_TEMPLATE_VARIABLES - ), - new JsonIngestPipelineConfig( - "logs@default-pipeline", - "/logs@default-pipeline.json", - REGISTRY_VERSION, - TEMPLATE_VERSION_VARIABLE, - List.of(), - ADDITIONAL_TEMPLATE_VARIABLES - ) + new JsonIngestPipelineConfig("logs@json-pipeline", "/logs@json-pipeline.json", REGISTRY_VERSION, TEMPLATE_VERSION_VARIABLE), + new JsonIngestPipelineConfig("logs@default-pipeline", "/logs@default-pipeline.json", REGISTRY_VERSION, TEMPLATE_VERSION_VARIABLE) ); @Override diff --git a/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/LegacyStackTemplateRegistryTests.java b/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/LegacyStackTemplateRegistryTests.java deleted file mode 100644 index b8c64f945db0a..0000000000000 --- a/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/LegacyStackTemplateRegistryTests.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.stack; - -import org.elasticsearch.client.internal.Client; -import org.elasticsearch.cluster.metadata.ComponentTemplate; -import org.elasticsearch.cluster.metadata.ComposableIndexTemplate; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.ingest.PipelineConfiguration; -import org.elasticsearch.test.ClusterServiceUtils; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.test.client.NoOpClient; -import org.elasticsearch.threadpool.TestThreadPool; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xcontent.NamedXContentRegistry; -import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.core.ilm.LifecyclePolicy; -import org.junit.After; -import org.junit.Before; - -public class LegacyStackTemplateRegistryTests extends ESTestCase { - private LegacyStackTemplateRegistry registry; - private ThreadPool threadPool; - - @Before - public void createRegistryAndClient() { - threadPool = new TestThreadPool(this.getClass().getName()); - Client client = new NoOpClient(threadPool); - ClusterService clusterService = ClusterServiceUtils.createClusterService(threadPool); - registry = new LegacyStackTemplateRegistry(Settings.EMPTY, clusterService, threadPool, client, NamedXContentRegistry.EMPTY); - } - - @After - @Override - public void tearDown() throws Exception { - super.tearDown(); - threadPool.shutdownNow(); - } - - public void testThatTemplatesAreDeprecated() { - for (ComposableIndexTemplate it : registry.getComposableTemplateConfigs().values()) { - assertTrue(it.isDeprecated()); - } - for (LifecyclePolicy ilm : registry.getLifecyclePolicies()) { - assertTrue(ilm.isDeprecated()); - } - for (ComponentTemplate ct : registry.getComponentTemplateConfigs().values()) { - assertTrue(ct.deprecated()); - } - registry.getIngestPipelines() - .stream() - .map(ipc -> new PipelineConfiguration(ipc.getId(), ipc.loadConfig(), XContentType.JSON)) - .map(PipelineConfiguration::getConfigAsMap) - .forEach(p -> assertTrue((Boolean) p.get("deprecated"))); - } - -} diff --git a/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/StackTemplateRegistryTests.java b/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/StackTemplateRegistryTests.java index 4d68b471a52f8..8e0cbc3f82f35 100644 --- a/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/StackTemplateRegistryTests.java +++ b/x-pack/plugin/stack/src/test/java/org/elasticsearch/xpack/stack/StackTemplateRegistryTests.java @@ -507,23 +507,6 @@ public void testThatNothingIsInstalledWhenAllNodesAreNotUpdated() { registry.clusterChanged(event); } - public void testThatTemplatesAreNotDeprecated() { - for (ComposableIndexTemplate it : registry.getComposableTemplateConfigs().values()) { - assertFalse(it.isDeprecated()); - } - for (LifecyclePolicy ilm : registry.getLifecyclePolicies()) { - assertFalse(ilm.isDeprecated()); - } - for (ComponentTemplate ct : registry.getComponentTemplateConfigs().values()) { - assertFalse(ct.deprecated()); - } - registry.getIngestPipelines() - .stream() - .map(ipc -> new PipelineConfiguration(ipc.getId(), ipc.loadConfig(), XContentType.JSON)) - .map(PipelineConfiguration::getConfigAsMap) - .forEach(p -> assertFalse((Boolean) p.get("deprecated"))); - } - // ------------- /** From 6b72def81b372b15de10f9289fc5a026639ba72c Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Thu, 9 Nov 2023 09:15:47 +0200 Subject: [PATCH 017/513] Fixed JWT principal from claims (#101333) This changes the format of a JWT's principal before the JWT is actually validated by any JWT realm. The JWT's principal is a convenient way to refer to a JWT that has not yet been verified by a JWT realm. The JWT's principal is printed in the audit and regular logs (notably for auditing authn failures), as well as the smart realm chain reordering optimization. The JWT principal is NOT required to be identical to the JWT-authenticated user's principal, but, in general, they should be similar. Previously, the JWT's principal was built by individual realms in the same way the realms built the authenticated user's principal. This had the advantage that, in simpler JWT realms configurations (e.g. a single JWT realm in the chain), the JWT principal and the authenticated user's principal are very similar. However, the drawback is that, in general, the JWT principal and the user principal can be very different (i.e. in the case where one JWT realm builds the JWT principal and a different one builds the user principal). Another downside is that the (unauthenticated) JWT principal depended on realm ordering, which makes identifying the JWT from its principal dependent on the ES authn realm configuration. This PR implements a consistent fixed logic to build the JWT principal, which now only depends on the JWT's claims and no ES configuration. Co-authored-by: Jake Landis jake.landis@elastic.co --- docs/changelog/101333.yaml | 29 ++++ .../authc/jwt/JwtRealmSingleNodeTests.java | 97 ++++++------ .../authc/jwt/JwtAuthenticationToken.java | 54 ++++++- .../xpack/security/authc/jwt/JwtRealm.java | 147 +++--------------- .../xpack/security/authc/jwt/JwtUtil.java | 38 +++++ 5 files changed, 184 insertions(+), 181 deletions(-) create mode 100644 docs/changelog/101333.yaml diff --git a/docs/changelog/101333.yaml b/docs/changelog/101333.yaml new file mode 100644 index 0000000000000..4452687b995d3 --- /dev/null +++ b/docs/changelog/101333.yaml @@ -0,0 +1,29 @@ +pr: 101333 +summary: Fixed JWT principal from claims +area: Authorization +type: breaking +issues: [] +breaking: + title: Fixed JWT principal from claims + area: Authorization + details: "This changes the format of a JWT's principal before the JWT is actually\ + \ validated by any JWT realm. The JWT's principal is a convenient way to refer\ + \ to a JWT that has not yet been verified by a JWT realm. The JWT's principal\ + \ is printed in the audit and regular logs (notably for auditing authn failures)\ + \ as well as the smart realm chain reordering optimization. The JWT principal\ + \ is NOT required to be identical to the JWT-authenticated user's principal, but\ + \ in general, they should be similar. Previously, the JWT's principal was built\ + \ by individual realms in the same way the realms built the authenticated user's\ + \ principal. This had the advantage that, in simpler JWT realms configurations\ + \ (e.g. a single JWT realm in the chain), the JWT principal and the authenticated\ + \ user's principal are very similar. However the drawback is that, in general,\ + \ the JWT principal and the user principal can be very different (i.e. in the\ + \ case where one JWT realm builds the JWT principal and a different one builds\ + \ the user principal). Another downside is that the (unauthenticated) JWT principal\ + \ depended on realm ordering, which makes identifying the JWT from its principal\ + \ dependent on the ES authn realm configuration. This PR implements a consistent\ + \ fixed logic to build the JWT principal, which only depends on the JWT's claims\ + \ and no ES configuration." + impact: "Users will observe changed format and values for the `user.name` attribute\ + \ of `authentication_failed` audit log events, in the JWT (failed) authn case." + notable: false diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/jwt/JwtRealmSingleNodeTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/jwt/JwtRealmSingleNodeTests.java index a738e18b0fb10..d84b93fa6f638 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/jwt/JwtRealmSingleNodeTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authc/jwt/JwtRealmSingleNodeTests.java @@ -188,53 +188,56 @@ public void testInvalidJWTDoesNotFallbackToAnonymousAccess() throws Exception { } public void testAnyJwtRealmWillExtractTheToken() throws ParseException { - final List jwtRealms = getJwtRealms(); - final JwtRealm jwtRealm = randomFrom(jwtRealms); - - final String sharedSecret = randomBoolean() ? randomAlphaOfLengthBetween(10, 20) : null; - final String iss = randomAlphaOfLengthBetween(5, 18); - final String aud = randomAlphaOfLengthBetween(5, 18); - final String sub = randomAlphaOfLengthBetween(5, 18); - - // Realm 1 will extract the token because the JWT has all iss, sub, aud, principal claims. - // Their values do not match what realm 1 expects but that does not matter when extracting the token - final SignedJWT signedJWT1 = getSignedJWT(Map.of("iss", iss, "aud", aud, "sub", sub)); - final ThreadContext threadContext1 = prepareThreadContext(signedJWT1, sharedSecret); - final var token1 = (JwtAuthenticationToken) jwtRealm.token(threadContext1); - final String principal1 = Strings.format("%s/%s/%s/%s", iss, aud, sub, sub); - assertJwtToken(token1, principal1, sharedSecret, signedJWT1); - - // Realm 2 for extracting the token from the following JWT - // Because it does not have the sub claim but client_id, which is configured as fallback by realm 2 - final String appId = randomAlphaOfLengthBetween(5, 18); - final SignedJWT signedJWT2 = getSignedJWT(Map.of("iss", iss, "aud", aud, "client_id", sub, "appid", appId)); - final ThreadContext threadContext2 = prepareThreadContext(signedJWT2, sharedSecret); - final var token2 = (JwtAuthenticationToken) jwtRealm.token(threadContext2); - final String principal2 = Strings.format("%s/%s/%s/%s", iss, aud, sub, appId); - assertJwtToken(token2, principal2, sharedSecret, signedJWT2); - - // Realm 3 will extract the token from the following JWT - // Because it has the oid claim which is configured as a fallback by realm 3 - final String email = randomAlphaOfLengthBetween(5, 18) + "@example.com"; - final SignedJWT signedJWT3 = getSignedJWT(Map.of("iss", iss, "aud", aud, "oid", sub, "email", email)); - final ThreadContext threadContext3 = prepareThreadContext(signedJWT3, sharedSecret); - final var token3 = (JwtAuthenticationToken) jwtRealm.token(threadContext3); - final String principal3 = Strings.format("%s/%s/%s/%s", iss, aud, sub, email); - assertJwtToken(token3, principal3, sharedSecret, signedJWT3); - - // The JWT does not match any realm's configuration, a token with generic token principal will be extracted - final SignedJWT signedJWT4 = getSignedJWT(Map.of("iss", iss, "aud", aud, "azp", sub, "email", email)); - final ThreadContext threadContext4 = prepareThreadContext(signedJWT4, sharedSecret); - final var token4 = (JwtAuthenticationToken) jwtRealm.token(threadContext4); - final String principal4 = Strings.format(" by %s", iss); - assertJwtToken(token4, principal4, sharedSecret, signedJWT4); - - // The JWT does not have an issuer, a token with generic token principal will be extracted - final SignedJWT signedJWT5 = getSignedJWT(Map.of("aud", aud, "sub", sub)); - final ThreadContext threadContext5 = prepareThreadContext(signedJWT5, sharedSecret); - final var token5 = (JwtAuthenticationToken) jwtRealm.token(threadContext5); - final String principal5 = ""; - assertJwtToken(token5, principal5, sharedSecret, signedJWT5); + for (JwtRealm jwtRealm : getJwtRealms()) { + final String sharedSecret = randomBoolean() ? randomAlphaOfLengthBetween(10, 20) : null; + final String iss = randomAlphaOfLengthBetween(5, 18); + final List aud = List.of(randomAlphaOfLengthBetween(5, 18), randomAlphaOfLengthBetween(5, 18)); + final String sub = randomAlphaOfLengthBetween(5, 18); + + // JWT1 has all iss, sub, aud, principal claims. + final SignedJWT signedJWT1 = getSignedJWT(Map.of("iss", iss, "aud", aud, "sub", sub)); + final ThreadContext threadContext1 = prepareThreadContext(signedJWT1, sharedSecret); + final var token1 = (JwtAuthenticationToken) jwtRealm.token(threadContext1); + final String principal1 = Strings.format("'aud:%s,%s' 'iss:%s' 'sub:%s'", aud.get(0), aud.get(1), iss, sub); + assertJwtToken(token1, principal1, sharedSecret, signedJWT1); + + // JWT2, JWT3, and JWT4 don't have the sub claim. + // Some realms define fallback claims for the sub claim (which themselves might not exist), + // but that is not relevant for token building (it's used for user principal assembling). + final String appId = randomAlphaOfLengthBetween(5, 18); + final SignedJWT signedJWT2 = getSignedJWT(Map.of("iss", iss, "aud", aud, "client_id", sub, "appid", appId)); + final ThreadContext threadContext2 = prepareThreadContext(signedJWT2, sharedSecret); + final var token2 = (JwtAuthenticationToken) jwtRealm.token(threadContext2); + final String principal2 = Strings.format( + "'appid:%s' 'aud:%s,%s' 'client_id:%s' 'iss:%s'", + appId, + aud.get(0), + aud.get(1), + sub, + iss + ); + assertJwtToken(token2, principal2, sharedSecret, signedJWT2); + + final String email = randomAlphaOfLengthBetween(5, 18) + "@example.com"; + final SignedJWT signedJWT3 = getSignedJWT(Map.of("iss", iss, "aud", aud, "oid", sub, "email", email)); + final ThreadContext threadContext3 = prepareThreadContext(signedJWT3, sharedSecret); + final var token3 = (JwtAuthenticationToken) jwtRealm.token(threadContext3); + final String principal3 = Strings.format("'aud:%s,%s' 'email:%s' 'iss:%s' 'oid:%s'", aud.get(0), aud.get(1), email, iss, sub); + assertJwtToken(token3, principal3, sharedSecret, signedJWT3); + + final SignedJWT signedJWT4 = getSignedJWT(Map.of("iss", iss, "aud", aud, "azp", sub, "email", email)); + final ThreadContext threadContext4 = prepareThreadContext(signedJWT4, sharedSecret); + final var token4 = (JwtAuthenticationToken) jwtRealm.token(threadContext4); + final String principal4 = Strings.format("'aud:%s,%s' 'azp:%s' 'email:%s' 'iss:%s'", aud.get(0), aud.get(1), sub, email, iss); + assertJwtToken(token4, principal4, sharedSecret, signedJWT4); + + // JWT5 does not have an issuer. + final SignedJWT signedJWT5 = getSignedJWT(Map.of("aud", aud, "sub", sub)); + final ThreadContext threadContext5 = prepareThreadContext(signedJWT5, sharedSecret); + final var token5 = (JwtAuthenticationToken) jwtRealm.token(threadContext5); + final String principal5 = Strings.format("'aud:%s,%s' 'sub:%s'", aud.get(0), aud.get(1), sub); + assertJwtToken(token5, principal5, sharedSecret, signedJWT5); + } } public void testJwtRealmReturnsNullTokenWhenJwtCredentialIsAbsent() { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtAuthenticationToken.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtAuthenticationToken.java index a80865428e592..9ca0ddb42e663 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtAuthenticationToken.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtAuthenticationToken.java @@ -15,36 +15,35 @@ import java.text.ParseException; import java.util.Arrays; +import java.util.List; import java.util.Objects; +import java.util.TreeSet; /** * An {@link AuthenticationToken} to hold JWT authentication related content. */ public class JwtAuthenticationToken implements AuthenticationToken { - private final String principal; private SignedJWT signedJWT; + private final String principal; private final byte[] userCredentialsHash; @Nullable private final SecureString clientAuthenticationSharedSecret; /** * Store a mandatory JWT and optional Shared Secret. - * @param principal The token's principal, useful as a realm order cache key * @param signedJWT The JWT parsed from the end-user credentials * @param userCredentialsHash The hash of the end-user credentials is used to compute the key for user cache at the realm level. * See also {@link JwtRealm#authenticate}. * @param clientAuthenticationSharedSecret URL-safe Shared Secret for Client authentication. Required by some JWT realms. */ public JwtAuthenticationToken( - String principal, SignedJWT signedJWT, byte[] userCredentialsHash, @Nullable final SecureString clientAuthenticationSharedSecret ) { - this.principal = Objects.requireNonNull(principal); this.signedJWT = Objects.requireNonNull(signedJWT); + this.principal = buildTokenPrincipal(); this.userCredentialsHash = Objects.requireNonNull(userCredentialsHash); - if ((clientAuthenticationSharedSecret != null) && (clientAuthenticationSharedSecret.isEmpty())) { throw new IllegalArgumentException("Client shared secret must be non-empty"); } @@ -70,7 +69,7 @@ public JWTClaimsSet getJWTClaimsSet() { return signedJWT.getJWTClaimsSet(); } catch (ParseException e) { assert false : "The JWT claims set should have already been successfully parsed before building the JWT authentication token"; - throw new IllegalArgumentException(e); + throw new IllegalStateException(e); } } @@ -95,4 +94,47 @@ public void clearCredentials() { public String toString() { return JwtAuthenticationToken.class.getSimpleName() + "=" + this.principal; } + + private String buildTokenPrincipal() { + JWTClaimsSet jwtClaimsSet = getJWTClaimsSet(); + StringBuilder principalBuilder = new StringBuilder(); + claimsLoop: for (String claimName : new TreeSet<>(jwtClaimsSet.getClaims().keySet())) { + Object claimValue = jwtClaimsSet.getClaim(claimName); + if (claimValue == null) { + continue; + } + // only use String or String[] claim values to assemble the principal + if (claimValue instanceof String) { + if (principalBuilder.isEmpty() == false) { + principalBuilder.append(' '); + } + principalBuilder.append('\'').append(claimName).append(':').append((String) claimValue).append('\''); + } else if (claimValue instanceof List) { + List claimValuesList = (List) claimValue; + if (claimValuesList.isEmpty()) { + continue; + } + for (Object claimValueElem : claimValuesList) { + if (claimValueElem instanceof String == false) { + continue claimsLoop; + } + } + if (principalBuilder.isEmpty() == false) { + principalBuilder.append(' '); + } + principalBuilder.append('\'').append(claimName).append(':'); + for (int i = 0; i < claimValuesList.size(); i++) { + if (i > 0) { + principalBuilder.append(','); + } + principalBuilder.append((String) claimValuesList.get(i)); + } + principalBuilder.append('\''); + } + } + if (principalBuilder.isEmpty()) { + return ""; + } + return principalBuilder.toString(); + } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtRealm.java index dea471846b9f4..eb2517f8e54e4 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtRealm.java @@ -39,8 +39,6 @@ import org.elasticsearch.xpack.security.authc.support.ClaimParser; import org.elasticsearch.xpack.security.authc.support.DelegatedAuthorizationSupport; -import java.text.ParseException; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Date; @@ -48,8 +46,6 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.TreeSet; -import java.util.function.Function; import static java.lang.String.join; import static org.elasticsearch.core.Strings.format; @@ -61,6 +57,8 @@ */ public class JwtRealm extends Realm implements CachingRealm, Releasable { + private static final String LATEST_MALFORMED_JWT = "_latest_malformed_jwt"; + private static final Logger logger = LogManager.getLogger(JwtRealm.class); public static final String HEADER_END_USER_AUTHENTICATION = "Authorization"; @@ -82,7 +80,6 @@ public class JwtRealm extends Realm implements CachingRealm, Releasable { private final JwtAuthenticator jwtAuthenticator; private final TimeValue allowedClockSkew; DelegatedAuthorizationSupport delegatedAuthorizationSupport = null; - private List> tokenPrincipalFunctions; public JwtRealm(final RealmConfig realmConfig, final SSLService sslService, final UserRoleMapper userRoleMapper) throws SettingsException { @@ -136,7 +133,6 @@ public JwtRealm(final RealmConfig realmConfig, final SSLService sslService, fina /** * If X-pack licensing allows it, initialize delegated authorization support. - * JWT realm will use the list of all realms to link to its named authorization realms. * @param allRealms List of all realms containing authorization realms for this JWT realm. * @param xpackLicenseState X-pack license state. */ @@ -147,14 +143,6 @@ public void initialize(final Iterable allRealms, final XPackLicenseState } // extract list of realms referenced by config.settings() value for DelegatedAuthorizationSettings.ROLES_REALMS delegatedAuthorizationSupport = new DelegatedAuthorizationSupport(allRealms, config, xpackLicenseState); - - final List> tokenPrincipalFunctions = new ArrayList<>(); - for (var realm : allRealms) { - if (realm instanceof final JwtRealm jwtRealm) { - tokenPrincipalFunctions.add(jwtRealm::buildTokenPrincipal); - } - } - this.tokenPrincipalFunctions = List.copyOf(tokenPrincipalFunctions); } /** @@ -191,69 +179,24 @@ public void expireAll() { @Override public AuthenticationToken token(final ThreadContext threadContext) { ensureInitialized(); - final SecureString userCredentials = JwtUtil.getHeaderValue( threadContext, JwtRealm.HEADER_END_USER_AUTHENTICATION, JwtRealm.HEADER_END_USER_AUTHENTICATION_SCHEME, false ); - if (userCredentials == null || userCredentials.isEmpty()) { - return null; - } - - // a lightweight pre-check for JWTs - if (containsAtLeastTwoDots(userCredentials) == false) { - return null; - } - - // custom realms can also consume the Bearer credentials scheme - final SignedJWT signedJWT; - try { - signedJWT = SignedJWT.parse(userCredentials.toString()); - } catch (ParseException e) { - logger.debug("Failed to parse JWT bearer token", e); + SignedJWT signedJWT = parseSignedJWT(userCredentials, threadContext); + if (signedJWT == null) { + // this is not a valid JWT for ES realms, but custom realms can also consume the Bearer credentials scheme in their own format return null; } - final JWTClaimsSet jwtClaimsSet; - try { - jwtClaimsSet = signedJWT.getJWTClaimsSet(); - } catch (ParseException e) { - logger.debug("Failed to parse JWT claims set", e); - return null; - } - final SecureString clientCredentials = JwtUtil.getHeaderValue( threadContext, JwtRealm.HEADER_CLIENT_AUTHENTICATION, JwtRealm.HEADER_SHARED_SECRET_AUTHENTICATION_SCHEME, true ); - - // If Issuer is not found, still return a JWT token since it is after still a JWT, authentication - // will fail later because issuer is mandated - final String issuer = jwtClaimsSet.getIssuer(); - if (Strings.hasText(issuer) == false) { - logger.warn("Issuer claim 'iss' is missing."); - return new JwtAuthenticationToken("", signedJWT, JwtUtil.sha256(userCredentials), clientCredentials); - } - - // Try all known extraction functions to build the token principal - for (Function func : tokenPrincipalFunctions) { - final String tokenPrincipalSuffix = func.apply(jwtClaimsSet); - if (tokenPrincipalSuffix != null) { - return new JwtAuthenticationToken( - issuer + "/" + tokenPrincipalSuffix, - signedJWT, - JwtUtil.sha256(userCredentials), - clientCredentials - ); - } - } - - // Token principal cannot be extracted even after trying all functions, but this is - // still a JWT token so that we should return as one. - return new JwtAuthenticationToken(" by " + issuer, signedJWT, JwtUtil.sha256(userCredentials), clientCredentials); + return new JwtAuthenticationToken(signedJWT, JwtUtil.sha256(userCredentials), clientCredentials); } @Override @@ -459,11 +402,6 @@ public void rotateClientSecret(SecureString clientSecret) { this.clientAuthenticationSharedSecret.rotate(clientSecret, config.getSetting(CLIENT_AUTH_SHARED_SECRET_ROTATION_GRACE_PERIOD)); } - // package private for testing - RotatableSecret getClientAuthenticationSharedSecret() { - return clientAuthenticationSharedSecret; - } - /** * Clean up JWT cache (if enabled). */ @@ -509,52 +447,22 @@ private Map buildUserMetadata(JWTClaimsSet claimsSet) { return Map.copyOf(metadata); } - // We construct the token principal as a function of the JWT realm configuration. We also short circuit the extraction of the - // token principal while we iterate through the realms. For realms like the file realm this is not an issue since there is only - // one file realm. For realms like LDAP this is also not an issue since the token principal is identical across all realms regardless - // of how the realm is configured. However, for realms like JWT (and PKI realm) where the token principal is a function of the - // realm configuration AND multiple realms of that type can exist this can be an issue. This is an issue because realm1 might - // result in the token principal "abc", but realm2 (same JWT) might result in the token principal as "xyz". Since we short circuit the - // extraction of the token principal (i.e. use the first one that does not error) then the same JWT token can result in a - // token principal of either "abc" or "xyz" depending on which came first. This means that we can not rely on the value calculated here - // to be logically correct within the context of a given realm. The value is technically correct as the value is a function of - // the JWT itself, but which function (from realm1 or realm2) can not be known. The value emitted here should be used judiciously. - private String buildTokenPrincipal(JWTClaimsSet jwtClaimsSet) { - final Map fallbackClaimNames = jwtAuthenticator.getFallbackClaimNames(); - final FallbackableClaim subClaim = new FallbackableClaim("sub", fallbackClaimNames, jwtClaimsSet); - final String subject = subClaim.getStringClaimValue(); - if (false == Strings.hasText(subject)) { - logger.debug( - "Can not build token principal for realm [{}] since claim [{}] is missing from claimSet [{}]", - name(), - subClaim, - jwtClaimsSet - ); - return null; - } - - final FallbackableClaim audClaim = new FallbackableClaim("aud", fallbackClaimNames, jwtClaimsSet); - final List audiences = audClaim.getStringListClaimValue(); - if (audiences == null || audiences.isEmpty()) { - logger.debug( - "Can not build token principal for realm [{}] since claim [{}] is missing from claimSet [{}]", - name(), - subClaim, - jwtClaimsSet - ); + /** + * Parses a {@link SignedJWT} from the provided {@param token}. + * This internally, for the local thread, remembers the last **malformed** token parsed, + * in order to avoid attempting to parse the same token multiple consecutive times (by different JWT realms in the chain). + */ + private SignedJWT parseSignedJWT(SecureString token, ThreadContext threadContext) { + if (Objects.equals(token, threadContext.getTransient(LATEST_MALFORMED_JWT))) { + // already tried to parse this token and it didn't work return null; } - - final String userPrincipal = claimParserPrincipal.getClaimValue(jwtClaimsSet); - if (false == Strings.hasText(userPrincipal)) { - logger.debug( - "Can not build token principal for realm [{}] since no user principal can be extracted with [{}]", - name(), - claimParserPrincipal - ); - return null; + SignedJWT signedJWT = JwtUtil.parseSignedJWT(token); + if (signedJWT == null) { + // this is a malformed JWT, update the latest malformed token reference + threadContext.putTransient(LATEST_MALFORMED_JWT, token); } - return String.join(",", new TreeSet<>(audiences)) + "/" + subject + "/" + userPrincipal; + return signedJWT; } /** @@ -581,21 +489,4 @@ record ExpiringUser(User user, Date exp) { Objects.requireNonNull(exp, "Expiration date must not be null"); } } - - /** - * This is a lightweight pre-check for the JWT token format. - * If this returns {@code true}, the token MIGHT be a JWT. Otherwise, the token is definitely not a JWT. - */ - private static boolean containsAtLeastTwoDots(SecureString secureString) { - if (secureString == null || secureString.length() < 2) { - return false; - } - int ndots = 0; - for (int i = 0; i < secureString.length(); i++) { - if (secureString.charAt(i) == '.' && ++ndots >= 2) { - return true; - } - } - return false; - } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtUtil.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtUtil.java index 9168c5c0925bd..928ecd7fa265d 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtUtil.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/jwt/JwtUtil.java @@ -13,6 +13,7 @@ import com.nimbusds.jose.util.Base64URL; import com.nimbusds.jose.util.JSONObjectUtils; import com.nimbusds.jwt.JWT; +import com.nimbusds.jwt.SignedJWT; import org.apache.http.HttpEntity; import org.apache.http.HttpResponse; @@ -59,6 +60,7 @@ import java.security.PrivilegedAction; import java.security.PrivilegedActionException; import java.security.PrivilegedExceptionAction; +import java.text.ParseException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -354,6 +356,25 @@ public static byte[] sha256(final CharSequence charSequence) { return messageDigest.digest(); } + public static SignedJWT parseSignedJWT(SecureString token) { + if (token == null || token.isEmpty()) { + return null; + } + // a lightweight pre-check for JWTs + if (containsAtLeastTwoDots(token) == false) { + return null; + } + try { + SignedJWT signedJWT = SignedJWT.parse(token.toString()); + // trigger claim set parsing (the parsed version will be cached internally) + signedJWT.getJWTClaimsSet(); + return signedJWT; + } catch (ParseException e) { + LOGGER.debug("Failed to parse JWT bearer token", e); + return null; + } + } + /** * Helper class to consolidate multiple trace level statements to a single trace statement with lazy evaluation. * If trace level is not enabled, then no work is performed. This class is not threadsafe and is not intended for a long lifecycle. @@ -412,4 +433,21 @@ public static Supplier toStringRedactSignature(JWT jwt) { return jwt::getParsedString; } } + + /** + * This is a lightweight pre-check for the JWT token format. + * If this returns {@code true}, the token MIGHT be a JWT. Otherwise, the token is definitely not a JWT. + */ + private static boolean containsAtLeastTwoDots(SecureString secureString) { + if (secureString == null || secureString.length() < 2) { + return false; + } + int ndots = 0; + for (int i = 0; i < secureString.length(); i++) { + if (secureString.charAt(i) == '.' && ++ndots >= 2) { + return true; + } + } + return false; + } } From 1efb62b8de5acd056466f4dd68fa4be7223d45b4 Mon Sep 17 00:00:00 2001 From: Ievgen Degtiarenko Date: Thu, 9 Nov 2023 09:14:32 +0100 Subject: [PATCH 018/513] Remove AwaitsFix for issues that marked as completed (#101919) --- .../elasticsearch/action/admin/indices/create/CreateIndexIT.java | 1 - .../org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java | 1 - .../java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java | 1 - .../org/elasticsearch/index/mapper/NumberFieldTypeTests.java | 1 - .../aggregations/metrics/HDRPercentilesAggregatorTests.java | 1 - 5 files changed, 5 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java index 4b395ec6856e5..d19c61f97efd9 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java @@ -206,7 +206,6 @@ public void testInvalidShardCountSettingsWithoutPrefix() throws Exception { } } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/96578") public void testCreateAndDeleteIndexConcurrently() throws InterruptedException { createIndex("test"); final AtomicInteger indexVersion = new AtomicInteger(0); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java index 206b866bd4758..a12f7feb05b48 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java @@ -377,7 +377,6 @@ private void assertEngineTypes() { } } - @AwaitsFix(bugUrl = "ES-4677") public void testRelocation() { var routingTableWatcher = new RoutingTableWatcher(); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java index 32a1d6724e0fd..1f86d4cb39ea4 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/FeatureStateResetApiIT.java @@ -48,7 +48,6 @@ protected Collection> nodePlugins() { } /** Check that the reset method cleans up a feature */ - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/97780") public void testResetSystemIndices() throws Exception { String systemIndex1 = ".test-system-idx-1"; String systemIndex2 = ".second-test-system-idx-1"; diff --git a/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldTypeTests.java index 20b9661ad0b42..40d1f2488749a 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/NumberFieldTypeTests.java @@ -573,7 +573,6 @@ public void doTestDocValueRangeQueries(NumberType type, Supplier valueSu dir.close(); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/74057") public void testIndexSortIntRange() throws Exception { doTestIndexSortRangeQueries(NumberType.INTEGER, random()::nextInt); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesAggregatorTests.java index 2e410cf0a10f5..1f61f507e88a6 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesAggregatorTests.java @@ -77,7 +77,6 @@ public void testStringField() throws IOException { /** * Attempting to use HDRPercentileAggregation on a range field throws IllegalArgumentException */ - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/42949") public void testRangeField() throws IOException { // Currently fails (throws ClassCast exception), but should be fixed once HDRPercentileAggregation uses the ValuesSource registry final String fieldName = "range"; From 3a09c64ef453e811c6e62099f8a5286e9d62db00 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lorenzo=20Dematt=C3=A9?= Date: Thu, 9 Nov 2023 10:01:12 +0100 Subject: [PATCH 019/513] Muting DownsampleActionIT testDownsampleTwice (#101946) Mute https://github.com/elastic/elasticsearch/issues/101428 --- .../org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java index 2d9d4163102b8..3ca8c7302d6dd 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/javaRestTest/java/org/elasticsearch/xpack/ilm/actions/DownsampleActionIT.java @@ -406,6 +406,7 @@ public void testRollupNonTSIndex() throws Exception { assertTrue("Source index should not have been deleted", indexExists(index)); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/101428") public void testDownsampleTwice() throws Exception { // Create the ILM policy Request request = new Request("PUT", "_ilm/policy/" + policy); From cefa750b9409608e9a50bccec39a5eef08c925e6 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 9 Nov 2023 12:02:39 +0000 Subject: [PATCH 020/513] Fail listener on exception in TcpTransport#openConnection (#101907) Today `TcpTransport#openConnection` may throw exceptions on certain kinds of failure, but other kinds of failure are passed to the listener. This is trappy and not all callers handle it correctly. This commit makes sure that all exceptions are passed to the listener. Closes #100510 --- docs/changelog/101907.yaml | 6 ++++ .../elasticsearch/transport/TcpTransport.java | 35 ++++++++++--------- .../AbstractSimpleTransportTestCase.java | 26 ++++++++++++++ 3 files changed, 50 insertions(+), 17 deletions(-) create mode 100644 docs/changelog/101907.yaml diff --git a/docs/changelog/101907.yaml b/docs/changelog/101907.yaml new file mode 100644 index 0000000000000..022c061555be1 --- /dev/null +++ b/docs/changelog/101907.yaml @@ -0,0 +1,6 @@ +pr: 101907 +summary: Fail listener on exception in `TcpTransport#openConnection` +area: Network +type: bug +issues: + - 100510 diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index 9bf773d083f5f..8612b5221c77b 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -362,23 +362,24 @@ protected Recycler createRecycler(Settings settings, PageCacheRecycler @Override public void openConnection(DiscoveryNode node, ConnectionProfile profile, ActionListener listener) { - - Objects.requireNonNull(profile, "connection profile cannot be null"); - if (node == null) { - throw new ConnectTransportException(null, "can't open connection to a null node"); - } - ConnectionProfile finalProfile = maybeOverrideConnectionProfile(profile); - if (closeLock.readLock().tryLock() == false) { - ensureOpen(); - assert false : "should not get here ever because close-write-lock should only be held on shutdown"; - throw new ConnectTransportException(node, "failed to acquire close-read-lock"); - } - try { - ensureOpen(); - initiateConnection(node, finalProfile, listener); - } finally { - closeLock.readLock().unlock(); - } + ActionListener.run(listener, l -> { + Objects.requireNonNull(profile, "connection profile cannot be null"); + if (node == null) { + throw new ConnectTransportException(null, "can't open connection to a null node"); + } + final var finalProfile = maybeOverrideConnectionProfile(profile); + if (closeLock.readLock().tryLock() == false) { + ensureOpen(); + assert false : "should not get here ever because close-write-lock should only be held on shutdown"; + throw new ConnectTransportException(node, "failed to acquire close-read-lock"); + } + try { + ensureOpen(); + initiateConnection(node, finalProfile, l); + } finally { + closeLock.readLock().unlock(); + } + }); } private void initiateConnection(DiscoveryNode node, ConnectionProfile connectionProfile, ActionListener listener) { diff --git a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java index f9085ec258627..ea9dd001e5ce8 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java @@ -25,6 +25,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodeUtils; import org.elasticsearch.cluster.node.VersionInformation; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.component.Lifecycle; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -881,6 +882,31 @@ public void handleException(TransportException exp) { assertThat(e.getCause().getCause().getMessage(), equalTo("runtime_exception: bad message !!!")); } + public void testExceptionOnConnect() { + final var transportA = serviceA.getOriginalTransport(); + + final var nullProfileFuture = new PlainActionFuture(); + transportA.openConnection(nodeB, null, nullProfileFuture); + assertTrue(nullProfileFuture.isDone()); + expectThrows(ExecutionException.class, NullPointerException.class, nullProfileFuture::get); + + final var profile = ConnectionProfile.buildDefaultConnectionProfile(Settings.EMPTY); + final var nullNodeFuture = new PlainActionFuture(); + transportA.openConnection(null, profile, nullNodeFuture); + assertTrue(nullNodeFuture.isDone()); + expectThrows(ExecutionException.class, ConnectTransportException.class, nullNodeFuture::get); + + serviceA.stop(); + assertEquals(Lifecycle.State.STOPPED, transportA.lifecycleState()); + serviceA.close(); + assertEquals(Lifecycle.State.CLOSED, transportA.lifecycleState()); + + final var closedTransportFuture = new PlainActionFuture(); + transportA.openConnection(nodeB, profile, closedTransportFuture); + assertTrue(closedTransportFuture.isDone()); + expectThrows(ExecutionException.class, IllegalStateException.class, closedTransportFuture::get); + } + public void testDisconnectListener() throws Exception { final CountDownLatch latch = new CountDownLatch(1); TransportConnectionListener disconnectListener = new TransportConnectionListener() { From 2a4414ddd2c72211ac2287e356aec8a8fd43b91f Mon Sep 17 00:00:00 2001 From: Abdon Pijpelink Date: Thu, 9 Nov 2023 13:19:13 +0100 Subject: [PATCH 021/513] [DOCS] Add multivalue limitations to 'Limitations' page (#101952) --- docs/reference/esql/esql-limitations.asciidoc | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/docs/reference/esql/esql-limitations.asciidoc b/docs/reference/esql/esql-limitations.asciidoc index 96103fc135271..f2b5c13aaa6f5 100644 --- a/docs/reference/esql/esql-limitations.asciidoc +++ b/docs/reference/esql/esql-limitations.asciidoc @@ -136,6 +136,15 @@ now() - 2023-10-26 include::esql-enrich-data.asciidoc[tag=limitations] +[discrete] +[[esql-limitations-mv]] +=== Multivalue limitations + +{esql} <>, but functions +return `null` when applied to a multivalued field, unless documented otherwise. +Work around this limitation by converting the field to single value with one of +the <>. + [discrete] [[esql-limitations-kibana]] === Kibana limitations From a32aae87d4ce51c9125d618d5295ce8ce9d8d7ce Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 9 Nov 2023 12:21:00 +0000 Subject: [PATCH 022/513] Improve logging when restoring in SnapshotStressTestsIT (#101953) Today the log messages about the steps leading up to a restore suggest that we are closing/deleting all the indices to be restored, which is confusing. This commit fixes the messages to specify the indices being closed/deleted, but also describes the restore plan to which they relate. --- .../snapshots/SnapshotStressTestsIT.java | 35 ++++++++++++------- 1 file changed, 22 insertions(+), 13 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java index 4721b1a186a99..7eaa49b27007d 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java @@ -55,6 +55,7 @@ import java.nio.file.Path; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashSet; @@ -496,6 +497,11 @@ private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePrevio final String[] indicesToRestore = indicesToRestoreList.toArray(new String[0]); final String[] indicesToClose = indicesToCloseList.toArray(new String[0]); final String[] indicesToDelete = indicesToDeleteList.toArray(new String[0]); + final String indicesToRestoreDescription = (restoreSpecificIndices ? "" : "*=") + Arrays.toString(indicesToRestore); + + if (restoreSpecificIndices == false) { + assertEquals(Set.copyOf(snapshotInfo.indices()), Set.of(indicesToRestore)); + } final ListenableFuture closeIndicesStep = new ListenableFuture<>(); final ListenableFuture deleteIndicesStep = new ListenableFuture<>(); @@ -515,15 +521,17 @@ private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePrevio ); logger.info( - "--> closing indices {} in preparation for restoring from [{}:{}]", - indicesToRestoreList, + "--> closing indices {} in preparation for restoring {} from [{}:{}]", + indicesToClose, + indicesToRestoreDescription, snapshotInfo.repository(), snapshotInfo.snapshotId().getName() ); indicesAdmin().prepareClose(indicesToClose).execute(mustSucceed(closeIndexResponse -> { logger.info( - "--> finished closing indices {} in preparation for restoring from [{}:{}]", - indicesToRestoreList, + "--> finished closing indices {} in preparation for restoring {} from [{}:{}]", + indicesToClose, + indicesToRestoreDescription, snapshotInfo.repository(), snapshotInfo.snapshotId().getName() ); @@ -538,15 +546,17 @@ private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePrevio if (indicesToDelete.length > 0) { logger.info( - "--> deleting indices {} in preparation for restoring from [{}:{}]", - indicesToRestoreList, + "--> deleting indices {} in preparation for restoring {} from [{}:{}]", + indicesToDelete, + indicesToRestore, snapshotInfo.repository(), snapshotInfo.snapshotId().getName() ); indicesAdmin().prepareDelete(indicesToDelete).execute(mustSucceed(deleteIndicesResponse -> { logger.info( - "--> finished deleting indices {} in preparation for restoring from [{}:{}]", - indicesToRestoreList, + "--> finished deleting indices {} in preparation for restoring {} from [{}:{}]", + indicesToDelete, + indicesToRestoreDescription, snapshotInfo.repository(), snapshotInfo.snapshotId().getName() ); @@ -569,9 +579,8 @@ private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePrevio } logger.info( - "--> restoring indices {}{} from [{}:{}]", - restoreSpecificIndices ? "" : "*=", - indicesToRestoreList, + "--> restoring indices {} from [{}:{}]", + indicesToRestoreDescription, snapshotInfo.repository(), snapshotInfo.snapshotId().getName() ); @@ -579,7 +588,7 @@ private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePrevio restoreSnapshotRequestBuilder.execute(mustSucceed(restoreSnapshotResponse -> { logger.info( "--> triggered restore of indices {} from [{}:{}], waiting for green health", - indicesToRestoreList, + indicesToRestoreDescription, snapshotInfo.repository(), snapshotInfo.snapshotId().getName() ); @@ -590,7 +599,7 @@ private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePrevio logger.info( "--> indices {} successfully restored from [{}:{}]", - indicesToRestoreList, + indicesToRestoreDescription, snapshotInfo.repository(), snapshotInfo.snapshotId().getName() ); From 433517ad0132b13268cbc730e647946abe698399 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 9 Nov 2023 13:46:00 +0100 Subject: [PATCH 023/513] Misc cleanup in o.e.search.fetch (#101939) Just some random findings from researching other things. Removing all kinds of dead code and fixing obvious duplication in 2 spots. --- .../join/query/ChildQuerySearchIT.java | 10 ++-- .../elasticsearch/join/query/InnerHitsIT.java | 7 ++- .../AnnotatedTextHighlighter.java | 2 +- .../aggregations/metrics/TopHitsIT.java | 12 ++--- .../search/fetch/subphase/InnerHitsIT.java | 7 ++- .../highlight/HighlighterSearchIT.java | 50 +++++++++---------- .../org/elasticsearch/search/SearchHit.java | 2 +- .../search/dfs/AggregatedDfs.java | 10 ++-- .../search/fetch/FetchPhase.java | 2 +- .../search/fetch/StoredFieldsContext.java | 4 +- .../search/fetch/StoredFieldsSpec.java | 4 +- .../fetch/subphase/FetchSourceContext.java | 50 ++++++++----------- .../highlight/AbstractHighlighterBuilder.java | 7 --- .../subphase/highlight/CustomQueryScorer.java | 18 ------- .../highlight/DefaultHighlighter.java | 5 +- .../highlight/FastVectorHighlighter.java | 2 +- .../highlight/FragmentBuilderHelper.java | 25 +--------- .../subphase/highlight/HighlightBuilder.java | 13 ----- .../subphase/highlight/HighlightField.java | 35 ++----------- .../SourceScoreOrderFragmentsBuilder.java | 17 +------ .../SourceSimpleFragmentsBuilder.java | 6 ++- .../elasticsearch/search/SearchHitTests.java | 2 +- .../highlight/HighlightFieldTests.java | 8 +-- .../search/fetch/HighlighterTestCase.java | 3 +- 24 files changed, 95 insertions(+), 206 deletions(-) diff --git a/modules/parent-join/src/internalClusterTest/java/org/elasticsearch/join/query/ChildQuerySearchIT.java b/modules/parent-join/src/internalClusterTest/java/org/elasticsearch/join/query/ChildQuerySearchIT.java index 34ead2c21480b..cc9a3a1a248db 100644 --- a/modules/parent-join/src/internalClusterTest/java/org/elasticsearch/join/query/ChildQuerySearchIT.java +++ b/modules/parent-join/src/internalClusterTest/java/org/elasticsearch/join/query/ChildQuerySearchIT.java @@ -806,8 +806,10 @@ public void testHasChildInnerHitsHighlighting() throws Exception { assertThat(response.getHits().getHits()[0].getId(), equalTo("1")); SearchHit[] searchHits = response.getHits().getHits()[0].getInnerHits().get("child").getHits(); assertThat(searchHits.length, equalTo(1)); - assertThat(searchHits[0].getHighlightFields().get("c_field").getFragments().length, equalTo(1)); - assertThat(searchHits[0].getHighlightFields().get("c_field").getFragments()[0].string(), equalTo("foo bar")); + HighlightField highlightField1 = searchHits[0].getHighlightFields().get("c_field"); + assertThat(highlightField1.fragments().length, equalTo(1)); + HighlightField highlightField = searchHits[0].getHighlightFields().get("c_field"); + assertThat(highlightField.fragments()[0].string(), equalTo("foo bar")); } ); } @@ -1786,7 +1788,7 @@ public void testHighlightersIgnoreParentChild() throws IOException { assertHitCount(response, 1); assertThat(response.getHits().getAt(0).getId(), equalTo("parent-id")); HighlightField highlightField = response.getHits().getAt(0).getHighlightFields().get("searchText"); - assertThat(highlightField.getFragments()[0].string(), equalTo("quick brown fox")); + assertThat(highlightField.fragments()[0].string(), equalTo("quick brown fox")); } ); @@ -1799,7 +1801,7 @@ public void testHighlightersIgnoreParentChild() throws IOException { assertHitCount(response, 1); assertThat(response.getHits().getAt(0).getId(), equalTo("child-id")); HighlightField highlightField = response.getHits().getAt(0).getHighlightFields().get("searchText"); - assertThat(highlightField.getFragments()[0].string(), equalTo("quick brown fox")); + assertThat(highlightField.fragments()[0].string(), equalTo("quick brown fox")); } ); } diff --git a/modules/parent-join/src/internalClusterTest/java/org/elasticsearch/join/query/InnerHitsIT.java b/modules/parent-join/src/internalClusterTest/java/org/elasticsearch/join/query/InnerHitsIT.java index 39a84f2d16d7f..02eaacba0b1de 100644 --- a/modules/parent-join/src/internalClusterTest/java/org/elasticsearch/join/query/InnerHitsIT.java +++ b/modules/parent-join/src/internalClusterTest/java/org/elasticsearch/join/query/InnerHitsIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder; +import org.elasticsearch.search.fetch.subphase.highlight.HighlightField; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortOrder; @@ -183,10 +184,8 @@ public void testSimpleParentChild() throws Exception { response -> { SearchHits innerHits = response.getHits().getAt(0).getInnerHits().get("comment"); assertThat(innerHits.getHits().length, equalTo(1)); - assertThat( - innerHits.getAt(0).getHighlightFields().get("message").getFragments()[0].string(), - equalTo("fox eat quick") - ); + HighlightField highlightField = innerHits.getAt(0).getHighlightFields().get("message"); + assertThat(highlightField.fragments()[0].string(), equalTo("fox eat quick")); assertThat(innerHits.getAt(0).getExplanation().toString(), containsString("weight(message:fox")); assertThat(innerHits.getAt(0).getFields().get("message").getValue().toString(), equalTo("fox eat quick")); assertThat(innerHits.getAt(0).getFields().get("script").getValue().toString(), equalTo("5")); diff --git a/plugins/mapper-annotated-text/src/main/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextHighlighter.java b/plugins/mapper-annotated-text/src/main/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextHighlighter.java index 45c2a9208b8d6..9b7c6afbb9f10 100644 --- a/plugins/mapper-annotated-text/src/main/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextHighlighter.java +++ b/plugins/mapper-annotated-text/src/main/java/org/elasticsearch/index/mapper/annotatedtext/AnnotatedTextHighlighter.java @@ -56,7 +56,7 @@ protected Analyzer wrapAnalyzer(Analyzer analyzer, Integer maxAnalyzedOffset) { } @Override - protected PassageFormatter getPassageFormatter(HitContext hitContext, SearchHighlightContext.Field field, Encoder encoder) { + protected PassageFormatter getPassageFormatter(SearchHighlightContext.Field field, Encoder encoder) { return new AnnotatedPassageFormatter(encoder); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java index ab9ab37894f70..616e036f35233 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/TopHitsIT.java @@ -630,8 +630,8 @@ public void testFetchFeatures() { SearchHit hit = hits.getAt(0); HighlightField highlightField = hit.getHighlightFields().get("text"); - assertThat(highlightField.getFragments().length, equalTo(1)); - assertThat(highlightField.getFragments()[0].string(), equalTo("some text to entertain")); + assertThat(highlightField.fragments().length, equalTo(1)); + assertThat(highlightField.fragments()[0].string(), equalTo("some text to entertain")); Explanation explanation = hit.getExplanation(); assertThat(explanation.toString(), containsString("text:text")); @@ -903,8 +903,8 @@ public void testNestedFetchFeatures() { assertThat(searchHit.getNestedIdentity().getOffset(), equalTo(0)); HighlightField highlightField = searchHit.getHighlightFields().get("comments.message"); - assertThat(highlightField.getFragments().length, equalTo(1)); - assertThat(highlightField.getFragments()[0].string(), equalTo("some comment")); + assertThat(highlightField.fragments().length, equalTo(1)); + assertThat(highlightField.fragments()[0].string(), equalTo("some comment")); // Can't explain nested hit with the main query, since both are in a different scopes, also the nested doc may not // even have matched with the main query. @@ -961,8 +961,8 @@ public void testTopHitsInNested() throws Exception { assertThat(extractValue("id", searchHits.getAt(j).getSourceAsMap()), equalTo(0)); HighlightField highlightField = searchHits.getAt(j).getHighlightFields().get("comments.message"); - assertThat(highlightField.getFragments().length, equalTo(1)); - assertThat(highlightField.getFragments()[0].string(), equalTo("some text")); + assertThat(highlightField.fragments().length, equalTo(1)); + assertThat(highlightField.fragments()[0].string(), equalTo("some text")); } } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java index 00c5342577231..dcfee8994b56b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/fetch/subphase/InnerHitsIT.java @@ -27,6 +27,7 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder; +import org.elasticsearch.search.fetch.subphase.highlight.HighlightField; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; @@ -193,10 +194,8 @@ public void testSimpleNested() throws Exception { innerHits = response.getHits().getAt(0).getInnerHits().get("comments"); assertThat(innerHits.getTotalHits().value, equalTo(2L)); assertThat(innerHits.getHits().length, equalTo(1)); - assertThat( - innerHits.getAt(0).getHighlightFields().get("comments.message").getFragments()[0].string(), - equalTo("fox eat quick") - ); + HighlightField highlightField = innerHits.getAt(0).getHighlightFields().get("comments.message"); + assertThat(highlightField.fragments()[0].string(), equalTo("fox eat quick")); assertThat(innerHits.getAt(0).getExplanation().toString(), containsString("weight(comments.message:fox in")); assertThat( innerHits.getAt(0).getFields().get("comments").getValue(), diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java index 79a28a053b3c2..6500b969ee273 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/fetch/subphase/highlight/HighlighterSearchIT.java @@ -3264,10 +3264,8 @@ public void testKeywordFieldHighlighting() throws IOException { ).get(); assertNoFailures(search); assertThat(search.getHits().getTotalHits().value, equalTo(1L)); - assertThat( - search.getHits().getAt(0).getHighlightFields().get("keyword_field").getFragments()[0].string(), - equalTo("some text") - ); + HighlightField highlightField = search.getHits().getAt(0).getHighlightFields().get("keyword_field"); + assertThat(highlightField.fragments()[0].string(), equalTo("some text")); } public void testCopyToFields() throws Exception { @@ -3294,8 +3292,8 @@ public void testCopyToFields() throws Exception { assertHitCount(response, 1); HighlightField field = response.getHits().getAt(0).getHighlightFields().get("foo_copy"); - assertThat(field.getFragments().length, equalTo(1)); - assertThat(field.getFragments()[0].string(), equalTo("how now brown cow")); + assertThat(field.fragments().length, equalTo(1)); + assertThat(field.fragments()[0].string(), equalTo("how now brown cow")); } public void testACopyFieldWithNestedQuery() throws Exception { @@ -3343,9 +3341,9 @@ public void testACopyFieldWithNestedQuery() throws Exception { .get(); assertHitCount(searchResponse, 1); HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("foo_text"); - assertThat(field.getFragments().length, equalTo(2)); - assertThat(field.getFragments()[0].string(), equalTo("brown")); - assertThat(field.getFragments()[1].string(), equalTo("cow")); + assertThat(field.fragments().length, equalTo(2)); + assertThat(field.fragments()[0].string(), equalTo("brown")); + assertThat(field.fragments()[1].string(), equalTo("cow")); } public void testFunctionScoreQueryHighlight() throws Exception { @@ -3360,8 +3358,8 @@ public void testFunctionScoreQueryHighlight() throws Exception { .get(); assertHitCount(searchResponse, 1); HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("text"); - assertThat(field.getFragments().length, equalTo(1)); - assertThat(field.getFragments()[0].string(), equalTo("brown")); + assertThat(field.fragments().length, equalTo(1)); + assertThat(field.fragments()[0].string(), equalTo("brown")); } public void testFiltersFunctionScoreQueryHighlight() throws Exception { @@ -3383,8 +3381,8 @@ public void testFiltersFunctionScoreQueryHighlight() throws Exception { ).highlighter(new HighlightBuilder().field(new Field("text"))).get(); assertHitCount(searchResponse, 1); HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("text"); - assertThat(field.getFragments().length, equalTo(1)); - assertThat(field.getFragments()[0].string(), equalTo("brown")); + assertThat(field.fragments().length, equalTo(1)); + assertThat(field.fragments()[0].string(), equalTo("brown")); } public void testHighlightQueryRewriteDatesWithNow() throws Exception { @@ -3465,33 +3463,33 @@ public void testWithNestedQuery() throws Exception { ).highlighter(new HighlightBuilder().field(new Field("foo.text").highlighterType(type))).get(); assertHitCount(searchResponse, 1); HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("foo.text"); - assertThat(field.getFragments().length, equalTo(2)); - assertThat(field.getFragments()[0].string(), equalTo("brown shoes")); - assertThat(field.getFragments()[1].string(), equalTo("cow")); + assertThat(field.fragments().length, equalTo(2)); + assertThat(field.fragments()[0].string(), equalTo("brown shoes")); + assertThat(field.fragments()[1].string(), equalTo("cow")); searchResponse = prepareSearch().setQuery(nestedQuery("foo", prefixQuery("foo.text", "bro"), ScoreMode.None)) .highlighter(new HighlightBuilder().field(new Field("foo.text").highlighterType(type))) .get(); assertHitCount(searchResponse, 1); field = searchResponse.getHits().getAt(0).getHighlightFields().get("foo.text"); - assertThat(field.getFragments().length, equalTo(1)); - assertThat(field.getFragments()[0].string(), equalTo("brown shoes")); + assertThat(field.fragments().length, equalTo(1)); + assertThat(field.fragments()[0].string(), equalTo("brown shoes")); searchResponse = prepareSearch().setQuery(nestedQuery("foo", matchPhraseQuery("foo.text", "brown shoes"), ScoreMode.None)) .highlighter(new HighlightBuilder().field(new Field("foo.text").highlighterType(type))) .get(); assertHitCount(searchResponse, 1); field = searchResponse.getHits().getAt(0).getHighlightFields().get("foo.text"); - assertThat(field.getFragments().length, equalTo(1)); - assertThat(field.getFragments()[0].string(), equalTo("brown shoes")); + assertThat(field.fragments().length, equalTo(1)); + assertThat(field.fragments()[0].string(), equalTo("brown shoes")); searchResponse = prepareSearch().setQuery(nestedQuery("foo", matchPhrasePrefixQuery("foo.text", "bro"), ScoreMode.None)) .highlighter(new HighlightBuilder().field(new Field("foo.text").highlighterType(type))) .get(); assertHitCount(searchResponse, 1); field = searchResponse.getHits().getAt(0).getHighlightFields().get("foo.text"); - assertThat(field.getFragments().length, equalTo(1)); - assertThat(field.getFragments()[0].string(), equalTo("brown shoes")); + assertThat(field.fragments().length, equalTo(1)); + assertThat(field.fragments()[0].string(), equalTo("brown shoes")); } // For unified and fvh highlighters we just check that the nested query is correctly extracted @@ -3503,8 +3501,8 @@ public void testWithNestedQuery() throws Exception { .get(); assertHitCount(searchResponse, 1); HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("text"); - assertThat(field.getFragments().length, equalTo(1)); - assertThat(field.getFragments()[0].string(), equalTo("brown")); + assertThat(field.fragments().length, equalTo(1)); + assertThat(field.fragments()[0].string(), equalTo("brown")); } } @@ -3526,8 +3524,8 @@ public void testWithNormalizer() throws Exception { .get(); assertHitCount(searchResponse, 1); HighlightField field = searchResponse.getHits().getAt(0).getHighlightFields().get("keyword"); - assertThat(field.getFragments().length, equalTo(1)); - assertThat(field.getFragments()[0].string(), equalTo("hello world")); + assertThat(field.fragments().length, equalTo(1)); + assertThat(field.fragments()[0].string(), equalTo("hello world")); } } diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 9fa99bb4a773f..6c04f6feddc96 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -1001,7 +1001,7 @@ private static Map parseHighlightFields(XContentParser p Map highlightFields = new HashMap<>(); while ((parser.nextToken()) != XContentParser.Token.END_OBJECT) { HighlightField highlightField = HighlightField.fromXContent(parser); - highlightFields.put(highlightField.getName(), highlightField); + highlightFields.put(highlightField.name(), highlightField); } return highlightFields; } diff --git a/server/src/main/java/org/elasticsearch/search/dfs/AggregatedDfs.java b/server/src/main/java/org/elasticsearch/search/dfs/AggregatedDfs.java index 1bd70b5c14817..0ce6824ec432b 100644 --- a/server/src/main/java/org/elasticsearch/search/dfs/AggregatedDfs.java +++ b/server/src/main/java/org/elasticsearch/search/dfs/AggregatedDfs.java @@ -21,9 +21,9 @@ public class AggregatedDfs implements Writeable { - private Map termStatistics; - private Map fieldStatistics; - private long maxDoc; + private final Map termStatistics; + private final Map fieldStatistics; + private final long maxDoc; public AggregatedDfs(StreamInput in) throws IOException { int size = in.readVInt(); @@ -51,10 +51,6 @@ public Map fieldStatistics() { return fieldStatistics; } - public long maxDoc() { - return maxDoc; - } - @Override public void writeTo(final StreamOutput out) throws IOException { out.writeMap(termStatistics, (o, k) -> { diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 5a04404c2e38a..91ac7356a9670 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -91,7 +91,7 @@ private static class PreloadedSourceProvider implements SourceProvider { Source source; @Override - public Source getSource(LeafReaderContext ctx, int doc) throws IOException { + public Source getSource(LeafReaderContext ctx, int doc) { return source; } } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/StoredFieldsContext.java b/server/src/main/java/org/elasticsearch/search/fetch/StoredFieldsContext.java index ae0e52ab69091..c3a91fde896bd 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/StoredFieldsContext.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/StoredFieldsContext.java @@ -115,7 +115,7 @@ public boolean equals(Object o) { StoredFieldsContext that = (StoredFieldsContext) o; if (fetchFields != that.fetchFields) return false; - return fieldNames != null ? fieldNames.equals(that.fieldNames) : that.fieldNames == null; + return Objects.equals(fieldNames, that.fieldNames); } @@ -164,7 +164,7 @@ public static StoredFieldsContext fromXContent(String fieldName, XContentParser return fromList(Collections.singletonList(parser.text())); } else if (token == XContentParser.Token.START_ARRAY) { ArrayList list = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { list.add(parser.text()); } return fromList(list); diff --git a/server/src/main/java/org/elasticsearch/search/fetch/StoredFieldsSpec.java b/server/src/main/java/org/elasticsearch/search/fetch/StoredFieldsSpec.java index 45054a90c749f..48aea98887ff0 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/StoredFieldsSpec.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/StoredFieldsSpec.java @@ -27,12 +27,12 @@ public boolean noRequirements() { /** * Use when no stored fields are required */ - public static StoredFieldsSpec NO_REQUIREMENTS = new StoredFieldsSpec(false, false, Set.of()); + public static final StoredFieldsSpec NO_REQUIREMENTS = new StoredFieldsSpec(false, false, Set.of()); /** * Use when the source should be loaded but no other stored fields are required */ - public static StoredFieldsSpec NEEDS_SOURCE = new StoredFieldsSpec(true, false, Set.of()); + public static final StoredFieldsSpec NEEDS_SOURCE = new StoredFieldsSpec(true, false, Set.of()); /** * Combine these stored field requirements with those from another StoredFieldsSpec diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/FetchSourceContext.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/FetchSourceContext.java index bba614dce78a5..4587d7560b2d9 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/FetchSourceContext.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/FetchSourceContext.java @@ -26,8 +26,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Map; -import java.util.function.Function; /** * Context used to fetch the {@code _source}. @@ -42,7 +40,6 @@ public class FetchSourceContext implements Writeable, ToXContentObject { private final boolean fetchSource; private final String[] includes; private final String[] excludes; - private Function, Map> filter; public static FetchSourceContext of(boolean fetchSource) { return fetchSource ? FETCH_SOURCE : DO_NOT_FETCH_SOURCE; @@ -153,33 +150,9 @@ public static FetchSourceContext fromXContent(XContentParser parser) throws IOEx currentFieldName = parser.currentName(); } else if (token == XContentParser.Token.START_ARRAY) { if (INCLUDES_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - List includesList = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - if (token == XContentParser.Token.VALUE_STRING) { - includesList.add(parser.text()); - } else { - throw new ParsingException( - parser.getTokenLocation(), - "Unknown key for a " + token + " in [" + currentFieldName + "].", - parser.getTokenLocation() - ); - } - } - includes = includesList.toArray(Strings.EMPTY_ARRAY); + includes = parseStringArray(parser, currentFieldName); } else if (EXCLUDES_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - List excludesList = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - if (token == XContentParser.Token.VALUE_STRING) { - excludesList.add(parser.text()); - } else { - throw new ParsingException( - parser.getTokenLocation(), - "Unknown key for a " + token + " in [" + currentFieldName + "].", - parser.getTokenLocation() - ); - } - } - excludes = excludesList.toArray(Strings.EMPTY_ARRAY); + excludes = parseStringArray(parser, currentFieldName); } else { throw new ParsingException( parser.getTokenLocation(), @@ -227,6 +200,25 @@ public static FetchSourceContext fromXContent(XContentParser parser) throws IOEx return FetchSourceContext.of(fetchSource, includes, excludes); } + private static String[] parseStringArray(XContentParser parser, String currentFieldName) throws IOException { + XContentParser.Token token; + String[] excludes; + List excludesList = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + if (token == XContentParser.Token.VALUE_STRING) { + excludesList.add(parser.text()); + } else { + throw new ParsingException( + parser.getTokenLocation(), + "Unknown key for a " + token + " in [" + currentFieldName + "].", + parser.getTokenLocation() + ); + } + } + excludes = excludesList.toArray(Strings.EMPTY_ARRAY); + return excludes; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { if (fetchSource) { diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/AbstractHighlighterBuilder.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/AbstractHighlighterBuilder.java index 3207f1ffa99f0..36cda88a063ec 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/AbstractHighlighterBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/AbstractHighlighterBuilder.java @@ -451,13 +451,6 @@ public HB boundaryScannerLocale(String boundaryScannerLocale) { return (HB) this; } - /** - * @return the value set by {@link #boundaryScannerLocale(String)} - */ - public Locale boundaryScannerLocale() { - return this.boundaryScannerLocale; - } - /** * Allows to set custom options for custom highlighters. */ diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java index 31e704fe30ff9..cae353bb91014 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/CustomQueryScorer.java @@ -8,7 +8,6 @@ package org.elasticsearch.search.fetch.subphase.highlight; -import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.Query; import org.apache.lucene.search.highlight.QueryScorer; import org.apache.lucene.search.highlight.WeightedSpanTerm; @@ -22,18 +21,6 @@ public final class CustomQueryScorer extends QueryScorer { - public CustomQueryScorer(Query query, IndexReader reader, String field, String defaultField) { - super(query, reader, field, defaultField); - } - - public CustomQueryScorer(Query query, IndexReader reader, String field) { - super(query, reader, field); - } - - public CustomQueryScorer(Query query, String field, String defaultField) { - super(query, field, defaultField); - } - public CustomQueryScorer(Query query, String field) { super(query, field); } @@ -42,10 +29,6 @@ public CustomQueryScorer(Query query) { super(query); } - public CustomQueryScorer(WeightedSpanTerm[] weightedTerms) { - super(weightedTerms); - } - @Override protected WeightedSpanTermExtractor newTermExtractor(String defaultField) { return defaultField == null ? new CustomWeightedSpanTermExtractor() : new CustomWeightedSpanTermExtractor(defaultField); @@ -69,7 +52,6 @@ protected void extractUnknownQuery(Query query, Map te protected void extract(Query query, float boost, Map terms) throws IOException { if (isChildOrParentQuery(query.getClass())) { // skip has_child or has_parent queries, see: https://github.com/elastic/elasticsearch/issues/14999 - return; } else if (query instanceof FunctionScoreQuery) { super.extract(((FunctionScoreQuery) query).getSubQuery(), boost, terms); } else if (query instanceof ESToParentBlockJoinQuery) { diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/DefaultHighlighter.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/DefaultHighlighter.java index d90aba24a94df..e77436ba61423 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/DefaultHighlighter.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/DefaultHighlighter.java @@ -32,7 +32,6 @@ import org.elasticsearch.lucene.search.uhighlight.Snippet; import org.elasticsearch.search.fetch.FetchContext; import org.elasticsearch.search.fetch.FetchSubPhase; -import org.elasticsearch.search.fetch.FetchSubPhase.HitContext; import java.io.IOException; import java.text.BreakIterator; @@ -120,7 +119,7 @@ CustomUnifiedHighlighter buildHighlighter(FieldHighlightContext fieldContext) { fieldContext.context.getSearchExecutionContext().getIndexAnalyzer(f -> Lucene.KEYWORD_ANALYZER), queryMaxAnalyzedOffset ); - PassageFormatter passageFormatter = getPassageFormatter(fieldContext.hitContext, fieldContext.field, encoder); + PassageFormatter passageFormatter = getPassageFormatter(fieldContext.field, encoder); IndexSearcher searcher = fieldContext.context.searcher(); OffsetSource offsetSource = getOffsetSource(fieldContext.context, fieldContext.fieldType); BreakIterator breakIterator; @@ -161,7 +160,7 @@ CustomUnifiedHighlighter buildHighlighter(FieldHighlightContext fieldContext) { ); } - protected PassageFormatter getPassageFormatter(HitContext hitContext, SearchHighlightContext.Field field, Encoder encoder) { + protected PassageFormatter getPassageFormatter(SearchHighlightContext.Field field, Encoder encoder) { return new CustomPassageFormatter(field.fieldOptions().preTags()[0], field.fieldOptions().postTags()[0], encoder); } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FastVectorHighlighter.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FastVectorHighlighter.java index 75a1777ae7d8f..8417c9d747981 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FastVectorHighlighter.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FastVectorHighlighter.java @@ -312,6 +312,6 @@ private static class FieldHighlightEntry { private static class HighlighterEntry { public org.apache.lucene.search.vectorhighlight.FastVectorHighlighter fvh; - public Map fields = new HashMap<>(); + public final Map fields = new HashMap<>(); } } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FragmentBuilderHelper.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FragmentBuilderHelper.java index 9c761936863d6..5421cd59a23e4 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FragmentBuilderHelper.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/FragmentBuilderHelper.java @@ -8,15 +8,11 @@ package org.elasticsearch.search.fetch.subphase.highlight; -import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.search.vectorhighlight.FastVectorHighlighter; import org.apache.lucene.search.vectorhighlight.FieldFragList.WeightedFragInfo; import org.apache.lucene.search.vectorhighlight.FieldFragList.WeightedFragInfo.SubInfo; import org.apache.lucene.search.vectorhighlight.FragmentsBuilder; import org.apache.lucene.util.CollectionUtil; -import org.elasticsearch.index.analysis.AnalyzerComponentsProvider; -import org.elasticsearch.index.analysis.NamedAnalyzer; -import org.elasticsearch.index.analysis.TokenFilterFactory; import java.util.List; @@ -45,7 +41,7 @@ public static WeightedFragInfo fixWeightedFragInfo(WeightedFragInfo fragInfo) { CollectionUtil.introSort(subInfos, (o1, o2) -> { int startOffset = o1.getTermsOffsets().get(0).getStartOffset(); int startOffset2 = o2.getTermsOffsets().get(0).getStartOffset(); - return compare(startOffset, startOffset2); + return Integer.compare(startOffset, startOffset2); }); return new WeightedFragInfo( Math.min(fragInfo.getSubInfos().get(0).getTermsOffsets().get(0).getStartOffset(), fragInfo.getStartOffset()), @@ -58,23 +54,4 @@ public static WeightedFragInfo fixWeightedFragInfo(WeightedFragInfo fragInfo) { } } - private static int compare(int x, int y) { - return (x < y) ? -1 : ((x == y) ? 0 : 1); - } - - private static boolean containsBrokenAnalysis(Analyzer analyzer) { - // TODO maybe we need a getter on Namedanalyzer that tells if this uses broken Analysis - if (analyzer instanceof NamedAnalyzer) { - analyzer = ((NamedAnalyzer) analyzer).analyzer(); - } - if (analyzer instanceof AnalyzerComponentsProvider) { - final TokenFilterFactory[] tokenFilters = ((AnalyzerComponentsProvider) analyzer).getComponents().getTokenFilters(); - for (TokenFilterFactory tokenFilterFactory : tokenFilters) { - if (tokenFilterFactory.breaksFastVectorHighlighter()) { - return true; - } - } - } - return false; - } } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilder.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilder.java index 7d371ac372774..0042b1eafba71 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilder.java @@ -45,8 +45,6 @@ * @see org.elasticsearch.search.builder.SearchSourceBuilder#highlight() */ public final class HighlightBuilder extends AbstractHighlighterBuilder { - /** default for whether to highlight fields based on the source even if stored separately */ - public static final boolean DEFAULT_FORCE_SOURCE = false; /** default for whether a field should be highlighted only if a query matches that field */ public static final boolean DEFAULT_REQUIRE_FIELD_MATCH = true; /** default for whether to stop highlighting at the defined max_analyzed_offset to avoid exceptions for longer texts */ @@ -149,17 +147,6 @@ public HighlightBuilder field(String name) { return field(new Field(name)); } - /** - * Adds a field to be highlighted with a provided fragment size (in characters), and - * default number of fragments of 5. - * - * @param name The field to highlight - * @param fragmentSize The size of a fragment in characters - */ - public HighlightBuilder field(String name, int fragmentSize) { - return field(new Field(name).fragmentSize(fragmentSize)); - } - /** * Adds a field to be highlighted with a provided fragment size (in characters), and * a provided (maximum) number of fragments. diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightField.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightField.java index d4b5234f4e0b2..6bc9f65ac655f 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightField.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightField.java @@ -30,23 +30,12 @@ */ public class HighlightField implements ToXContentFragment, Writeable { - private String name; + private final String name; - private Text[] fragments; + private final Text[] fragments; public HighlightField(StreamInput in) throws IOException { - name = in.readString(); - if (in.readBoolean()) { - int size = in.readVInt(); - if (size == 0) { - fragments = Text.EMPTY_ARRAY; - } else { - fragments = new Text[size]; - for (int i = 0; i < size; i++) { - fragments[i] = in.readText(); - } - } - } + this(in.readString(), in.readOptionalArray(StreamInput::readText, Text[]::new)); } public HighlightField(String name, Text[] fragments) { @@ -61,13 +50,6 @@ public String name() { return name; } - /** - * The name of the field highlighted. - */ - public String getName() { - return name(); - } - /** * The highlighted fragments. {@code null} if failed to highlight (for example, the field is not stored). */ @@ -75,13 +57,6 @@ public Text[] fragments() { return fragments; } - /** - * The highlighted fragments. {@code null} if failed to highlight (for example, the field is not stored). - */ - public Text[] getFragments() { - return fragments(); - } - @Override public String toString() { return "[" + name + "], fragments[" + Arrays.toString(fragments) + "]"; @@ -101,14 +76,14 @@ public void writeTo(StreamOutput out) throws IOException { public static HighlightField fromXContent(XContentParser parser) throws IOException { ensureExpectedToken(XContentParser.Token.FIELD_NAME, parser.currentToken(), parser); String fieldName = parser.currentName(); - Text[] fragments = null; + Text[] fragments; XContentParser.Token token = parser.nextToken(); if (token == XContentParser.Token.START_ARRAY) { List values = new ArrayList<>(); while (parser.nextToken() != XContentParser.Token.END_ARRAY) { values.add(new Text(parser.text())); } - fragments = values.toArray(new Text[values.size()]); + fragments = values.toArray(Text.EMPTY_ARRAY); } else if (token == XContentParser.Token.VALUE_NULL) { fragments = null; } else { diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/SourceScoreOrderFragmentsBuilder.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/SourceScoreOrderFragmentsBuilder.java index f1bb3f2c773ac..79c7198564be5 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/SourceScoreOrderFragmentsBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/SourceScoreOrderFragmentsBuilder.java @@ -8,7 +8,6 @@ package org.elasticsearch.search.fetch.subphase.highlight; import org.apache.lucene.document.Field; -import org.apache.lucene.document.TextField; import org.apache.lucene.index.IndexReader; import org.apache.lucene.search.highlight.Encoder; import org.apache.lucene.search.vectorhighlight.BoundaryScanner; @@ -20,8 +19,6 @@ import org.elasticsearch.search.lookup.Source; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; public class SourceScoreOrderFragmentsBuilder extends ScoreOrderFragmentsBuilder { @@ -51,19 +48,7 @@ public SourceScoreOrderFragmentsBuilder( @Override protected Field[] getFields(IndexReader reader, int docId, String fieldName) throws IOException { // we know its low level reader, and matching docId, since that's how we call the highlighter with - List values = valueFetcher.fetchValues(source, docId, new ArrayList<>()); - if (values.size() > 1 && fetchContext.sourceLoader().reordersFieldValues()) { - throw new IllegalArgumentException( - "The fast vector highlighter doesn't support loading multi-valued fields from _source in index [" - + fetchContext.getIndexName() - + "] because _source can reorder field values" - ); - } - Field[] fields = new Field[values.size()]; - for (int i = 0; i < values.size(); i++) { - fields[i] = new Field(fieldType.name(), values.get(i).toString(), TextField.TYPE_NOT_STORED); - } - return fields; + return SourceSimpleFragmentsBuilder.doGetFields(docId, valueFetcher, source, fetchContext, fieldType); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/SourceSimpleFragmentsBuilder.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/SourceSimpleFragmentsBuilder.java index 0a7a5d300339b..c6b69717b8f75 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/SourceSimpleFragmentsBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/highlight/SourceSimpleFragmentsBuilder.java @@ -46,6 +46,11 @@ public SourceSimpleFragmentsBuilder( @Override protected Field[] getFields(IndexReader reader, int docId, String fieldName) throws IOException { // we know its low level reader, and matching docId, since that's how we call the highlighter with + return doGetFields(docId, valueFetcher, source, fetchContext, fieldType); + } + + static Field[] doGetFields(int docId, ValueFetcher valueFetcher, Source source, FetchContext fetchContext, MappedFieldType fieldType) + throws IOException { List values = valueFetcher.fetchValues(source, docId, new ArrayList<>()); if (values.isEmpty()) { return EMPTY_FIELDS; @@ -63,5 +68,4 @@ protected Field[] getFields(IndexReader reader, int docId, String fieldName) thr } return fields; } - } diff --git a/server/src/test/java/org/elasticsearch/search/SearchHitTests.java b/server/src/test/java/org/elasticsearch/search/SearchHitTests.java index 7bade799bc8ea..d08abe5065984 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchHitTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchHitTests.java @@ -102,7 +102,7 @@ public static SearchHit createTestItem(XContentType xContentType, boolean withOp Map highlightFields = Maps.newMapWithExpectedSize(size); for (int i = 0; i < size; i++) { HighlightField testItem = HighlightFieldTests.createTestItem(); - highlightFields.put(testItem.getName(), testItem); + highlightFields.put(testItem.name(), testItem); } hit.highlightFields(highlightFields); } diff --git a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightFieldTests.java b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightFieldTests.java index f174ae9180522..2d081fe2e3049 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightFieldTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightFieldTests.java @@ -113,9 +113,9 @@ public void testSerialization() throws IOException { } private static HighlightField mutate(HighlightField original) { - Text[] fragments = original.getFragments(); + Text[] fragments = original.fragments(); if (randomBoolean()) { - return new HighlightField(original.getName() + "_suffix", fragments); + return new HighlightField(original.name() + "_suffix", fragments); } else { if (fragments == null) { fragments = new Text[] { new Text("field") }; @@ -123,12 +123,12 @@ private static HighlightField mutate(HighlightField original) { fragments = Arrays.copyOf(fragments, fragments.length + 1); fragments[fragments.length - 1] = new Text("something new"); } - return new HighlightField(original.getName(), fragments); + return new HighlightField(original.name(), fragments); } } private static HighlightField copy(HighlightField original) { - return new HighlightField(original.getName(), original.getFragments()); + return new HighlightField(original.name(), original.fragments()); } } diff --git a/test/framework/src/main/java/org/elasticsearch/search/fetch/HighlighterTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/fetch/HighlighterTestCase.java index 8cbf233616c53..a9f1ab7780f7f 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/fetch/HighlighterTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/fetch/HighlighterTestCase.java @@ -101,7 +101,8 @@ private static Map> storedFields(StoredFieldsSpec spec, Par */ protected static void assertHighlights(Map highlights, String field, String... fragments) { assertNotNull("No highlights reported for field [" + field + "]", highlights.get(field)); - List actualFragments = Arrays.stream(highlights.get(field).getFragments()).map(Text::toString).collect(Collectors.toList()); + HighlightField highlightField = highlights.get(field); + List actualFragments = Arrays.stream(highlightField.fragments()).map(Text::toString).collect(Collectors.toList()); List expectedFragments = List.of(fragments); assertEquals(expectedFragments, actualFragments); } From 5f958a0a6ea72d84f3cbce207d3f048547d26b5e Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 9 Nov 2023 13:48:18 +0100 Subject: [PATCH 024/513] Remove unused SnapshotCreationException (#101940) Random find, resolving the TODO I added way back when. --- .../elasticsearch/ElasticsearchException.java | 7 +---- .../snapshots/SnapshotCreationException.java | 26 ------------------- .../ExceptionSerializationTests.java | 2 +- 3 files changed, 2 insertions(+), 33 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/snapshots/SnapshotCreationException.java diff --git a/server/src/main/java/org/elasticsearch/ElasticsearchException.java b/server/src/main/java/org/elasticsearch/ElasticsearchException.java index 4bbfe994f7f6d..5c5133e478ee1 100644 --- a/server/src/main/java/org/elasticsearch/ElasticsearchException.java +++ b/server/src/main/java/org/elasticsearch/ElasticsearchException.java @@ -1131,12 +1131,7 @@ private enum ElasticsearchExceptionHandle { UNKNOWN_VERSION_ADDED ), // 26 was BatchOperationException - SNAPSHOT_CREATION_EXCEPTION( - org.elasticsearch.snapshots.SnapshotCreationException.class, - org.elasticsearch.snapshots.SnapshotCreationException::new, - 27, - UNKNOWN_VERSION_ADDED - ), + // 27 was SnapshotCreationException // 28 was DeleteFailedEngineException, deprecated in 6.0, removed in 7.0 DOCUMENT_MISSING_EXCEPTION( org.elasticsearch.index.engine.DocumentMissingException.class, diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotCreationException.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotCreationException.java deleted file mode 100644 index 7c2dde414c8ad..0000000000000 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotCreationException.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0 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 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.snapshots; - -import org.elasticsearch.common.io.stream.StreamInput; - -import java.io.IOException; - -/** - * Thrown when snapshot creation fails completely - * TODO: Remove this class in 8.0 - * @deprecated This exception isn't thrown anymore. It's only here for BwC. - */ -@Deprecated -public class SnapshotCreationException extends SnapshotException { - - public SnapshotCreationException(StreamInput in) throws IOException { - super(in); - } -} diff --git a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index 405211423f602..abf79243b6a61 100644 --- a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -693,7 +693,7 @@ public void testIds() { ids.put(24, org.elasticsearch.search.SearchContextMissingException.class); ids.put(25, org.elasticsearch.script.GeneralScriptException.class); ids.put(26, null); - ids.put(27, org.elasticsearch.snapshots.SnapshotCreationException.class); + ids.put(27, null); // was SnapshotCreationException ids.put(28, null); // was DeleteFailedEngineException, deprecated in 6.0 and removed in 7.0 ids.put(29, org.elasticsearch.index.engine.DocumentMissingException.class); ids.put(30, org.elasticsearch.snapshots.SnapshotException.class); From 30af5b16cf2006835543d8ad0f1e6dc05918ad93 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 9 Nov 2023 13:04:36 +0000 Subject: [PATCH 025/513] Fix up git queries in TransportVersions.java (#101900) --- .../main/java/org/elasticsearch/TransportVersions.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 0c7145730e447..b62a24fdc0b45 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -202,15 +202,17 @@ static TransportVersion def(int id) { * If your git checkout has the expected minor-version-numbered branches and the expected release-version tags then you can find the * transport versions known by a particular release ... * - * git show v8.9.1:server/src/main/java/org/elasticsearch/TransportVersions.java | grep def + * git show v8.11.0:server/src/main/java/org/elasticsearch/TransportVersions.java | grep '= def' * * ... or by a particular branch ... * - * git show 8.10:server/src/main/java/org/elasticsearch/TransportVersions.java | grep def + * git show 8.11:server/src/main/java/org/elasticsearch/TransportVersions.java | grep '= def' * * ... and you can see which versions were added in between two versions too ... * - * git diff 8.10..main -- server/src/main/java/org/elasticsearch/TransportVersions.java + * git diff v8.11.0..main -- server/src/main/java/org/elasticsearch/TransportVersions.java + * + * In branches 8.7-8.10 see server/src/main/java/org/elasticsearch/TransportVersion.java for the equivalent definitions. */ /** From abbd2db88f1a45885c62fd913229ed8184f5d308 Mon Sep 17 00:00:00 2001 From: David Kyle Date: Thu, 9 Nov 2023 13:34:12 +0000 Subject: [PATCH 026/513] [ML] Skip multi-deployment test in single processor tests (#101957) MlAssignmentPlannerUpgradeIT creates 2 model deployments, the second will fail in single processor tests due to insufficient CPUs to run the deployment. Skip the test when running on a single CPU. --- .../upgrades/MlAssignmentPlannerUpgradeIT.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java index 48d7bd918255d..22ff69bf5ff55 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java @@ -30,6 +30,10 @@ public class MlAssignmentPlannerUpgradeIT extends AbstractUpgradeTestCase { + private static final boolean IS_SINGLE_PROCESSOR_TEST = Boolean.parseBoolean( + System.getProperty("tests.configure_test_clusters_with_one_processor", "false") + ); + private Logger logger = LogManager.getLogger(MlAssignmentPlannerUpgradeIT.class); // See PyTorchModelIT for how this model was created @@ -61,9 +65,9 @@ public class MlAssignmentPlannerUpgradeIT extends AbstractUpgradeTestCase { RAW_MODEL_SIZE = Base64.getDecoder().decode(BASE_64_ENCODED_MODEL).length; } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/101926") public void testMlAssignmentPlannerUpgrade() throws Exception { assumeTrue("NLP model deployments added in 8.0", isOriginalClusterVersionAtLeast(Version.V_8_0_0)); + assumeFalse("This test deploys multiple models which cannot be accommodated on a single processor", IS_SINGLE_PROCESSOR_TEST); logger.info("Starting testMlAssignmentPlannerUpgrade, model size {}", RAW_MODEL_SIZE); From aae4970a4ccb05404adde99c5d031446fa06353a Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 9 Nov 2023 13:36:31 +0000 Subject: [PATCH 027/513] AwaitsFix for #100957 --- .../elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java index 31baba05c3b09..6dc940d191685 100644 --- a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java +++ b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java @@ -1238,6 +1238,7 @@ public void testRemoteClusterOnlyCCSWithFailuresOnAllShards() throws Exception { } } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/100957") public void testCancelViaTasksAPI() throws Exception { Map testClusterInfo = setupTwoClusters(); String localIndex = (String) testClusterInfo.get("local.index"); From 04f26a052b51fea34c1bf28da0a7d545a0de1298 Mon Sep 17 00:00:00 2001 From: Michael Peterson Date: Thu, 9 Nov 2023 09:07:28 -0500 Subject: [PATCH 028/513] Add logging of errors in _msearch and msearch template (#101576) --- .../mustache/TransportMultiSearchTemplateAction.java | 8 ++++++++ .../action/search/TransportMultiSearchAction.java | 7 +++++++ 2 files changed, 15 insertions(+) diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportMultiSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportMultiSearchTemplateAction.java index d859fb509e915..4b0c365ba8b13 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportMultiSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportMultiSearchTemplateAction.java @@ -8,6 +8,9 @@ package org.elasticsearch.script.mustache; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchResponse; @@ -31,6 +34,8 @@ public class TransportMultiSearchTemplateAction extends HandledTransportAction { + private static final Logger logger = LogManager.getLogger(TransportMultiSearchTemplateAction.class); + private final ScriptService scriptService; private final NamedXContentRegistry xContentRegistry; private final NodeClient client; @@ -76,6 +81,9 @@ protected void doExecute(Task task, MultiSearchTemplateRequest request, ActionLi searchRequest = convert(searchTemplateRequest, searchTemplateResponse, scriptService, xContentRegistry, searchUsageHolder); } catch (Exception e) { items[i] = new MultiSearchTemplateResponse.Item(null, e); + if (ExceptionsHelper.status(e).getStatus() >= 500 && ExceptionsHelper.isNodeOrShardUnavailableTypeException(e) == false) { + logger.warn("MultiSearchTemplate convert failure", e); + } continue; } items[i] = new MultiSearchTemplateResponse.Item(searchTemplateResponse, null); diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportMultiSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportMultiSearchAction.java index a2324010876bf..f51c700c8c8c9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportMultiSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportMultiSearchAction.java @@ -8,6 +8,9 @@ package org.elasticsearch.action.search; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; @@ -31,6 +34,7 @@ public class TransportMultiSearchAction extends HandledTransportAction { + private static final Logger logger = LogManager.getLogger(TransportMultiSearchAction.class); private final int allocatedProcessors; private final ThreadPool threadPool; private final ClusterService clusterService; @@ -155,6 +159,9 @@ public void onResponse(final SearchResponse searchResponse) { @Override public void onFailure(final Exception e) { + if (ExceptionsHelper.status(e).getStatus() >= 500 && ExceptionsHelper.isNodeOrShardUnavailableTypeException(e) == false) { + logger.warn("TransportMultiSearchAction failure", e); + } handleResponse(request.responseSlot, new MultiSearchResponse.Item(null, e)); } From 58b27ccfb4f942d0d63eebb8e4271146d1835fa2 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Thu, 9 Nov 2023 15:44:36 +0100 Subject: [PATCH 029/513] [Profiling] Load integration test data from file (#101947) With this commit we remove the hardcoded document construction from profiling integration tests and instead load test data from ndjson files. --- .../xpack/profiling/CancellationIT.java | 6 - .../profiling/GetFlameGraphActionIT.java | 15 +- .../profiling/GetStackTracesActionIT.java | 29 +- .../xpack/profiling/ProfilingTestCase.java | 51 +- .../data/profiling-events-all.ndjson | 88 + .../data/profiling-executables.ndjson | 50 + .../data/profiling-stackframes.ndjson | 1414 +++++++++++++++++ .../data/profiling-stacktraces.ndjson | 88 + 8 files changed, 1679 insertions(+), 62 deletions(-) create mode 100644 x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-events-all.ndjson create mode 100644 x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-executables.ndjson create mode 100644 x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-stackframes.ndjson create mode 100644 x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-stacktraces.ndjson diff --git a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/CancellationIT.java b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/CancellationIT.java index 6c4184f77a28b..b4b8242a6e456 100644 --- a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/CancellationIT.java +++ b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/CancellationIT.java @@ -56,12 +56,6 @@ protected Collection> nodePlugins() { return plugins; } - @Override - protected boolean useOnlyAllEvents() { - // we assume that all indices have been created to simplify the testing logic. - return false; - } - public void testAutomaticCancellation() throws Exception { Request restRequest = new Request("POST", "/_profiling/stacktraces"); restRequest.setEntity(new StringEntity(""" diff --git a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/GetFlameGraphActionIT.java b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/GetFlameGraphActionIT.java index 308e377c2826a..7d7bae20ef983 100644 --- a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/GetFlameGraphActionIT.java +++ b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/GetFlameGraphActionIT.java @@ -8,19 +8,14 @@ package org.elasticsearch.xpack.profiling; public class GetFlameGraphActionIT extends ProfilingTestCase { - @Override - protected boolean useOnlyAllEvents() { - return true; - } - public void testGetStackTracesUnfiltered() throws Exception { - GetStackTracesRequest request = new GetStackTracesRequest(1, null); + GetStackTracesRequest request = new GetStackTracesRequest(10, null); GetFlamegraphResponse response = client().execute(GetFlamegraphAction.INSTANCE, request).get(); // only spot-check top level properties - detailed tests are done in unit tests - assertEquals(4, response.getSize()); + assertEquals(231, response.getSize()); assertEquals(1.0d, response.getSamplingRate(), 0.001d); - assertEquals(3, response.getSelfCPU()); - assertEquals(4, response.getTotalCPU()); - assertEquals(1, response.getTotalSamples()); + assertEquals(60, response.getSelfCPU()); + assertEquals(1204, response.getTotalCPU()); + assertEquals(40, response.getTotalSamples()); } } diff --git a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/GetStackTracesActionIT.java b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/GetStackTracesActionIT.java index c8ee6d91a1e47..050ff5dd47800 100644 --- a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/GetStackTracesActionIT.java +++ b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/GetStackTracesActionIT.java @@ -11,23 +11,28 @@ public class GetStackTracesActionIT extends ProfilingTestCase { public void testGetStackTracesUnfiltered() throws Exception { - GetStackTracesRequest request = new GetStackTracesRequest(1, null); + GetStackTracesRequest request = new GetStackTracesRequest(10, null); + request.setAdjustSampleCount(true); GetStackTracesResponse response = client().execute(GetStackTracesAction.INSTANCE, request).get(); - assertEquals(1, response.getTotalFrames()); + assertEquals(40, response.getTotalSamples()); + assertEquals(285, response.getTotalFrames()); + + assertNotNull(response.getStackTraceEvents()); + assertEquals(4, (int) response.getStackTraceEvents().get("14cFLjgoe-BTQd17mhedeA")); + assertNotNull(response.getStackTraces()); - StackTrace stackTrace = response.getStackTraces().get("QjoLteG7HX3VUUXr-J4kHQ"); - assertEquals(List.of(1083999), stackTrace.addressOrLines); - assertEquals(List.of("QCCDqjSg3bMK1C4YRK6Tiw"), stackTrace.fileIds); - assertEquals(List.of("QCCDqjSg3bMK1C4YRK6TiwAAAAAAEIpf"), stackTrace.frameIds); - assertEquals(List.of(2), stackTrace.typeIds); + // just do a high-level spot check. Decoding is tested in unit-tests + StackTrace stackTrace = response.getStackTraces().get("JvISdnJ47BQ01489cwF9DA"); + assertEquals(4, stackTrace.addressOrLines.size()); + assertEquals(4, stackTrace.fileIds.size()); + assertEquals(4, stackTrace.frameIds.size()); + assertEquals(4, stackTrace.typeIds.size()); assertNotNull(response.getStackFrames()); - StackFrame stackFrame = response.getStackFrames().get("QCCDqjSg3bMK1C4YRK6TiwAAAAAAEIpf"); - assertEquals(List.of("_raw_spin_unlock_irqrestore", "inlined_frame_1", "inlined_frame_0"), stackFrame.functionName); - assertNotNull(response.getStackTraceEvents()); - assertEquals(1, (int) response.getStackTraceEvents().get("QjoLteG7HX3VUUXr-J4kHQ")); + StackFrame stackFrame = response.getStackFrames().get("lHp5_WAgpLy2alrUVab6HAAAAAAATgeq"); + assertEquals(List.of("blkdev_issue_flush"), stackFrame.functionName); assertNotNull(response.getExecutables()); - assertNotNull("libc.so.6", response.getExecutables().get("QCCDqjSg3bMK1C4YRK6Tiw")); + assertNotNull("vmlinux", response.getExecutables().get("lHp5_WAgpLy2alrUVab6HA")); } } diff --git a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/ProfilingTestCase.java b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/ProfilingTestCase.java index 29981c8e2f2a3..0f602d9d99c25 100644 --- a/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/ProfilingTestCase.java +++ b/x-pack/plugin/profiling/src/internalClusterTest/java/org/elasticsearch/xpack/profiling/ProfilingTestCase.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; @@ -19,6 +20,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.transport.netty4.Netty4Plugin; +import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.ilm.LifecycleSettings; import org.elasticsearch.xpack.ilm.IndexLifecycle; @@ -27,7 +29,7 @@ import org.junit.After; import org.junit.Before; -import java.time.Instant; +import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Map; @@ -71,18 +73,6 @@ private void indexDoc(String index, String id, Map source) { assertEquals(RestStatus.CREATED, indexResponse.status()); } - /** - * Only the index "profiling-events-all" is always present. All other indices (e.g. "profiling-events-5pow02") are created on demand - * at a later point when there are enough samples. With this flag we simulate that data should be retrieved briefly after cluster - * start when only profiling-events-all is present. We expect that also in this case, available data is returned but we rely only - * on the single existing index. - * - * @return true iff this test should rely on only "profiling-events-all" being present. - */ - protected boolean useOnlyAllEvents() { - return randomBoolean(); - } - /** * @return true iff this test relies that data (and the corresponding indices / data streams) are present for this test. */ @@ -112,6 +102,16 @@ protected void updateProfilingTemplatesEnabled(boolean newValue) { assertTrue("Update of profiling templates enabled setting is not acknowledged", response.isAcknowledged()); } + protected final byte[] read(String resource) throws IOException { + return ProfilingTestCase.class.getClassLoader().getResourceAsStream(resource).readAllBytes(); + } + + protected final void bulkIndex(String file) throws Exception { + byte[] bulkData = read(file); + BulkResponse response = client().prepareBulk().add(bulkData, 0, bulkData.length, XContentType.JSON).execute().actionGet(); + assertFalse(response.hasFailures()); + } + @Before public void setupData() throws Exception { if (requiresDataSetup() == false) { @@ -119,30 +119,13 @@ public void setupData() throws Exception { } // only enable index management while setting up indices to avoid interfering with the rest of the test infrastructure updateProfilingTemplatesEnabled(true); - Collection eventsIndices = useOnlyAllEvents() ? List.of(EventsIndex.FULL_INDEX.getName()) : EventsIndex.indexNames(); waitForIndices(); ensureGreen(); - // ensure that we have this in every index, so we find an event - for (String idx : eventsIndices) { - indexDoc( - idx, - "QjoLteG7HX3VUUXr-J4kHQ", - Map.of("@timestamp", Instant.now().toEpochMilli(), "Stacktrace.id", "QjoLteG7HX3VUUXr-J4kHQ", "Stacktrace.count", 1) - ); - } - - indexDoc( - "profiling-stacktraces", - "QjoLteG7HX3VUUXr-J4kHQ", - Map.of("Stacktrace.frame.ids", "QCCDqjSg3bMK1C4YRK6TiwAAAAAAEIpf", "Stacktrace.frame.types", "AQI") - ); - indexDoc( - "profiling-stackframes", - "QCCDqjSg3bMK1C4YRK6TiwAAAAAAEIpf", - Map.of("Stackframe.function.name", List.of("_raw_spin_unlock_irqrestore", "inlined_frame_1", "inlined_frame_0")) - ); - indexDoc("profiling-executables", "QCCDqjSg3bMK1C4YRK6Tiw", Map.of("Executable.file.name", "libc.so.6")); + bulkIndex("data/profiling-events-all.ndjson"); + bulkIndex("data/profiling-stacktraces.ndjson"); + bulkIndex("data/profiling-stackframes.ndjson"); + bulkIndex("data/profiling-executables.ndjson"); refresh(); } diff --git a/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-events-all.ndjson b/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-events-all.ndjson new file mode 100644 index 0000000000000..071b0a2edbe1a --- /dev/null +++ b/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-events-all.ndjson @@ -0,0 +1,88 @@ +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["S07KmaoGhvNte78xwwRbZQ"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["4tB_mGJrj1xVuMFbXVYwGA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["VxcDhJCIVlMC063NXB1S1Q"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["_eitfgJr5SICw3IID0ieUw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["AIcHF1g0UqebNhdXerG6rQ"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["ZZ003BQ7N2U_lm0xfG38Bw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["SsxQM9_HdP1iq1X4sNsK_Q"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["TEeScu0pOzeesTxGaw4JAg"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["Cf-uAEYevmDh96dG8LiA7w"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["L3nUGoNiPS6jpV4lzI0HKw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["8Nt8q4FpMS1YBSbb8JjA4Q"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["JvISdnJ47BQ01489cwF9DA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["105167004320218"], "container.name": ["instance-0000000010"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["p0FpNwvinkonmYTrP_m8AQ"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["111571015508996"], "container.name": ["instance-0000000010"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["Ce77w10WeIDow3kd1jowlA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["lDY7ho5Q3VRiBNIA0utPkg"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["Hwc9ojSnNRegxwpOeXsNnw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["fL6FlEuXKQQ1vbZSuOkNDA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["599103450330106"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["14cFLjgoe-BTQd17mhedeA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["599103450330106"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["hm6pJJBFG1EoukWkIgUkSg"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["evah_aPn6ywh3ieU3e4MDQ"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["kU23IHsR5pRIb6YvFIRS3Q"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["CT7Jn1_cfRttYWpYz0WvHA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["p_qI8Ts1o4R4GHiu2MHhmg"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["828nO5IsKeisTjrd1qZGjA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["Qn60Qbgyi9gMjkEiQhHG0w"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["ZnXwrIATmk5PmXMsv4nKtg"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["231306382266776"], "container.name": ["instance-0000000010"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["YVZcVDoPQkcegaQYBCZkzw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["542231923413871"], "container.name": ["instance-0000000010"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["uZJLF8SFtg9OqPXwQZVelg"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["NPliZYuvPv2y5wnRJfOutw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["5RX98LQ-TugfdPLcy0tTFA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["ehqmjj-2Kv8A1vCgGKyP2Q"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["V-R5JWu8vu2g_Vodg6Kd6w"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["LpsjtOqp4gQNWANoCtMhpw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["P7SH4zhzkMyu8ClDXegYrw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["507517422226861"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["bA1f839p2HCzC7fcTWp8Kw"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["41422267885458"], "container.name": ["instance-0000000010"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["XF9MchOwpePfa6_hYy-vZQ"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["L7kj7UvlKbT-vN73el4faQ"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["hRqQI2CBPiapzgFG9jrmDA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["599103450330106"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["FWCWmB5KaAfOjnip-rDwUQ"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["-lrLeM0iQSsI3WnVH08B3w"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["7Se-2CaL4CbSRMgSyNxl5w"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "container.name": ["instance-0000000010"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["iCDmSM5ZMerasUWJeO1RoA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["497295213074376"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["qRkm6b_RZPI0741rrxfzIA"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["239239240528656"], "container.name": ["instance-0000000010"]} +{"create": {"_index": "profiling-events-all"}} +{"Stacktrace.count": [1], "profiling.project.id": ["100"], "os.kernel": ["9.9.9-0"], "tags": ["environment:qa", "region:eu-west-1"], "host.ip": ["192.168.1.2"], "@timestamp": ["1698624000"], "ecs.version": ["1.12.0"], "Stacktrace.id": ["P4G4FrOCtB7UkFb-8UfLLQ"], "agent.version": ["head-be593ef3-1688111067"], "host.name": ["ip-192-168-1-2"], "host.id": ["8457605156473051743"], "process.thread.name": ["336544854664378"], "container.name": ["instance-0000000010"]} diff --git a/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-executables.ndjson b/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-executables.ndjson new file mode 100644 index 0000000000000..388588482859a --- /dev/null +++ b/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-executables.ndjson @@ -0,0 +1,50 @@ +{"create": {"_index": "profiling-executables", "_id": "4rFFPORY8_4FvK9QaA3QUQ"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": "63e115dedee42e3aee75e0d5fa41243ac2cf9228"}, "file": {"name": "python2.7"}}, "Symbolization": {"next_time": "4851753124"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "-FiXU0afkg-sRwubj2WiAw"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": "a607524ba9b6f400a9af233fa574bc7f2abd5f33"}, "file": {"name": "dbus-daemon"}}, "Symbolization": {"next_time": "4852488441"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "vQ7EacG6CR4rzcFaNFEsTQ"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": "7b4536f41cdaa5888408e82d0836e33dcf436466"}, "file": {"name": "libpthread-2.31.so"}}, "Symbolization": {"next_time": "4851738741"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "8NlMClggx8jaziUTJXlmWA"}} +{"@timestamp": "1698019200", "Executable": {"build": {"id": "f0983025f0e0f327a6da752ff4ffa675e0be393f"}, "file": {"name": "libpthread-2.31.so"}}, "Symbolization": {"next_time": "4851892090"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "wEIfl0pGBuepuUcTwdKc2g"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": "101d8983a254f1b26637242e10b80ecae42eff2d"}, "file": {"name": "libz.so.1.2.11"}}, "Symbolization": {"next_time": "4851735147"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "hrCIM-i5cm0sSd-oVTZrsg"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": ""}, "file": {"name": "libzip.so"}}, "Symbolization": {"next_time": "4851816838"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "EkX-Bbjdn5I_vHMz9xIc5g"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": ""}, "file": {"name": "libnio.so"}}, "Symbolization": {"next_time": "4851764933"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "lW7fiQNIMpeUsn4R6s23pw"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": ""}, "file": {"name": "pf-host-agent"}}, "Symbolization": {"next_time": "4851745916"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "-6ARRafpA91HAsyIFfBmBQ"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": ""}, "file": {"name": "libjava.so"}}, "Symbolization": {"next_time": "4852281623"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "SA0fsx010mzWc4t8ff1-lA"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": "ecf04bbd82c8ee9cc79e17dfca893ffa37a7ff5d"}, "file": {"name": "systemd"}}, "Symbolization": {"next_time": "4852496637"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "cEvvhuguaGYCmGWrnWRz1Q"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": "7ce3ef6b0c6c2c687b4c6643596171dc03569610"}, "file": {"name": "auditbeat"}}, "Symbolization": {"next_time": "4851745898"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "SVqTovndX3vgWwiNPDrUZA"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": ""}, "file": {"name": "libnio.so"}}, "Symbolization": {"next_time": "4851818483"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "UJmIMEc6K8gJ39NHYaD0Ww"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": "f7307432a8b162377e77a182b6cc2e53d771ec4b"}, "file": {"name": "libc-2.27.so"}}, "Symbolization": {"next_time": "4851753127"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "VNoyxdugL4O4pEtN-lEJQw"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": "dc83469c2c5d9aff0b32d242be021a5d8875e846"}, "file": {"name": "libsystemd-shared-237.so"}}, "Symbolization": {"next_time": "4852496565"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "634wiWh6F21tPpXr0Zz3mg"}} +{"@timestamp": "1698019200", "Executable": {"build": {"id": "9fdb74e7b217d06c93172a8243f8547f947ee6d1"}, "file": {"name": "libc-2.31.so"}}, "Symbolization": {"next_time": "4851892087"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "qjB_amR59Lv13Nhle_Nwbw"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": "1878e6b475720c7c51969e69ab2d276fae6d1dee"}, "file": {"name": "libc-2.31.so"}}, "Symbolization": {"next_time": "4851738739"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "QTXwwuqN0mLb-8lqckiaTQ"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": "6eee75cb89aff959ae1ba2c54194fd82d6aa604e"}, "file": {"name": "xfs"}}, "Symbolization": {"next_time": "4852490265"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "lHp5_WAgpLy2alrUVab6HA"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": "c5f89ea1c68710d2a493bb604c343a92c4f8ddeb"}, "file": {"name": "vmlinux"}}, "Symbolization": {"next_time": "4852491791"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "qvacTmpSpuAOHxTh9l_exA"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": ""}, "file": {"name": "libjvm.so"}}, "Symbolization": {"next_time": "4852281581"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "fhsEKXDuxJ-jIJrZpdRuSA"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": "0c75f746ff8d9d30d6417d79b7732a66f5890a8b"}, "file": {"name": "libz.so.1.2.11"}}, "Symbolization": {"next_time": "4851764966"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "Z3CKaS4aU08RhYA19y5ITQ"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": "ced267ea05869f9c3b8e789e83b7fa545f0b7c49"}, "file": {"name": "dockerd"}}, "Symbolization": {"next_time": "4852034064"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "z5k14PsCcDvFyUC1kmlV3Q"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": "0eca98a615313a9d2be9a993097f65969dee8816"}, "file": {"name": "containerd"}}, "Symbolization": {"next_time": "4851843260"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "Fuiq0RN8VMfwSLHsA6DzhA"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": "c52b0055d6c5b32b0561ffe7c25b40eba4aec3c2"}, "file": {"name": "libdbus-1.so.3.19.4"}}, "Symbolization": {"next_time": "4851947999"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "GXH6S9Nv2Lf0omTz4cH4RA"}} +{"@timestamp": "1699228800", "Executable": {"build": {"id": ""}, "file": {"name": "libzip.so"}}, "Symbolization": {"next_time": "4852250623"}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-executables", "_id": "qn_i-43Ca0UMIY4VcWF0Ag"}} +{"@timestamp": "1698624000", "Executable": {"build": {"id": ""}, "file": {"name": "libzip.so"}}, "Symbolization": {"next_time": "4852282517"}, "ecs": {"version": "1.12.0"}} diff --git a/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-stackframes.ndjson b/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-stackframes.ndjson new file mode 100644 index 0000000000000..b23b32d70d423 --- /dev/null +++ b/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-stackframes.ndjson @@ -0,0 +1,1414 @@ +{"create": {"_index": "profiling-stackframes", "_id": "Ms8C2PsOPTwJ_BWU-NqKjwAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractEventExecutor.java"], "Stackframe.function.name": ["void io.netty.util.concurrent.AbstractEventExecutor.safeExecute(java.lang.Runnable)"], "Stackframe.line.number": [167], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "cnnfHiXpYZi_Xvw7GhRVowAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ElasticsearchReaderManager.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.index.engine.ElasticsearchReaderManager.refreshIfNeeded(java.lang.Object)"], "Stackframe.line.number": [27], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ZUblhfsP3eghXbFQlRo7WwAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TemplateUtils.java"], "Stackframe.function.name": ["java.lang.String org.elasticsearch.xpack.core.template.TemplateUtils.loadTemplate(java.lang.String, java.lang.String, java.lang.String, java.util.Map)"], "Stackframe.line.number": [78], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Y8sJdN8JrOKeDGcKRLKzYgAAAAAAAAIR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository.doSnapshotShard(org.elasticsearch.repositories.SnapshotShardContext)"], "Stackframe.line.number": [2774], "Stackframe.function.offset": [86]} +{"create": {"_index": "profiling-stackframes", "_id": "v6rQl1rwEGF72NGhRc1i-wAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportService.java"], "Stackframe.function.name": ["void org.elasticsearch.transport.TransportService$6.doRun()"], "Stackframe.line.number": [1045], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "miWKFzr_hZhlQAnEjidKYAAAAAAAAABI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["java.lang.Object com.amazonaws.services.s3.AmazonS3Client.uploadObject(java.io.InputStream, java.io.File, com.amazonaws.services.s3.model.ObjectMetadata, com.amazonaws.event.ProgressListener, com.amazonaws.Request, com.amazonaws.services.s3.model.S3DataSource, boolean, boolean, com.amazonaws.services.s3.internal.UploadObjectStrategy, boolean)"], "Stackframe.line.number": [1808], "Stackframe.function.offset": [19]} +{"create": {"_index": "profiling-stackframes", "_id": "v7DgkUS5-ysky2RXxPMSjAAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PrioritizedEsThreadPoolExecutor.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.run()"], "Stackframe.line.number": [223], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "H7Tmwg9rKZuxl8eAfe0cQwAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECOperations.java"], "Stackframe.function.name": ["void sun.security.ec.ECOperations.double4(sun.security.ec.point.ProjectivePoint$Mutable, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP)"], "Stackframe.line.number": [221], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "K8SfjEwLlz1-kQVCzD5OrAAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileChannel.java"], "Stackframe.function.name": ["java.nio.channels.FileChannel java.nio.channels.FileChannel.open(java.nio.file.Path, java.util.Set, java.nio.file.attribute.FileAttribute[])"], "Stackframe.line.number": [298], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "U2mkS0m3fBX-awRgTvpBLAAAAAAAAAAl"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InflaterInputStream.java"], "Stackframe.function.name": ["int java.util.zip.InflaterInputStream.read(byte[], int, int)"], "Stackframe.line.number": [153], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "kgFL8vDWeva5yNPPlsZUygAAAAAAAAA3"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportReplicationAction.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.TransportReplicationAction$PrimaryShardReference.perform(org.elasticsearch.action.support.replication.ReplicationRequest, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [1132], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "JG_5wUb6Q1WN6hfRJFcqEwAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$1.doRun()"], "Stackframe.line.number": [341], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "r21mW5wIwvlwHfvj4Leh8wAAAAAAAACo"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexFileDeleter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexFileDeleter.checkpoint(org.apache.lucene.index.SegmentInfos, boolean)"], "Stackframe.line.number": [594], "Stackframe.function.offset": [29]} +{"create": {"_index": "profiling-stackframes", "_id": "p_xoiXGCsXgpBNHrp24JKQAAAAAAAAAV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixFileSystemProvider.java"], "Stackframe.function.name": ["java.nio.channels.FileChannel sun.nio.fs.UnixFileSystemProvider.newFileChannel(java.nio.file.Path, java.util.Set, java.nio.file.attribute.FileAttribute[])"], "Stackframe.line.number": [179], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "sh1T4lVaqf9yZbH1lhPrZwAAAAAAAACN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3RetryingInputStream.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3RetryingInputStream.openStream()"], "Stackframe.line.number": [88], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "mZnKLpDGXH5r07376P6zgQAAAAAAAACC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.authorize(org.elasticsearch.xpack.core.security.authc.Authentication, java.lang.String, org.elasticsearch.transport.TransportRequest, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [250], "Stackframe.function.offset": [37]} +{"create": {"_index": "profiling-stackframes", "_id": "YfB7-hrLT6gRL7Zld6Lp8QAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.store.CompositeRolesStore$$Lambda$7215+0x0000000802517b50.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "rHKWkGh7pRChoM8BKsgaigAAAAAAAAAU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["XContentBuilder.java"], "Stackframe.function.name": ["org.elasticsearch.xcontent.XContentBuilder org.elasticsearch.xcontent.XContentBuilder.field(java.lang.String, java.lang.String)"], "Stackframe.line.number": [643], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "MKXd2GM1KQ-HCXB8XbvuBAAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.s3.S3BlobContainer$$Lambda$8686+0x0000000802083178.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "A0rFfZQgIUFzz7A2AHAg7gAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["JsonXContentParser.java"], "Stackframe.function.name": ["org.elasticsearch.xcontent.XContentParser$Token org.elasticsearch.xcontent.provider.json.JsonXContentParser.currentToken()"], "Stackframe.line.number": [82], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Llon2cTU88EAb0xt_e8N6QAAAAAAAABB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Boolean2ScorerSupplier.java"], "Stackframe.function.name": ["org.apache.lucene.search.Scorer org.apache.lucene.search.Boolean2ScorerSupplier.getInternal(long)"], "Stackframe.line.number": [116], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "77BcdPidwQf8JepqsYlnCAAAAAAAAACP"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BuiltinClassLoader.java"], "Stackframe.function.name": ["java.net.URL jdk.internal.loader.BuiltinClassLoader.findResource(java.lang.String)"], "Stackframe.line.number": [355], "Stackframe.function.offset": [38]} +{"create": {"_index": "profiling-stackframes", "_id": "NNX6LKmc60fQ-V_Nw2_9cAAAAAAAAACw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InternalHttpClient.java"], "Stackframe.function.name": ["org.apache.http.client.methods.CloseableHttpResponse org.apache.http.impl.client.InternalHttpClient.doExecute(org.apache.http.HttpHost, org.apache.http.HttpRequest, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [185], "Stackframe.function.offset": [28]} +{"create": {"_index": "profiling-stackframes", "_id": "-FiXU0afkg-sRwubj2WiAwAAAAAAAKMr"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../../../bus/main.c"], "Stackframe.function.name": ["main"], "Stackframe.line.number": [719]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAATXEi"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["bio_alloc_bioset"]} +{"create": {"_index": "profiling-stackframes", "_id": "P8GYPQV63_J7gmzBFEQnJwAAAAAAAAFg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["com.amazonaws.services.s3.model.PutObjectResult com.amazonaws.services.s3.AmazonS3Client.putObject(com.amazonaws.services.s3.model.PutObjectRequest)"], "Stackframe.line.number": [1768], "Stackframe.function.offset": [57]} +{"create": {"_index": "profiling-stackframes", "_id": "sfblkOdEaLUvfgvW-QAdjgAAAAAAAAAS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ContextPreservingActionListener.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.ContextPreservingActionListener.onResponse(java.lang.Object)"], "Stackframe.line.number": [31], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "Iz1h-lJR9hawlSRETyh1qQAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.cluster.coordination.Coordinator$$Lambda$4965+0x0000000801ffcb58.apply(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ThW7aRtMu5otJJ-vTe9tmQAAAAAAAABM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["RoleReferenceIntersection.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.core.security.authz.store.RoleReferenceIntersection.lambda$buildRole$0(org.elasticsearch.action.ActionListener, java.util.Collection)"], "Stackframe.line.number": [49], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "P4ZjI7g37fWz45cghUwTxQAAAAAAAAA_"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IOUtils.java"], "Stackframe.function.name": ["void org.apache.lucene.util.IOUtils.fsync(java.nio.file.Path, boolean)"], "Stackframe.line.number": [467], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "obEloCxQqTOnivGYEKZEEAAAAAAAAAAu"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SslHandler.java"], "Stackframe.function.name": ["void io.netty.handler.ssl.SslHandler.wrapAndFlush(io.netty.channel.ChannelHandlerContext)"], "Stackframe.line.number": [799], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "BqEVAd8MPrtkYhJgO4GhZAAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService$$Lambda$10693+0x0000000802972b08.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "6bLW5yM_cMbSli8_NsjzNQAAAAAAAAAc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer()"], "Stackframe.line.number": [764], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "R_ZB9SjojSrHSQ8dwIq_WQAAAAAAAAOr"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IntegerPolynomialP256.java"], "Stackframe.function.name": ["void sun.security.util.math.intpoly.IntegerPolynomialP256.mult(long[], long[], long[])"], "Stackframe.line.number": [323], "Stackframe.function.offset": [20]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAApz47"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["down_read"]} +{"create": {"_index": "profiling-stackframes", "_id": "OUMeXZZ6fO-t9RTzPCiRtQAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots org.elasticsearch.repositories.blobstore.BlobStoreRepository.lambda$static$4(java.lang.String, org.elasticsearch.xcontent.XContentParser)"], "Stackframe.line.number": [310], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ggEpzKYUUToCZ_IIz775eQAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FilterMergePolicy.java"], "Stackframe.function.name": ["int org.apache.lucene.index.FilterMergePolicy.numDeletesToMerge(org.apache.lucene.index.SegmentCommitInfo, int, org.apache.lucene.util.IOSupplier)"], "Stackframe.line.number": [125], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "11cfhNddX4HypzZBkL8GJAAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractPipeline.java"], "Stackframe.function.name": ["java.util.stream.Sink java.util.stream.AbstractPipeline.wrapAndCopyInto(java.util.stream.Sink, java.util.Spliterator)"], "Stackframe.line.number": [499], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "2s9Z-1EblvqRm49b6lgFUQAAAAAAAAAy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexShard.java"], "Stackframe.function.name": ["void org.elasticsearch.index.shard.IndexShard.acquirePrimaryOperationPermit(org.elasticsearch.action.ActionListener, java.lang.String, java.lang.Object, boolean)"], "Stackframe.line.number": [3325], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "r5B7woHUx7hmwtrjMRtOuwAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Boolean2ScorerSupplier.java"], "Stackframe.function.name": ["org.apache.lucene.search.Scorer org.apache.lucene.search.Boolean2ScorerSupplier.get(long)"], "Stackframe.line.number": [96], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAIgE0"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["mempool_alloc_slab"]} +{"create": {"_index": "profiling-stackframes", "_id": "QTXwwuqN0mLb-8lqckiaTQAAAAAACHly"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["xfs_xattr_get"]} +{"create": {"_index": "profiling-stackframes", "_id": "89ogc5uTbOqh2aLJlzHKmAAAAAAAAAAX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["KeyPairGenerator.java"], "Stackframe.function.name": ["java.security.KeyPair java.security.KeyPairGenerator$Delegate.generateKeyPair()"], "Stackframe.line.number": [727], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "izoIqZ2EJGa3SfYoWrHMNwAAAAAAAAFJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportReplicationAction.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.TransportReplicationAction$AsyncPrimaryAction.runWithPrimaryShardReference(org.elasticsearch.action.support.replication.TransportReplicationAction$PrimaryShardReference)"], "Stackframe.line.number": [508], "Stackframe.function.offset": [82]} +{"create": {"_index": "profiling-stackframes", "_id": "ZgY89pr05YIxi0DTL7hyTAAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Thread.java"], "Stackframe.function.name": ["void java.lang.Thread.run()"], "Stackframe.line.number": [833], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "IGvGOb8CKDtqOos6VI5kCAAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object jdk.internal.loader.BuiltinClassLoader$$Lambda$289+0x0000000800d277b0.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "c6kGRam5UKx_l-9ot9811wAAAAAAAACL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecureRandom.java"], "Stackframe.function.name": ["void sun.security.provider.SecureRandom.engineNextBytes(byte[])"], "Stackframe.line.number": [250], "Stackframe.function.offset": [28]} +{"create": {"_index": "profiling-stackframes", "_id": "pT78-BAaZ59fFXwdquC9KQAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexFileDeleter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexFileDeleter.deleteFile(java.lang.String)"], "Stackframe.line.number": [764], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "zVmgO3uV6bZtx2XUjAkXcgAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3RetryingInputStream.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3RetryingInputStream.(org.elasticsearch.repositories.s3.S3BlobStore, java.lang.String)"], "Stackframe.line.number": [60], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "i0MJsjg1nTMYg21Lh8j5pAAAAAAAAABT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileGenerator.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileGenerator.writeString(java.lang.String)"], "Stackframe.line.number": [977], "Stackframe.function.offset": [27]} +{"create": {"_index": "profiling-stackframes", "_id": "DjDYSMBm5cQnwKCRH0dlgwAAAAAAAAA0"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SslHandler.java"], "Stackframe.function.name": ["void io.netty.handler.ssl.SslHandler.flush(io.netty.channel.ChannelHandlerContext)"], "Stackframe.line.number": [780], "Stackframe.function.offset": [15]} +{"create": {"_index": "profiling-stackframes", "_id": "npAgnwz74Cv0gdn19xaCuAAAAAAAAADy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FieldAndDocumentLevelSecurityRequestInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.FieldAndDocumentLevelSecurityRequestInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [79], "Stackframe.function.offset": [28]} +{"create": {"_index": "profiling-stackframes", "_id": "g6A5O5DE8lToSDEW-xkfTwAAAAAAAAES"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DlsFlsLicenseRequestInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.DlsFlsLicenseRequestInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [106], "Stackframe.function.offset": [57]} +{"create": {"_index": "profiling-stackframes", "_id": "3tyP2MUj5DlzxqSh6DRnAwAAAAAAAAFk"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.runTask(java.lang.String, java.util.function.Function, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [437], "Stackframe.function.offset": [53]} +{"create": {"_index": "profiling-stackframes", "_id": "igHK0MNBkupTzE8Gzv6EygAAAAAAAAAJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessController.java"], "Stackframe.function.name": ["java.lang.Object java.security.AccessController.doPrivileged(java.security.PrivilegedExceptionAction)"], "Stackframe.line.number": [569], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "DPKZG86WCe-rkFTx_IAdagAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.s3.S3RetryingInputStream$$Lambda$8093+0x00000008026d5b90.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "y87oVbGEsnZxcfXcdrLYswAAAAAAAAB6"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLSocketImpl.java"], "Stackframe.function.name": ["void sun.security.ssl.SSLSocketImpl.startHandshake(boolean)"], "Stackframe.line.number": [455], "Stackframe.function.offset": [25]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAFFQe"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__x64_sys_futex"]} +{"create": {"_index": "profiling-stackframes", "_id": "XC-En-mvMkLnH_EQYbHTPwAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractEventExecutor.java"], "Stackframe.function.name": ["void io.netty.util.concurrent.AbstractEventExecutor.runTask(java.lang.Runnable)"], "Stackframe.line.number": [174], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Hc4fbH8wnl-aZPA1_7gB-wAAAAAAAAAy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3BlobContainer.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3BlobContainer.writeMetadataBlob(java.lang.String, boolean, boolean, org.elasticsearch.core.CheckedConsumer)"], "Stackframe.line.number": [211], "Stackframe.function.offset": [69]} +{"create": {"_index": "profiling-stackframes", "_id": "mwuQyMHK5RYUe_aaboQONgAAAAAAAAAD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TemplateUtils.java"], "Stackframe.function.name": ["java.lang.String org.elasticsearch.xpack.core.template.TemplateUtils.load(java.lang.String)"], "Stackframe.line.number": [91], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "we1dsXqMIsi7fg7veaiJyQAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.blobstore.BlobStoreRepository$$Lambda$720+0x0000000800ff2238.apply(java.lang.Object, java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAADSlT"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["wake_up_q"]} +{"create": {"_index": "profiling-stackframes", "_id": "Slb8MNVKIFfE9HJqtN3BhwAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Inflater.java"], "Stackframe.function.name": ["long java.util.zip.Inflater.inflateBytesBytes(long, byte[], int, int, byte[], int, int)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "8NlMClggx8jaziUTJXlmWAAAAAAAAIYI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["/build/glibc-sMfBJT/glibc-2.31/nptl/pthread_create.c"], "Stackframe.function.name": ["start_thread"], "Stackframe.line.number": [477]} +{"create": {"_index": "profiling-stackframes", "_id": "kpBi8LFR3RTjSJqFuljOuQAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Deflater.java"], "Stackframe.function.name": ["long java.util.zip.Deflater.deflateBytesBytes(long, byte[], int, int, byte[], int, int, int, int)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "nTSx0380z3H9OoIBkTGK2wAAAAAAAAAx"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedInputStream.java"], "Stackframe.function.name": ["int java.io.BufferedInputStream.implRead(byte[], int, int)"], "Stackframe.line.number": [382], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "-FIPITFlgylPOavKEh5SEQAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileParser._loadMoreGuaranteed()"], "Stackframe.line.number": [268], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "GASd-GxTI3CnNP2dmtJ0gQAAAAAAAAHe"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexWriter.startCommit(org.apache.lucene.index.SegmentInfos)"], "Stackframe.line.number": [5099], "Stackframe.function.offset": [70]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAC3tR"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["wq_barrier_func"]} +{"create": {"_index": "profiling-stackframes", "_id": "GXzDmM-hgGlNDieJ9EUhGAAAAAAAAAEw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshots.java"], "Stackframe.function.name": ["org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots.fromXContent(org.elasticsearch.xcontent.XContentParser)"], "Stackframe.line.number": [276], "Stackframe.function.offset": [33]} +{"create": {"_index": "profiling-stackframes", "_id": "Sm149Q4n4lGGxDiZNf94FQAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassLoader.java"], "Stackframe.function.name": ["java.io.InputStream java.net.URLClassLoader.getResourceAsStream(java.lang.String)"], "Stackframe.line.number": [290], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "YRdREBgpcH5AEXcyjzcstgAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["org.elasticsearch.index.engine.Engine$IndexCommitRef org.elasticsearch.index.IndexModule$$Lambda$1431+0x00000008017d2f18.acquireIndexCommitForSnapshot(org.elasticsearch.index.engine.Engine)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "1AYJQILFnHXEV9oxyMp7FgAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["StandardDirectoryReader.java"], "Stackframe.function.name": ["org.apache.lucene.index.DirectoryReader org.apache.lucene.index.StandardDirectoryReader.doOpenIfChanged()"], "Stackframe.line.number": [345], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ObVu4EglQ3fhG0H69eyyrwAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.ServerTransportFilter$NodeProfile$$Lambda$5828+0x0000000801afad38.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "RO5fkjSwhs-w1RFeE1q0WQAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ByteSizeCachingDirectory.java"], "Stackframe.function.name": ["void org.elasticsearch.index.store.ByteSizeCachingDirectory.deleteFile(java.lang.String)"], "Stackframe.line.number": [187], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "t5nOOkOTN6QC2_E5A_LICgAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["int com.fasterxml.jackson.dataformat.smile.SmileParser._fourBytesToInt()"], "Stackframe.line.number": [2216], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "kBc-zardyidmU9M2FtS_7wAAAAAAAACU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexFileDeleter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexFileDeleter.deleteFiles(java.util.Collection)"], "Stackframe.line.number": [758], "Stackframe.function.offset": [24]} +{"create": {"_index": "profiling-stackframes", "_id": "wlgkZprTs7F_QEEM-7VvZwAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["JsonXContentGenerator.java"], "Stackframe.function.name": ["void org.elasticsearch.xcontent.json.JsonXContentGenerator.writeStringField(java.lang.String, java.lang.String)"], "Stackframe.line.number": [266], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "yzkMNxhTqdw8OfBH6KpVbgAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixPath.java"], "Stackframe.function.name": ["void sun.nio.fs.UnixPath.(sun.nio.fs.UnixFileSystem, java.lang.String)"], "Stackframe.line.number": [68], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "KPNYkONEetadZox7ZPj-_wAAAAAAAACo"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SnapshotShardsService.java"], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService.snapshot(org.elasticsearch.index.shard.ShardId, org.elasticsearch.snapshots.Snapshot, org.elasticsearch.repositories.IndexId, java.util.Map, org.elasticsearch.index.snapshots.IndexShardSnapshotStatus, org.elasticsearch.Version, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [370], "Stackframe.function.offset": [19]} +{"create": {"_index": "profiling-stackframes", "_id": "mcb8sJ6l3hl2564ZMljLvwAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ActionRunnable.java"], "Stackframe.function.name": ["void org.elasticsearch.action.ActionRunnable$2.doRun()"], "Stackframe.line.number": [62], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "TF5EMbS9G90c1yPz7xfKpAAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Netty4WriteThrottlingHandler.java"], "Stackframe.function.name": ["void org.elasticsearch.transport.netty4.Netty4WriteThrottlingHandler.flush(io.netty.channel.ChannelHandlerContext)"], "Stackframe.line.number": [101], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "QTXwwuqN0mLb-8lqckiaTQAAAAAAB4A6"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["xfs_ilock"]} +{"create": {"_index": "profiling-stackframes", "_id": "X6aGRt4o2LlAY8FCZg8t_gAAAAAAAACW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FilePermission.java"], "Stackframe.function.name": ["int java.io.FilePermission.containsPath(java.nio.file.Path, java.nio.file.Path)"], "Stackframe.line.number": [744], "Stackframe.function.offset": [43]} +{"create": {"_index": "profiling-stackframes", "_id": "16j8HtYPNjd3qdRoXnMi0AAAAAAAAABa"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClientKeyExchange.java"], "Stackframe.function.name": ["byte[] sun.security.ssl.ClientKeyExchange$ClientKeyExchangeProducer.produce(sun.security.ssl.ConnectionContext, sun.security.ssl.SSLHandshake$HandshakeMessage)"], "Stackframe.line.number": [65], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "DJLJRrJNARIRyzxCnxa4WQAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadContext.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun()"], "Stackframe.line.number": [777], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "vtg0KUtDPlzazpv9gP6MtQAAAAAAAABQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileChannelImpl.java"], "Stackframe.function.name": ["void sun.nio.ch.FileChannelImpl.force(boolean)"], "Stackframe.line.number": [468], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "CQUIxcdtvT35ZznMVnzc_AAAAAAAAACX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndicesClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.indices.cluster.IndicesClusterStateService.updateShard(org.elasticsearch.cluster.node.DiscoveryNodes, org.elasticsearch.cluster.routing.ShardRouting, org.elasticsearch.indices.cluster.IndicesClusterStateService$Shard, org.elasticsearch.cluster.routing.RoutingTable, org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [614], "Stackframe.function.offset": [14]} +{"create": {"_index": "profiling-stackframes", "_id": "yknYMrbzZKxPQGFhtsDUFgAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecureRandom.java"], "Stackframe.function.name": ["void java.security.SecureRandom.nextBytes(byte[])"], "Stackframe.line.number": [758], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "VGJu-bYJBqeYrugO5COQdQAAAAAAAAHG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ProtocolExec.java"], "Stackframe.function.name": ["org.apache.http.client.methods.CloseableHttpResponse org.apache.http.impl.execchain.ProtocolExec.execute(org.apache.http.conn.routing.HttpRoute, org.apache.http.client.methods.HttpRequestWrapper, org.apache.http.client.protocol.HttpClientContext, org.apache.http.client.methods.HttpExecutionAware)"], "Stackframe.line.number": [186], "Stackframe.function.offset": [76]} +{"create": {"_index": "profiling-stackframes", "_id": "hCl4jaZsLnxk3C0-qCjENwAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3RetryingInputStream.java"], "Stackframe.function.name": ["com.amazonaws.services.s3.model.S3Object org.elasticsearch.repositories.s3.S3RetryingInputStream.lambda$openStream$0(org.elasticsearch.repositories.s3.AmazonS3Reference, com.amazonaws.services.s3.model.GetObjectRequest)"], "Stackframe.line.number": [88], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "x9iJb0OhVB_kYAbvkT0IJAAAAAAAAABL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ConjunctionUtils.java"], "Stackframe.function.name": ["org.apache.lucene.search.DocIdSetIterator org.apache.lucene.search.ConjunctionUtils.intersectScorers(java.util.Collection)"], "Stackframe.line.number": [44], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "mT2KIvEsDcA0I2RiMDHSuAAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportReplicationAction.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.TransportReplicationAction.acquirePrimaryOperationPermit(org.elasticsearch.index.shard.IndexShard, org.elasticsearch.action.support.replication.ReplicationRequest, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [1069], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "DsEcu-FxsA-5RKaWB1grdAAAAAAAAAAp"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractChannelHandlerContext.java"], "Stackframe.function.name": ["void io.netty.channel.AbstractChannelHandlerContext$WriteTask.run()"], "Stackframe.line.number": [1247], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "Cqi1_91FMYO-otI3ESGKwAAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixNativeDispatcher.java"], "Stackframe.function.name": ["int sun.nio.fs.UnixNativeDispatcher.open(sun.nio.fs.UnixPath, int, int)"], "Stackframe.line.number": [68], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "HtE3mFX5tbjDXSQ7-hE7bwAAAAAAAAF5"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(com.amazonaws.http.AmazonHttpClient$RequestExecutor$ExecOneRequestParams)"], "Stackframe.line.number": [1323], "Stackframe.function.offset": [58]} +{"create": {"_index": "profiling-stackframes", "_id": "HBnk-r2jmt48gRTNgMOf8wAAAAAAAAAy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService$1.onResponse(java.lang.Void)"], "Stackframe.line.number": [603], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "WIcSMVTq4UI8IrRuG9Ls8gAAAAAAAAAS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute()"], "Stackframe.line.number": [738], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "vRwDdjzKXpreYxi90MDGQQAAAAAAAABm"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Method.java"], "Stackframe.function.name": ["java.lang.Object java.lang.reflect.Method.invoke(java.lang.Object, java.lang.Object[])"], "Stackframe.line.number": [578], "Stackframe.function.offset": [19]} +{"create": {"_index": "profiling-stackframes", "_id": "6bg5h6o_GLUtahiXyigwlQAAAAAAAAAs"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SingleThreadEventExecutor.java"], "Stackframe.function.name": ["void io.netty.util.concurrent.SingleThreadEventExecutor$4.run()"], "Stackframe.line.number": [997], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "TqPaoMp2He0IHYVdVGyXigAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat$$Lambda$8685+0x000000080207b720.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "O4wSxfOE-c6Sft1wmb_6fgAAAAAAAABp"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECDHClientKeyExchange.java"], "Stackframe.function.name": ["byte[] sun.security.ssl.ECDHClientKeyExchange$ECDHEClientKeyExchangeProducer.produce(sun.security.ssl.ConnectionContext, sun.security.ssl.SSLHandshake$HandshakeMessage)"], "Stackframe.line.number": [396], "Stackframe.function.offset": [21]} +{"create": {"_index": "profiling-stackframes", "_id": "Gjm3c7dzM21apoh-_mmOrwAAAAAAAAA2"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshots.java"], "Stackframe.function.name": ["org.elasticsearch.xcontent.XContentBuilder org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots.toXContent(org.elasticsearch.xcontent.XContentBuilder, org.elasticsearch.xcontent.ToXContent$Params)"], "Stackframe.line.number": [238], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "AxJNlcXBvrgVOpR8a1WEYgAAAAAAAALR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["MainClientExec.java"], "Stackframe.function.name": ["org.apache.http.client.methods.CloseableHttpResponse org.apache.http.impl.execchain.MainClientExec.execute(org.apache.http.conn.routing.HttpRoute, org.apache.http.client.methods.HttpRequestWrapper, org.apache.http.client.protocol.HttpClientContext, org.apache.http.client.methods.HttpExecutionAware)"], "Stackframe.line.number": [272], "Stackframe.function.offset": [117]} +{"create": {"_index": "profiling-stackframes", "_id": "W80Dopg4GIca7JXLWHFwVAAAAAAAAAA6"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ServerTransportFilter.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.ServerTransportFilter$NodeProfile.lambda$inbound$1(java.lang.String, org.elasticsearch.transport.TransportRequest, org.elasticsearch.action.ActionListener, org.elasticsearch.Version, org.elasticsearch.xpack.core.security.authc.Authentication)"], "Stackframe.line.number": [136], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "ibtF619pQHGNKblUfb9MKwAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLHandshake.java"], "Stackframe.function.name": ["void sun.security.ssl.SSLHandshake.consume(sun.security.ssl.ConnectionContext, java.nio.ByteBuffer)"], "Stackframe.line.number": [396], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "rQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService$1.onResponse(java.lang.Object)"], "Stackframe.line.number": [597], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAIgIA"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["mempool_alloc"]} +{"create": {"_index": "profiling-stackframes", "_id": "Iir7eGdVZsjNItLfykxs3gAAAAAAAABa"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PersistedClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.PersistedClusterStateService$Writer.writeIncrementalStateAndCommit(long, org.elasticsearch.cluster.ClusterState, org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [777], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "VB_A4ihbc-HQ5QDUj7Wz0QAAAAAAAAB_"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ElasticsearchConcurrentMergeScheduler.java"], "Stackframe.function.name": ["void org.elasticsearch.index.engine.ElasticsearchConcurrentMergeScheduler.doMerge(org.apache.lucene.index.MergeScheduler$MergeSource, org.apache.lucene.index.MergePolicy$OneMerge)"], "Stackframe.line.number": [118], "Stackframe.function.offset": [22]} +{"create": {"_index": "profiling-stackframes", "_id": "PK6VPfk6aJqBe-5Qji8O5gAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PendingReplicationActions.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.PendingReplicationActions.accept(java.lang.Object)"], "Stackframe.line.number": [25], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "7HvWKt-xH9wdfCiPfTMv1gAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Engine.java"], "Stackframe.function.name": ["org.elasticsearch.index.engine.Engine$IndexCommitRef org.elasticsearch.index.engine.Engine.acquireIndexCommitForSnapshot()"], "Stackframe.line.number": [1101], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "IP-4kS9K2z9dstopMSzdEAAAAAAAAAAK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexTemplateRegistry.java"], "Stackframe.function.name": ["boolean org.elasticsearch.xpack.core.template.IndexTemplateRegistry.componentTemplatesExist(org.elasticsearch.cluster.ClusterState, org.elasticsearch.xpack.core.template.IndexTemplateConfig)"], "Stackframe.line.number": [345], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "p0eo8IwDes1buf90_PyLCgAAAAAAAAAw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ConcurrentMergeScheduler.java"], "Stackframe.function.name": ["void org.apache.lucene.index.ConcurrentMergeScheduler$MergeThread.run()"], "Stackframe.line.number": [700], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "sCbyN1LnxDvKtK16chc1UAAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FilterMergePolicy.java"], "Stackframe.function.name": ["org.apache.lucene.index.MergePolicy$MergeSpecification org.apache.lucene.index.FilterMergePolicy.findMerges(org.apache.lucene.index.MergeTrigger, org.apache.lucene.index.SegmentInfos, org.apache.lucene.index.MergePolicy$MergeContext)"], "Stackframe.line.number": [47], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "DT06D5A7Yxiqx6IJ7Kvu0wAAAAAAAACC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECKeyPairGenerator.java"], "Stackframe.function.name": ["java.util.Optional sun.security.ec.ECKeyPairGenerator.generateKeyPairImpl(java.security.SecureRandom)"], "Stackframe.line.number": [200], "Stackframe.function.offset": [17]} +{"create": {"_index": "profiling-stackframes", "_id": "T2JZOvbQqWZFxul4uMvPzgAAAAAAAABw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["OutputRecord.java"], "Stackframe.function.name": ["long sun.security.ssl.OutputRecord.t13Encrypt(sun.security.ssl.SSLCipher$SSLWriteCipher, byte, java.nio.ByteBuffer, int, int, int, sun.security.ssl.ProtocolVersion)"], "Stackframe.line.number": [418], "Stackframe.function.offset": [21]} +{"create": {"_index": "profiling-stackframes", "_id": "5GqAUaYv86rgi1w0nJY6kwAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService$$Lambda$8707+0x00000008020ba880.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "5iqVHRdlT1dn5BdxVeiR7QAAAAAAAABL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(com.amazonaws.http.HttpResponseHandler)"], "Stackframe.line.number": [680], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "qAh8Zr_sGDY6rzZXJvta_gAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DefaultBHttpClientConnection.java"], "Stackframe.function.name": ["void org.apache.http.impl.DefaultBHttpClientConnection.sendRequestEntity(org.apache.http.HttpEntityEnclosingRequest)"], "Stackframe.line.number": [156], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "ZJExOMsbBfy8WqRgcylCLwAAAAAAAABp"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["RequestHandlerRegistry.java"], "Stackframe.function.name": ["void org.elasticsearch.transport.RequestHandlerRegistry.processMessageReceived(org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel)"], "Stackframe.line.number": [67], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "FTndS9swIMm3K06ZEY-ZcgAAAAAAAACO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileChannelImpl.java"], "Stackframe.function.name": ["void sun.nio.ch.FileChannelImpl.implCloseChannel()"], "Stackframe.line.number": [207], "Stackframe.function.offset": [30]} +{"create": {"_index": "profiling-stackframes", "_id": "fo33a47SQa_ikvx14Wo6GwAAAAAAAAAm"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractChannelHandlerContext.java"], "Stackframe.function.name": ["void io.netty.channel.AbstractChannelHandlerContext.invokeFlush0()"], "Stackframe.line.number": [923], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "jJKyCKnhVDSt_DYivEq7yQAAAAAAAAAi"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexShard.java"], "Stackframe.function.name": ["org.elasticsearch.index.engine.Engine$RefreshResult org.elasticsearch.index.shard.IndexShard.refresh(java.lang.String)"], "Stackframe.line.number": [1236], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "u6cCfBpCKfnm1TY7pHx6ZgAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Store.java"], "Stackframe.function.name": ["void org.elasticsearch.index.store.Store$StoreDirectory.deleteFile(java.lang.String)"], "Stackframe.line.number": [751], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "gvYbo0YBmE65VwrpTWYalQAAAAAAAABi"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndicesClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.indices.cluster.IndicesClusterStateService.applyClusterState(org.elasticsearch.cluster.ClusterChangedEvent)"], "Stackframe.line.number": [223], "Stackframe.function.offset": [23]} +{"create": {"_index": "profiling-stackframes", "_id": "2sLCSYuHZvfeGvojnGEVfAAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BooleanWeight.java"], "Stackframe.function.name": ["org.apache.lucene.search.Scorer org.apache.lucene.search.BooleanWeight.scorer(org.apache.lucene.index.LeafReaderContext)"], "Stackframe.line.number": [511], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "gWzF_k06817SsAPAEeEuVAAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3BlobContainer.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3BlobContainer.lambda$executeSingleUpload$19(org.elasticsearch.repositories.s3.AmazonS3Reference, com.amazonaws.services.s3.model.PutObjectRequest)"], "Stackframe.line.number": [490], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAwAIU"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["ret_from_fork"]} +{"create": {"_index": "profiling-stackframes", "_id": "0vu-8-EP1KjTNjFoZrojbgAAAAAAAAAh"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InternalEngine.java"], "Stackframe.function.name": ["void org.elasticsearch.index.engine.InternalEngine.commitIndexWriter(org.apache.lucene.index.IndexWriter, org.elasticsearch.index.translog.Translog)"], "Stackframe.line.number": [2644], "Stackframe.function.offset": [29]} +{"create": {"_index": "profiling-stackframes", "_id": "C5AvyP3bA4PjoVWj5P4reAAAAAAAAAAD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["RoleReferenceIntersection.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.core.security.authz.store.RoleReferenceIntersection.lambda$buildRole$1(java.util.function.BiConsumer, org.elasticsearch.action.support.GroupedActionListener, org.elasticsearch.xpack.core.security.authz.store.RoleReference)"], "Stackframe.line.number": [53], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "4O-ovD5bG6j4SStsVkgzcQAAAAAAAABH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.callClusterStateListener(org.elasticsearch.cluster.ClusterChangedEvent, org.elasticsearch.cluster.service.ClusterApplierRecordingService$Recorder, java.util.Collection)"], "Stackframe.line.number": [573], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "4wHidvPJpZTy7Q8lK21otgAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexShardOperationPermits.java"], "Stackframe.function.name": ["void org.elasticsearch.index.shard.IndexShardOperationPermits.acquire(org.elasticsearch.action.ActionListener, java.lang.String, boolean, java.lang.Object)"], "Stackframe.line.number": [221], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "nENxGWN4qoBcDN73jfy1WgAAAAAAAADf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["HttpRequestExecutor.java"], "Stackframe.function.name": ["org.apache.http.HttpResponse org.apache.http.protocol.HttpRequestExecutor.doSendRequest(org.apache.http.HttpRequest, org.apache.http.HttpClientConnection, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [238], "Stackframe.function.offset": [42]} +{"create": {"_index": "profiling-stackframes", "_id": "iZ3mYpUZ_vDzUycli_tCcQAAAAAAAAAu"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileParser._parseNumericValue()"], "Stackframe.line.number": [1983], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "CW_r9nc-U1Fi4aU8XoGOXAAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassLoader.java"], "Stackframe.function.name": ["java.net.URL java.net.URLClassLoader$2.run()"], "Stackframe.line.number": [629], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "KZn_I-ZtnojJ7fIDZHHneQAAAAAAAACF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshots.java"], "Stackframe.function.name": ["org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots.fromXContent(org.elasticsearch.xcontent.XContentParser)"], "Stackframe.line.number": [278], "Stackframe.function.offset": [18]} +{"create": {"_index": "profiling-stackframes", "_id": "hvijQK1a5iiWkFoVp40y5AAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClientConnectionManagerFactory.java"], "Stackframe.function.name": ["java.lang.Object com.amazonaws.http.conn.ClientConnectionManagerFactory$Handler.invoke(java.lang.Object, java.lang.reflect.Method, java.lang.Object[])"], "Stackframe.line.number": [76], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "k8ZGDLdSxDSM2lL9n7Ll1AAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["int org.apache.lucene.index.IndexWriter.numDeletesToMerge(org.apache.lucene.index.SegmentCommitInfo)"], "Stackframe.line.number": [5996], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "_iIILZSpQek0elrVmCyWigAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3BlobContainer.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3BlobContainer.writeBlob(java.lang.String, java.io.InputStream, long, boolean)"], "Stackframe.line.number": [126], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "92jCQ7FJd_qE8nezHXMXDQAAAAAAAACY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DocumentsWriter.java"], "Stackframe.function.name": ["long org.apache.lucene.index.DocumentsWriter.flushAllThreads()"], "Stackframe.line.number": [672], "Stackframe.function.offset": [22]} +{"create": {"_index": "profiling-stackframes", "_id": "_GFFImAT2VE6Ar5VgmaN7QAAAAAAAAHy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReplicationTracker.java"], "Stackframe.function.name": ["void org.elasticsearch.index.seqno.ReplicationTracker.updateFromMaster(long, java.util.Set, org.elasticsearch.cluster.routing.IndexShardRoutingTable)"], "Stackframe.line.number": [1198], "Stackframe.function.offset": [47]} +{"create": {"_index": "profiling-stackframes", "_id": "HbqGqBv_PKwJzIrNCTkRMwAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReadersAndUpdates.java"], "Stackframe.function.name": ["int org.apache.lucene.index.ReadersAndUpdates.numDeletesToMerge(org.apache.lucene.index.MergePolicy)"], "Stackframe.line.number": [239], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "2CwcPQcRkPoyzD89B8CtBQAAAAAAAAB-"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PersistedClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.PersistedClusterStateService$MetadataIndexWriter.prepareCommit(java.lang.String, long, long, org.elasticsearch.Version)"], "Stackframe.line.number": [654], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "X5HtFQ851Qd03wmhWrZR4QAAAAAAAAAq"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SingleThreadEventExecutor.java"], "Stackframe.function.name": ["boolean io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(long)"], "Stackframe.line.number": [470], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "V6Xv8yqxVPgM9JZWMvD_ogAAAAAAAAAD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CloseableHttpClient.java"], "Stackframe.function.name": ["org.apache.http.HttpResponse org.apache.http.impl.client.CloseableHttpClient.execute(org.apache.http.client.methods.HttpUriRequest, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [56], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "4FrN4VeCu4HJZNavjw0tmgAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FSDirectory.java"], "Stackframe.function.name": ["void org.apache.lucene.store.FSDirectory.sync(java.util.Collection)"], "Stackframe.line.number": [255], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "h7EwhN3qpUCOhT4f95Qz0gAAAAAAAAEx"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["RBACEngine.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.RBACEngine.authorizeIndexAction(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AsyncSupplier, java.util.Map, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [360], "Stackframe.function.offset": [75]} +{"create": {"_index": "profiling-stackframes", "_id": "bUctjbWcypk0wJUtGkKmFgAAAAAAAADJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ServerTransportFilter.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.ServerTransportFilter$NodeProfile.inbound(java.lang.String, org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [128], "Stackframe.function.offset": [33]} +{"create": {"_index": "profiling-stackframes", "_id": "SCkRMoBKKPg9IolzXt51kAAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository$$Lambda$7036+0x000000080246f9c8.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lMvGIQySgdUzptqnRUQpUAAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["StoredFieldsConsumer.java"], "Stackframe.function.name": ["void org.apache.lucene.index.StoredFieldsConsumer.flush(org.apache.lucene.index.SegmentWriteState, org.apache.lucene.index.Sorter$DocMap)"], "Stackframe.line.number": [82], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "qOuY7z4cBuGpkQnMsu5M4QAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixPath.java"], "Stackframe.function.name": ["sun.nio.fs.UnixPath sun.nio.fs.UnixPath.getName(int)"], "Stackframe.line.number": [301], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "X3YYuA6U4OM2ZjQu8fVwYwAAAAAAAABL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutionBuilderImpl.execute(com.amazonaws.http.HttpResponseHandler)"], "Stackframe.line.number": [697], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "rLErn8W8_gFwO7zvGO-LBwAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.blobstore.BlobStoreRepository$$Lambda$907+0x00000008016ae128.apply(java.lang.Object, java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "_NbGvNUAYnubBA2-cDqDrQAAAAAAAAAe"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Iterable.java"], "Stackframe.function.name": ["void java.lang.Iterable.forEach(java.util.function.Consumer)"], "Stackframe.line.number": [75], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "LewwMUXjDx8itumiJ5m9uAAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$3.onResponse(java.lang.Void)"], "Stackframe.line.number": [590], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadPoolExecutor.java"], "Stackframe.function.name": ["void java.util.concurrent.ThreadPoolExecutor$Worker.run()"], "Stackframe.line.number": [642], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Y8sJdN8JrOKeDGcKRLKzYgAAAAAAAACu"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository.doSnapshotShard(org.elasticsearch.repositories.SnapshotShardContext)"], "Stackframe.line.number": [2713], "Stackframe.function.offset": [25]} +{"create": {"_index": "profiling-stackframes", "_id": "oxSW321mwoDVUa0haFaP7QAAAAAAAAAu"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PersistedClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.PersistedClusterStateService$Writer.prepareCommit(long, long, org.elasticsearch.Version)"], "Stackframe.line.number": [1175], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "-FiXU0afkg-sRwubj2WiAwAAAAAAAlz6"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../../../dbus/dbus-mainloop.c", "../../../dbus/dbus-mainloop.c"], "Stackframe.function.name": ["_dbus_loop_iterate", "_dbus_loop_dispatch"], "Stackframe.line.number": [862, 520]} +{"create": {"_index": "profiling-stackframes", "_id": "BPKHF0uWLBGF-EDvSz50AwAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UpdateRequestInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.UpdateRequestInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [27], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "bYAsS36pm68v9ebHURyhdAAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IntegerPolynomial.java"], "Stackframe.function.name": ["sun.security.util.math.MutableIntegerModuloP sun.security.util.math.intpoly.IntegerPolynomial$MutableElement.setProduct(sun.security.util.math.IntegerModuloP)"], "Stackframe.line.number": [631], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "WyzAcL0AWgIkl4A0YO5RQwAAAAAAAAAm"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["StandardDirectoryReader.java"], "Stackframe.function.name": ["org.apache.lucene.index.DirectoryReader org.apache.lucene.index.StandardDirectoryReader.doOpenFromWriter(org.apache.lucene.index.IndexCommit)"], "Stackframe.line.number": [381], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "t5uE-GFMsmesNeLBgx_z4wAAAAAAAAAD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassPath.java"], "Stackframe.function.name": ["java.net.URL jdk.internal.loader.URLClassPath$JarLoader.findResource(java.lang.String, boolean)"], "Stackframe.line.number": [942], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "YhErxbFc2y82dI3FdqHUOwAAAAAAAADs"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InternalEngine.java"], "Stackframe.function.name": ["boolean org.elasticsearch.index.engine.InternalEngine.flush(boolean, boolean)"], "Stackframe.line.number": [1990], "Stackframe.function.offset": [36]} +{"create": {"_index": "profiling-stackframes", "_id": "tgCBGeongyDKoP0eQjcOvQAAAAAAAAA3"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NioSocketImpl.java"], "Stackframe.function.name": ["int sun.nio.ch.NioSocketImpl.tryWrite(java.io.FileDescriptor, byte[], int, int)"], "Stackframe.line.number": [395], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "RCG4Qx5E9EZ0R-BBEbZnmwAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PrioritizedEsThreadPoolExecutor.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.runAndClean(java.lang.Runnable)"], "Stackframe.line.number": [257], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "v5Q60hHLUFC78bfvz1WyQQAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["GaloisCounterMode.java"], "Stackframe.function.name": ["void com.sun.crypto.provider.GaloisCounterMode.checkInit()"], "Stackframe.line.number": [321], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ipWtYiD0L8zHlsu-KRlKIwAAAAAAAAAU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService$$Lambda$7200+0x000000080244b7f0.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Avk7vHT4UprOTP5MohWAugAAAAAAAAAa"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndicesClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.indices.cluster.IndicesClusterStateService.failMissingShards(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [393], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "HoqCWZ471w3YYcVCfd06hAAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessController.java"], "Stackframe.function.name": ["void java.security.AccessController.checkPermission(java.security.Permission)"], "Stackframe.line.number": [1042], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "jPaG_bLbWLTHMVbsZ6KO3wAAAAAAAAPS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository.snapshotShard(org.elasticsearch.repositories.SnapshotShardContext)"], "Stackframe.line.number": [2796], "Stackframe.function.offset": [155]} +{"create": {"_index": "profiling-stackframes", "_id": "5Ga434JXyMCORvI3aSPl0wAAAAAAAAAx"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractPipeline.java"], "Stackframe.function.name": ["void java.util.stream.AbstractPipeline.copyInto(java.util.stream.Sink, java.util.Spliterator)"], "Stackframe.line.number": [513], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "Y8sJdN8JrOKeDGcKRLKzYgAAAAAAAANV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository.doSnapshotShard(org.elasticsearch.repositories.SnapshotShardContext)"], "Stackframe.line.number": [2839], "Stackframe.function.offset": [151]} +{"create": {"_index": "profiling-stackframes", "_id": "PUPItXL9ODmL0kwnalRigwAAAAAAAAAj"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClassLoader.java"], "Stackframe.function.name": ["java.net.URL java.lang.ClassLoader.getResource(java.lang.String)"], "Stackframe.line.number": [1404], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "LqytSQO0xoKueTdBUmzm9gAAAAAAAAAc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TieredMergePolicy.java"], "Stackframe.function.name": ["org.apache.lucene.index.MergePolicy$MergeSpecification org.apache.lucene.index.TieredMergePolicy.findMerges(org.apache.lucene.index.MergeTrigger, org.apache.lucene.index.SegmentInfos, org.apache.lucene.index.MergePolicy$MergeContext)"], "Stackframe.line.number": [329], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAARLUs"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["get_vfs_caps_from_disk"]} +{"create": {"_index": "profiling-stackframes", "_id": "-FiXU0afkg-sRwubj2WiAwAAAAAAAZtb"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../../../bus/dispatch.c", "../../../bus/dispatch.c"], "Stackframe.function.name": ["bus_dispatch_message_filter", "bus_dispatch"], "Stackframe.line.number": [548, 489]} +{"create": {"_index": "profiling-stackframes", "_id": "QzYQ9wb1jS08S2BufSVqWAAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractXContentParser.java"], "Stackframe.function.name": ["long org.elasticsearch.xcontent.support.AbstractXContentParser.longValue()"], "Stackframe.line.number": [194], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "qzb1PmzSJpYKqqQC4Es1dAAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Socket.java"], "Stackframe.function.name": ["void java.net.Socket$SocketOutputStream.write(byte[], int, int)"], "Stackframe.line.number": [1035], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "GUwipo03N4glyS6268EEmgAAAAAAAAAK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.read(java.lang.String, org.elasticsearch.common.blobstore.BlobContainer, java.lang.String, org.elasticsearch.xcontent.NamedXContentRegistry)"], "Stackframe.line.number": [120], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "H2XMIGvXG_73d5XzBvFSxAAAAAAAAAEB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportContext.java"], "Stackframe.function.name": ["void sun.security.ssl.TransportContext.dispatch(sun.security.ssl.Plaintext)"], "Stackframe.line.number": [201], "Stackframe.function.offset": [38]} +{"create": {"_index": "profiling-stackframes", "_id": "BjHQJvdwWPCTBjvnSeRZ2wAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NioSocketImpl.java"], "Stackframe.function.name": ["void sun.nio.ch.NioSocketImpl$2.write(byte[], int, int)"], "Stackframe.line.number": [823], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Boyt9QmW9-zzCwzKZ9LhFAAAAAAAAAEX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixChannelFactory.java"], "Stackframe.function.name": ["java.io.FileDescriptor sun.nio.fs.UnixChannelFactory.open(int, sun.nio.fs.UnixPath, java.lang.String, sun.nio.fs.UnixChannelFactory$Flags, int)"], "Stackframe.line.number": [248], "Stackframe.function.offset": [61]} +{"create": {"_index": "profiling-stackframes", "_id": "PUPItXL9ODmL0kwnalRigwAAAAAAAAAR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClassLoader.java"], "Stackframe.function.name": ["java.net.URL java.lang.ClassLoader.getResource(java.lang.String)"], "Stackframe.line.number": [1399], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "GQWHl0s0fmnGMrrNRd7yDQAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileChannelImpl.java"], "Stackframe.function.name": ["void sun.nio.ch.FileChannelImpl$Closer.run()"], "Stackframe.line.number": [115], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "fhsEKXDuxJ-jIJrZpdRuSAAAAAAAAFtj"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["./deflate.c", "./deflate.c"], "Stackframe.function.name": ["deflate", "deflate"], "Stackframe.line.number": [825, 1066]} +{"create": {"_index": "profiling-stackframes", "_id": "oi3QXV0PR2nGOV5pzW_MvAAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["JsonXContentGenerator.java"], "Stackframe.function.name": ["void org.elasticsearch.xcontent.provider.json.JsonXContentGenerator.writeStringField(java.lang.String, java.lang.String)"], "Stackframe.line.number": [352], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "8KbuOHlqa1mTqGljNuuiVAAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["MessageDigest.java"], "Stackframe.function.name": ["void java.security.MessageDigest.update(byte[])"], "Stackframe.line.number": [359], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAADAxg"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["kthread"]} +{"create": {"_index": "profiling-stackframes", "_id": "DTQx8l7VJrszvdqL-9PytgAAAAAAAAAR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.lambda$authorize$3(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, java.lang.String, org.elasticsearch.action.ActionListener, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo)"], "Stackframe.line.number": [314], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "P7mHtlyIKp9O861pj2g7OwAAAAAAAAAd"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractThrottledTaskRunner.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.AbstractThrottledTaskRunner$1.doRun()"], "Stackframe.line.number": [134], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "BPzs7KzDmaKPMROQHmdZzAAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PrioritizedThrottledTaskRunner.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.PrioritizedThrottledTaskRunner$TaskWrapper.onResponse(java.lang.Object)"], "Stackframe.line.number": [27], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "GwvNKsXwAWDPAp3O-Td6NAAAAAAAAAAX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ProtectionDomain.java"], "Stackframe.function.name": ["boolean java.security.ProtectionDomain.impliesWithAltFilePerm(java.security.Permission)"], "Stackframe.line.number": [357], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "Ho9Mi8DG7FUtIimhU2yBagAAAAAAAABq"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixFileSystem.java"], "Stackframe.function.name": ["java.nio.file.Path sun.nio.fs.UnixFileSystem.getPath(java.lang.String, java.lang.String[])"], "Stackframe.line.number": [279], "Stackframe.function.offset": [16]} +{"create": {"_index": "profiling-stackframes", "_id": "wnSBrxGSumHiAQQABJeNtQAAAAAAAAAk"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReplicationTracker.java"], "Stackframe.function.name": ["void org.elasticsearch.index.seqno.ReplicationTracker.updateReplicationGroupAndNotify()"], "Stackframe.line.number": [994], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALbjV"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["vfs_statx"]} +{"create": {"_index": "profiling-stackframes", "_id": "AxJNlcXBvrgVOpR8a1WEYgAAAAAAAAG2"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["MainClientExec.java"], "Stackframe.function.name": ["org.apache.http.client.methods.CloseableHttpResponse org.apache.http.impl.execchain.MainClientExec.execute(org.apache.http.conn.routing.HttpRoute, org.apache.http.client.methods.HttpRequestWrapper, org.apache.http.client.protocol.HttpClientContext, org.apache.http.client.methods.HttpExecutionAware)"], "Stackframe.line.number": [236], "Stackframe.function.offset": [81]} +{"create": {"_index": "profiling-stackframes", "_id": "nrTqwGPzO0uzQD3C7T948wAAAAAAAAAP"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["java.lang.String com.fasterxml.jackson.dataformat.smile.SmileParser._decodeShortAsciiValue(int)"], "Stackframe.line.number": [2223], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "s-5AzRQS0AyjARcxs2D2DQAAAAAAAAAX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DirectMethodHandleAccessor.java"], "Stackframe.function.name": ["java.lang.Object jdk.internal.reflect.DirectMethodHandleAccessor.invoke(java.lang.Object, java.lang.Object[])"], "Stackframe.line.number": [104], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "Fuiq0RN8VMfwSLHsA6DzhAAAAAAAATq8"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../../../dbus/dbus-connection.c"], "Stackframe.function.name": ["dbus_connection_dispatch"], "Stackframe.line.number": [4703]} +{"create": {"_index": "profiling-stackframes", "_id": "XrSW2Ro9Ws10jZmj20O0hQAAAAAAAAAD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PublicationTransportHandler.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.coordination.PublicationTransportHandler.lambda$new$0(org.elasticsearch.transport.BytesTransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.tasks.Task)"], "Stackframe.line.number": [111], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "6KK4nudrIWi-u7NWSV5S7QAAAAAAAABn"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthenticatorChain.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authc.AuthenticatorChain.authenticateAsync(org.elasticsearch.xpack.security.authc.Authenticator$Context, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [102], "Stackframe.function.offset": [27]} +{"create": {"_index": "profiling-stackframes", "_id": "B0jzxgjaMAT2f_-sp0qKdQAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SnapshotShardsService.java"], "Stackframe.function.name": ["java.lang.String org.elasticsearch.snapshots.SnapshotShardsService.getShardStateId(org.elasticsearch.index.shard.IndexShard, org.apache.lucene.index.IndexCommit)"], "Stackframe.line.number": [409], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "BJKRobhikTEVTM_TJ5G21wAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Files.java"], "Stackframe.function.name": ["void java.nio.file.Files.delete(java.nio.file.Path)"], "Stackframe.line.number": [1152], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "_OOeQuiXtvgynmE43QMbEAAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileDispatcherImpl.java"], "Stackframe.function.name": ["int sun.nio.ch.FileDispatcherImpl.force0(java.io.FileDescriptor, boolean)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "UJLc0zscqzwPwar7clkaagAAAAAAAAAP"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NioSocketImpl.java"], "Stackframe.function.name": ["int sun.nio.ch.NioSocketImpl.implWrite(byte[], int, int)"], "Stackframe.line.number": [410], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAC6Xu"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["process_one_work"]} +{"create": {"_index": "profiling-stackframes", "_id": "w0BKBBEkfs3oopm7NSQhfQAAAAAAAADM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ConjunctionDISI.java"], "Stackframe.function.name": ["org.apache.lucene.search.DocIdSetIterator org.apache.lucene.search.ConjunctionDISI.createConjunction(java.util.List, java.util.List)"], "Stackframe.line.number": [119], "Stackframe.function.offset": [20]} +{"create": {"_index": "profiling-stackframes", "_id": "e8ARrqT9JmZcuLd6G8ZTMgAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SdkHttpClient.java"], "Stackframe.function.name": ["org.apache.http.HttpResponse com.amazonaws.http.apache.client.impl.SdkHttpClient.execute(org.apache.http.client.methods.HttpUriRequest, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [72], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "YKWsiTTdS4chcntYalLpvQAAAAAAAAFD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessControlContext.java"], "Stackframe.function.name": ["void java.security.AccessControlContext.checkPermission(java.security.Permission)"], "Stackframe.line.number": [463], "Stackframe.function.offset": [56]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAF0JU"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__audit_inode"]} +{"create": {"_index": "profiling-stackframes", "_id": "ooplCYR951bf9t1YS6OUBQAAAAAAAACr"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CoordinationState.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishResponse org.elasticsearch.cluster.coordination.CoordinationState.handlePublishRequest(org.elasticsearch.cluster.coordination.PublishRequest)"], "Stackframe.line.number": [392], "Stackframe.function.offset": [27]} +{"create": {"_index": "profiling-stackframes", "_id": "xBi4pQ8utWGRmJVqaVBkxwAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["RepeatableInputStreamRequestEntity.java"], "Stackframe.function.name": ["void com.amazonaws.http.RepeatableInputStreamRequestEntity.writeTo(java.io.OutputStream)"], "Stackframe.line.number": [160], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "_eHurMk1AuWI20vPBelYUQAAAAAAAABR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DeflaterOutputStream.java"], "Stackframe.function.name": ["void java.util.zip.DeflaterOutputStream.write(byte[], int, int)"], "Stackframe.line.number": [210], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "epNUCeEe4JglhIlXIVE76AAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SnapshotShardsService.java"], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService.lambda$startNewSnapshots$1(java.util.List)"], "Stackframe.line.number": [244], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALW22"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["ksys_write"]} +{"create": {"_index": "profiling-stackframes", "_id": "gi_774C-EJhuJfyXXhzVgwAAAAAAAABE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.callClusterStateAppliers(org.elasticsearch.cluster.ClusterChangedEvent, org.elasticsearch.cluster.service.ClusterApplierRecordingService$Recorder, java.util.Collection)"], "Stackframe.line.number": [553], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAMRwH"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["vfs_fsync_range"]} +{"create": {"_index": "profiling-stackframes", "_id": "DT06D5A7Yxiqx6IJ7Kvu0wAAAAAAAABI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECKeyPairGenerator.java"], "Stackframe.function.name": ["java.util.Optional sun.security.ec.ECKeyPairGenerator.generateKeyPairImpl(java.security.SecureRandom)"], "Stackframe.line.number": [194], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "hEBKqh2xU-RorWUVO4ewtQAAAAAAAADt"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Netty4WriteThrottlingHandler.java"], "Stackframe.function.name": ["boolean org.elasticsearch.transport.netty4.Netty4WriteThrottlingHandler.doFlush(io.netty.channel.ChannelHandlerContext)"], "Stackframe.line.number": [146], "Stackframe.function.offset": [33]} +{"create": {"_index": "profiling-stackframes", "_id": "3-s2ANt1vNwpWiAFGYs2eAAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$3.onResponse(java.lang.Object)"], "Stackframe.line.number": [394], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "SWMbBt07f_PdUThaE5iWmwAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityManager.java"], "Stackframe.function.name": ["void java.lang.SecurityManager.checkPermission(java.security.Permission)"], "Stackframe.line.number": [411], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "eL-oIR1CC2R0ykcUAAfoowAAAAAAAABT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Lucene90CompressingStoredFieldsWriter.java"], "Stackframe.function.name": ["void org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsWriter.finish(int)"], "Stackframe.line.number": [486], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "vQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["/build/glibc-SzIz7B/glibc-2.31/nptl/pthread_create.c"], "Stackframe.function.name": ["start_thread"], "Stackframe.line.number": [477]} +{"create": {"_index": "profiling-stackframes", "_id": "92JDYcQA4cXvpv7axYYOPwAAAAAAAAAo"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["XContentParserUtils.java"], "Stackframe.function.name": ["java.util.List org.elasticsearch.common.xcontent.XContentParserUtils.parseList(org.elasticsearch.xcontent.XContentParser, org.elasticsearch.core.CheckedFunction)"], "Stackframe.line.number": [177], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "SlqES-nwtjqDlYXOe7K1YAAAAAAAAAAJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SocketAccess.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.SocketAccess.doPrivilegedVoid(java.lang.Runnable)"], "Stackframe.line.number": [45], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "y6xtuPPjpVFeK4CXqSN6fQAAAAAAAAHc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Coordinator.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishWithJoinResponse org.elasticsearch.cluster.coordination.Coordinator.handlePublishRequest(org.elasticsearch.cluster.coordination.PublishRequest)"], "Stackframe.line.number": [437], "Stackframe.function.offset": [51]} +{"create": {"_index": "profiling-stackframes", "_id": "S5_Ae_ABcHEi_tFSU_YdMgAAAAAAAAC9"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ServerTransportFilter.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.ServerTransportFilter.inbound(java.lang.String, org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [104], "Stackframe.function.offset": [33]} +{"create": {"_index": "profiling-stackframes", "_id": "LSgILL0Q8I6llgRQX49QDQAAAAAAAADH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler.messageReceived(org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.tasks.Task)"], "Stackframe.line.number": [415], "Stackframe.function.offset": [48]} +{"create": {"_index": "profiling-stackframes", "_id": "B8SjIPdc3ZwlcMQD5LrUqAAAAAAAAACO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SslHandler.java"], "Stackframe.function.name": ["javax.net.ssl.SSLEngineResult io.netty.handler.ssl.SslHandler.wrap(io.netty.buffer.ByteBufAllocator, javax.net.ssl.SSLEngine, io.netty.buffer.ByteBuf, io.netty.buffer.ByteBuf)"], "Stackframe.line.number": [1041], "Stackframe.function.offset": [33]} +{"create": {"_index": "profiling-stackframes", "_id": "Z7ZAUH4Eu0mE1Lr-7VKuiwAAAAAAAACg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexShardOperationPermits.java"], "Stackframe.function.name": ["void org.elasticsearch.index.shard.IndexShardOperationPermits.acquire(org.elasticsearch.action.ActionListener, java.lang.String, boolean, java.lang.Object, java.lang.StackTraceElement[])"], "Stackframe.line.number": [275], "Stackframe.function.offset": [44]} +{"create": {"_index": "profiling-stackframes", "_id": "UnU61_VzRQulrS0OJYkU7gAAAAAAAABa"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PersistedClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.PersistedClusterStateService$Writer.writeIncrementalStateAndCommit(long, org.elasticsearch.cluster.ClusterState, org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [912], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "Fq10nEfKWtXEt510UwEUUAAAAAAAAAB7"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PendingReplicationActions.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.PendingReplicationActions.acceptNewTrackedAllocationIds(java.util.Set)"], "Stackframe.line.number": [95], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAFEre"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["do_futex"]} +{"create": {"_index": "profiling-stackframes", "_id": "NeO57Qkcge62xvmelDstIAAAAAAAAAAo"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository$$Lambda$9113+0x00000008020df600.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "nVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadContext.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run()"], "Stackframe.line.number": [716], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "rg1U8-_Qg7P4zN0uZwHMZAAAAAAAAAAh"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SoftDeletesRetentionMergePolicy.java"], "Stackframe.function.name": ["org.apache.lucene.search.Scorer org.apache.lucene.index.SoftDeletesRetentionMergePolicy.getScorer(org.apache.lucene.search.Query, org.apache.lucene.index.CodecReader)"], "Stackframe.line.number": [149], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "hJgyhaKZ9eSJym5BMY7B2wAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$1.doRun()"], "Stackframe.line.number": [527], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "E0bQU1G2uDlA0HZGujxA9wAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SocketAccess.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.s3.SocketAccess.doPrivilegedIOException(java.security.PrivilegedExceptionAction)"], "Stackframe.line.number": [37], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "GUwipo03N4glyS6268EEmgAAAAAAAAAX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.read(java.lang.String, org.elasticsearch.common.blobstore.BlobContainer, java.lang.String, org.elasticsearch.xcontent.NamedXContentRegistry)"], "Stackframe.line.number": [121], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "6a5XWhjsuz26DnHoF0x5RwAAAAAAAABS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["org.elasticsearch.xcontent.ToXContent org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.deserialize(java.lang.String, org.elasticsearch.xcontent.NamedXContentRegistry, java.io.InputStream)"], "Stackframe.line.number": [112], "Stackframe.function.offset": [14]} +{"create": {"_index": "profiling-stackframes", "_id": "XjeOrvBx98FSmUHFqJE5JAAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["OutputStreamIndexOutput.java"], "Stackframe.function.name": ["void org.apache.lucene.store.OutputStreamIndexOutput.writeByte(byte)"], "Stackframe.line.number": [53], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "YxIrzYSolBsbC9VHbJjZmgAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["java.lang.Object com.amazonaws.services.s3.AmazonS3Client.invoke(com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, java.lang.String, java.lang.String)"], "Stackframe.line.number": [5000], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "FTD_DMq9LCr2rMh4XAWQegAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessController.java"], "Stackframe.function.name": ["java.lang.Object java.security.AccessController.doPrivileged(java.security.PrivilegedAction)"], "Stackframe.line.number": [318], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "YgOo_ePEdS6xTt4Sk9uAVQAAAAAAAABp"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.maybeAuthorizeRunAs(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, java.lang.String, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [413], "Stackframe.function.offset": [20]} +{"create": {"_index": "profiling-stackframes", "_id": "9UBMSNV9JIwVE7bRWf7rNAAAAAAAAACI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileParser._loadToHaveAtLeast(int)"], "Stackframe.line.number": [289], "Stackframe.function.offset": [15]} +{"create": {"_index": "profiling-stackframes", "_id": "KOnuKkBYBgm4HtfyhrUS2wAAAAAAAAC7"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ESPolicy.java"], "Stackframe.function.name": ["boolean org.elasticsearch.bootstrap.ESPolicy.implies(java.security.ProtectionDomain, java.security.Permission)"], "Stackframe.line.number": [122], "Stackframe.function.offset": [36]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALkCo"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["walk_component"]} +{"create": {"_index": "profiling-stackframes", "_id": "hYgsx8HjLzhm9LYVIHuquwAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InternalEngine.java"], "Stackframe.function.name": ["org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader org.elasticsearch.index.engine.InternalEngine$ExternalReaderManager.refreshIfNeeded(org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader)"], "Stackframe.line.number": [387], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAlcA7"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["tcp_sendmsg"]} +{"create": {"_index": "profiling-stackframes", "_id": "vQ7EacG6CR4rzcFaNFEsTQAAAAAAATk7"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["", "", "../sysdeps/unix/sysv/linux/fsync.c"], "Stackframe.function.name": ["fsync", "fsync", "fsync"], "Stackframe.line.number": [0, 0, 27]} +{"create": {"_index": "profiling-stackframes", "_id": "0iStdyhoG4WqQkJ1aFKZnQAAAAAAAAA1"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TieredMergePolicy.java"], "Stackframe.function.name": ["java.util.List org.apache.lucene.index.TieredMergePolicy.getSortedBySegmentSize(org.apache.lucene.index.SegmentInfos, org.apache.lucene.index.MergePolicy$MergeContext)"], "Stackframe.line.number": [300], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "IWt15Ku8-XXZGHssEyOb_gAAAAAAAAAw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["RoleReferenceIntersection.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.core.security.authz.store.RoleReferenceIntersection.buildRole(java.util.function.BiConsumer, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [53], "Stackframe.function.offset": [13]} +{"create": {"_index": "profiling-stackframes", "_id": "MH4ZLlkHvkfZTK_KWydMDgAAAAAAAAC4"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository.snapshotShard(org.elasticsearch.repositories.SnapshotShardContext)"], "Stackframe.line.number": [2666], "Stackframe.function.offset": [25]} +{"create": {"_index": "profiling-stackframes", "_id": "qnvWBP24iZLcQ-Wi76ZDxQAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LockSupport.java"], "Stackframe.function.name": ["void java.util.concurrent.locks.LockSupport.unpark(java.lang.Thread)"], "Stackframe.line.number": [177], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "rDli4oCQ8GWnljRH2_1bLAAAAAAAAAEZ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLTransport.java"], "Stackframe.function.name": ["sun.security.ssl.Plaintext sun.security.ssl.SSLTransport.decode(sun.security.ssl.TransportContext, java.nio.ByteBuffer[], int, int, java.nio.ByteBuffer[], int, int)"], "Stackframe.line.number": [172], "Stackframe.function.offset": [62]} +{"create": {"_index": "profiling-stackframes", "_id": "tqJQIOovDd8UazKau6rQMwAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractChannelHandlerContext.java"], "Stackframe.function.name": ["void io.netty.channel.AbstractChannelHandlerContext.invokeFlush()"], "Stackframe.line.number": [907], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "N4c5sJszjyVzcx3AmWN8pwAAAAAAAADS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexShard.java"], "Stackframe.function.name": ["void org.elasticsearch.index.shard.IndexShard.updateShardState(org.elasticsearch.cluster.routing.ShardRouting, long, java.util.function.BiConsumer, long, java.util.Set, org.elasticsearch.cluster.routing.IndexShardRoutingTable)"], "Stackframe.line.number": [535], "Stackframe.function.offset": [24]} +{"create": {"_index": "profiling-stackframes", "_id": "3r1MnCyPIgP6mYXLoLcz9AAAAAAAAAAK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.cluster.coordination.PublicationTransportHandler$$Lambda$4630+0x000000080196cf18.messageReceived(org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.tasks.Task)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Kz3aUSMwqIxIxaAplngd8AAAAAAAAACX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndicesClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.indices.cluster.IndicesClusterStateService.updateShard(org.elasticsearch.cluster.node.DiscoveryNodes, org.elasticsearch.cluster.routing.ShardRouting, org.elasticsearch.indices.cluster.IndicesClusterStateService$Shard, org.elasticsearch.cluster.routing.RoutingTable, org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [614], "Stackframe.function.offset": [14]} +{"create": {"_index": "profiling-stackframes", "_id": "Csbq98qEMjLfznv89zxcQwAAAAAAAABA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Inflater.java"], "Stackframe.function.name": ["int java.util.zip.Inflater.inflate(byte[], int, int)"], "Stackframe.line.number": [373], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "d8JoiANFYdN_HlsRPvshMwAAAAAAAAAx"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PublicationTransportHandler.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishWithJoinResponse org.elasticsearch.cluster.coordination.PublicationTransportHandler$PublicationContext$1.get()"], "Stackframe.line.number": [388], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "zgecUfJLYoiYIxBtYNFbZwAAAAAAAAAo"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["XContentParserUtils.java"], "Stackframe.function.name": ["java.util.List org.elasticsearch.common.xcontent.XContentParserUtils.parseList(org.elasticsearch.xcontent.XContentParser, org.elasticsearch.core.CheckedFunction)"], "Stackframe.line.number": [178], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "Ao_nvps-se9i_u9kVJi0LQAAAAAAAAAK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.cluster.coordination.PublicationTransportHandler$$Lambda$4544+0x0000000801832cf8.messageReceived(org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.tasks.Task)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "BtfAnw52Zh-pzF1hN6odGgAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractRunnable.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.AbstractRunnable.run()"], "Stackframe.line.number": [26], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "zV-LQYJm41zGMkfXUp2AHgAAAAAAAAF_"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshot.java"], "Stackframe.function.name": ["org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo.fromXContent(org.elasticsearch.xcontent.XContentParser)"], "Stackframe.line.number": [312], "Stackframe.function.offset": [21]} +{"create": {"_index": "profiling-stackframes", "_id": "I9AUeiUn_SK_Dgdkbb3mdAAAAAAAAAAS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedInputStream.java"], "Stackframe.function.name": ["int java.io.BufferedInputStream.read(byte[], int, int)"], "Stackframe.line.number": [361], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "8iRqFHNeeA-Vq5mM-jaESgAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassPath.java"], "Stackframe.function.name": ["java.net.URL jdk.internal.loader.URLClassPath.findResource(java.lang.String, boolean)"], "Stackframe.line.number": [298], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "AFikCbtP_Dm7iUthjnlnEgAAAAAAAAEq"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.applyChanges(org.elasticsearch.cluster.ClusterState, org.elasticsearch.cluster.ClusterState, java.lang.String, org.elasticsearch.cluster.service.ClusterApplierRecordingService$Recorder)"], "Stackframe.line.number": [503], "Stackframe.function.offset": [25]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAUd51"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["_copy_from_iter_full"]} +{"create": {"_index": "profiling-stackframes", "_id": "4w-7ZRIHk2tM9RfD6v6f4gAAAAAAAAAV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["boolean com.fasterxml.jackson.dataformat.smile.SmileParser._loadMore()"], "Stackframe.line.number": [278], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAFz87"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__audit_syscall_exit"]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAlbpN"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["tcp_sendmsg_locked"]} +{"create": {"_index": "profiling-stackframes", "_id": "ZGKy7ArdiDfRv4uogAG0CQAAAAAAAAAz"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["org.elasticsearch.core.Tuple org.elasticsearch.repositories.blobstore.BlobStoreRepository.buildBlobStoreIndexShardSnapshots(java.util.Set, org.elasticsearch.common.blobstore.BlobContainer, org.elasticsearch.repositories.ShardGeneration)"], "Stackframe.line.number": [3416], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAmOK4"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["inet_sendmsg"]} +{"create": {"_index": "profiling-stackframes", "_id": "VWOUd0jMGfSYTm8Ns8_hDgAAAAAAAAME"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["long org.apache.lucene.index.IndexWriter.prepareCommitInternal()"], "Stackframe.line.number": [3460], "Stackframe.function.offset": [130]} +{"create": {"_index": "profiling-stackframes", "_id": "2Swt2qN1JJo2lNqpZIN9-wAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileDispatcherImpl.java"], "Stackframe.function.name": ["int sun.nio.ch.FileDispatcherImpl.write0(java.io.FileDescriptor, long, int)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "-FiXU0afkg-sRwubj2WiAwAAAAAAAla0"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../../../dbus/dbus-mainloop.c"], "Stackframe.function.name": ["_dbus_loop_dispatch"], "Stackframe.line.number": [532]} +{"create": {"_index": "profiling-stackframes", "_id": "oKMpPO_rrguOLYx8VBTGhgAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NamedGroup.java"], "Stackframe.function.name": ["sun.security.ssl.SSLPossession sun.security.ssl.NamedGroup.createPossession(java.security.SecureRandom)"], "Stackframe.line.number": [428], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "qND66z7potQhxSJ1H4ezHgAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NamedGroup.java"], "Stackframe.function.name": ["sun.security.ssl.SSLPossession sun.security.ssl.NamedGroup$ECDHEScheme.createPossession(sun.security.ssl.NamedGroup, java.security.SecureRandom)"], "Stackframe.line.number": [584], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "HJWDExCp8dmw9-CAF-WXfwAAAAAAAAAJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PersistedClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.PersistedClusterStateService$Writer.commit(long, long, org.elasticsearch.Version)"], "Stackframe.line.number": [1151], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "9ccfl0YdabzwSUY2PdYA0QAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECKeyPairGenerator.java"], "Stackframe.function.name": ["java.security.KeyPair sun.security.ec.ECKeyPairGenerator.generateKeyPair()"], "Stackframe.line.number": [149], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "ayyYx9DIcjy4NGNqpq88GQAAAAAAAAAP"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Store.java"], "Stackframe.function.name": ["void org.elasticsearch.index.store.Store$StoreDirectory.deleteFile(java.lang.String, java.lang.String)"], "Stackframe.line.number": [746], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "wEIfl0pGBuepuUcTwdKc2gAAAAAAAEJ_"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["./deflate.c"], "Stackframe.function.name": ["deflate_fast"], "Stackframe.line.number": [1902]} +{"create": {"_index": "profiling-stackframes", "_id": "pKAwMR-9-3afO3tccFzRKwAAAAAAAAIX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECOperations.java"], "Stackframe.function.name": ["void sun.security.ec.ECOperations.setDouble(sun.security.ec.point.ProjectivePoint$Mutable, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP)"], "Stackframe.line.number": [338], "Stackframe.function.offset": [43]} +{"create": {"_index": "profiling-stackframes", "_id": "JwuJ_YrJlPmIg66vmsQiFAAAAAAAAAAz"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthenticationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authc.AuthenticationService.authenticate(java.lang.String, org.elasticsearch.transport.TransportRequest, boolean, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [199], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "mZJt6uLSgudylMZ0eHd7CAAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.s3.SocketAccess$$Lambda$8688+0x00000008020835a0.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "wRG5B2klLAxXtt0B9lVulAAAAAAAAAB-"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.serialize(java.lang.Object, java.lang.String, boolean, java.util.Map, java.io.OutputStream)"], "Stackframe.line.number": [386], "Stackframe.function.offset": [26]} +{"create": {"_index": "profiling-stackframes", "_id": "bnYc3RJmtKwpcpofJJsHfwAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ActionListener.java"], "Stackframe.function.name": ["void org.elasticsearch.action.ActionListener.run(org.elasticsearch.action.ActionListener, org.elasticsearch.core.CheckedConsumer)"], "Stackframe.line.number": [567], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "QTXwwuqN0mLb-8lqckiaTQAAAAAAB4Dj"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["xfs_ilock_attr_map_shared"]} +{"create": {"_index": "profiling-stackframes", "_id": "0GU5IksNSYIYre39EnRVtQAAAAAAAAAc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractXContentParser.java"], "Stackframe.function.name": ["long org.elasticsearch.xcontent.support.AbstractXContentParser.longValue(boolean)"], "Stackframe.line.number": [204], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "xmj6c8GMqxu-eoFnl7D4qwAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ArrayList.java"], "Stackframe.function.name": ["boolean java.util.ArrayList.add(java.lang.Object)"], "Stackframe.line.number": [466], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "-FiXU0afkg-sRwubj2WiAwAAAAAAAkK3"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../../../bus/signals.c", "../../../bus/signals.c"], "Stackframe.function.name": ["bus_matchmaker_get_recipients", "get_recipients_from_list"], "Stackframe.line.number": [2167, 2073]} +{"create": {"_index": "profiling-stackframes", "_id": "HSftxLHu5xLDzJX5F8kBXAAAAAAAAAGV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PublicationTransportHandler.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishWithJoinResponse org.elasticsearch.cluster.coordination.PublicationTransportHandler.handleIncomingPublishRequest(org.elasticsearch.transport.BytesTransportRequest)"], "Stackframe.line.number": [225], "Stackframe.function.offset": [52]} +{"create": {"_index": "profiling-stackframes", "_id": "RoDaQXQoj9oJAPcF2bxG2gAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.xpack.core.security.authz.store.RoleReferenceIntersection$$Lambda$7790+0x0000000802546600.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "SLSuJBznTE3BvHPDUjF7JgAAAAAAAABq"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SoftDeletesRetentionMergePolicy.java"], "Stackframe.function.name": ["int org.apache.lucene.index.SoftDeletesRetentionMergePolicy.numDeletesToMerge(org.apache.lucene.index.SegmentCommitInfo, int, org.apache.lucene.util.IOSupplier)"], "Stackframe.line.number": [164], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "ds9yTvk8UzKAgDSjs6pNkgAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3BlobContainer.java"], "Stackframe.function.name": ["java.io.InputStream org.elasticsearch.repositories.s3.S3BlobContainer.readBlob(java.lang.String)"], "Stackframe.line.number": [95], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Uu1pGUpLQ35aan6Rej1VEwAAAAAAAADJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ServerTransportFilter.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.ServerTransportFilter$NodeProfile.inbound(java.lang.String, org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [128], "Stackframe.function.offset": [33]} +{"create": {"_index": "profiling-stackframes", "_id": "d0aX3KonLcPjMEO-H4cl5QAAAAAAAACD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["GatewayMetaState.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.GatewayMetaState$LucenePersistedState.setLastAcceptedState(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [593], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "7TfJIA5ukUl_URVlhByP5QAAAAAAAAAu"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ArrayList.java"], "Stackframe.function.name": ["void java.util.ArrayList.forEach(java.util.function.Consumer)"], "Stackframe.line.number": [1511], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "pKAwMR-9-3afO3tccFzRKwAAAAAAAAEn"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECOperations.java"], "Stackframe.function.name": ["void sun.security.ec.ECOperations.setDouble(sun.security.ec.point.ProjectivePoint$Mutable, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP)"], "Stackframe.line.number": [316], "Stackframe.function.offset": [21]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALkVr"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["link_path_walk.part.33"]} +{"create": {"_index": "profiling-stackframes", "_id": "86ShTOKEhMBNNZND3RBl5QAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PublicationTransportHandler.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.cluster.coordination.PublicationTransportHandler$PublicationContext$1.get()"], "Stackframe.line.number": [377], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "fhsEKXDuxJ-jIJrZpdRuSAAAAAAAAKGd"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["./inflate.c"], "Stackframe.function.name": ["inflate"], "Stackframe.line.number": [1076]} +{"create": {"_index": "profiling-stackframes", "_id": "BULXlXluMevM23wGn-scGgAAAAAAAABI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient.execute(com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, com.amazonaws.http.HttpResponseHandler, com.amazonaws.http.ExecutionContext, com.amazonaws.RequestConfig)"], "Stackframe.line.number": [561], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "wEIfl0pGBuepuUcTwdKc2gAAAAAAAFtj"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["./deflate.c", "./deflate.c"], "Stackframe.function.name": ["deflate", "deflate"], "Stackframe.line.number": [825, 1066]} +{"create": {"_index": "profiling-stackframes", "_id": "1QVyanYcPpNfuP3xZ0ogLQAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CPoolProxy.java"], "Stackframe.function.name": ["void org.apache.http.impl.conn.CPoolProxy.sendRequestEntity(org.apache.http.HttpEntityEnclosingRequest)"], "Stackframe.line.number": [152], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "7MyQRgXoynWLAIL3MMBcgwAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService$$Lambda$10694+0x0000000802972d18.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ie51PKalSPz0oTw25hOjdgAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["java.lang.Object com.amazonaws.services.s3.AmazonS3Client.invoke(com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, java.lang.String, java.lang.String)"], "Stackframe.line.number": [5403], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "_B6c9784WlLvFF9uk3pSswAAAAAAAAAc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeWithTimer()"], "Stackframe.line.number": [781], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "GUEIf0h4CmbF3WXaliniYgAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadExecutorMap.java"], "Stackframe.function.name": ["void io.netty.util.internal.ThreadExecutorMap$2.run()"], "Stackframe.line.number": [74], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "GV9odb7eqAONOiRGAnVZsQAAAAAAAAFY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DocumentsWriterPerThread.java"], "Stackframe.function.name": ["org.apache.lucene.index.DocumentsWriterPerThread$FlushedSegment org.apache.lucene.index.DocumentsWriterPerThread.flush(org.apache.lucene.index.DocumentsWriter$FlushNotifications)"], "Stackframe.line.number": [392], "Stackframe.function.offset": [49]} +{"create": {"_index": "profiling-stackframes", "_id": "wEIfl0pGBuepuUcTwdKc2gAAAAAAAJF2"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["", "", "./inffast.c"], "Stackframe.function.name": ["inflate_fast", "inflate_fast", "inflate_fast"], "Stackframe.line.number": [0, 0, 171]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALb_l"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__x64_sys_newlstat"]} +{"create": {"_index": "profiling-stackframes", "_id": "n0mWG92K0zbUz5S-_ixlIAAAAAAAAABr"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["MainClientExec.java"], "Stackframe.function.name": ["void org.apache.http.impl.execchain.MainClientExec.establishRoute(org.apache.http.auth.AuthState, org.apache.http.HttpClientConnection, org.apache.http.conn.routing.HttpRoute, org.apache.http.HttpRequest, org.apache.http.client.protocol.HttpClientContext)"], "Stackframe.line.number": [393], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALTsY"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__vfs_write"]} +{"create": {"_index": "profiling-stackframes", "_id": "0Ju-cCCi3kHxLcKkTtnYSAAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixPath.java"], "Stackframe.function.name": ["void sun.nio.fs.UnixPath.checkNotNul(java.lang.String, char)"], "Stackframe.line.number": [89], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "etyLdtogTiVt3WXDEehDtAAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportShardRefreshAction.java"], "Stackframe.function.name": ["void org.elasticsearch.action.admin.indices.refresh.TransportShardRefreshAction.shardOperationOnPrimary(org.elasticsearch.action.support.replication.ReplicationRequest, org.elasticsearch.index.shard.IndexShard, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [41], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "dWyxTJ01gHAXEF1z9sDOKwAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$3.onResponse(java.lang.Void)"], "Stackframe.line.number": [404], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "BASYhiEtRRk5wvHC9yWtCgAAAAAAAABV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PublicationTransportHandler.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishWithJoinResponse org.elasticsearch.cluster.coordination.PublicationTransportHandler.acceptState(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [242], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "AE1Q7tv3S3rfQh4l5ABt6AAAAAAAAABn"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SslHandler.java"], "Stackframe.function.name": ["void io.netty.handler.ssl.SslHandler.wrap(io.netty.channel.ChannelHandlerContext, boolean)"], "Stackframe.line.number": [828], "Stackframe.function.offset": [19]} +{"create": {"_index": "profiling-stackframes", "_id": "p_sL8jeyDjPZI0awM1-lMQAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileDescriptor.java"], "Stackframe.function.name": ["void java.io.FileDescriptor$1.close(java.io.FileDescriptor)"], "Stackframe.line.number": [88], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "DnrzOZkhcnoTNvvkP1TdqAAAAAAAAAAi"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassPath.java"], "Stackframe.function.name": ["jdk.internal.loader.Resource jdk.internal.loader.URLClassPath$JarLoader.getResource(java.lang.String, boolean)"], "Stackframe.line.number": [961], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "6QvV8Ag8LiqGHdRYVptEnwAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CleanerImpl.java"], "Stackframe.function.name": ["void jdk.internal.ref.CleanerImpl$PhantomCleanableRef.performCleanup()"], "Stackframe.line.number": [178], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "SNDYMaokxZpNhjFZnC0cqQAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(com.amazonaws.http.AmazonHttpClient$RequestExecutor)"], "Stackframe.line.number": [715], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAFzaR"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["audit_filter_syscall.constprop.19"]} +{"create": {"_index": "profiling-stackframes", "_id": "ySBUSdLpTE4kxptOyYACMwAAAAAAAACH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DocumentsWriter.java"], "Stackframe.function.name": ["boolean org.apache.lucene.index.DocumentsWriter.doFlush(org.apache.lucene.index.DocumentsWriterPerThread)"], "Stackframe.line.number": [493], "Stackframe.function.offset": [37]} +{"create": {"_index": "profiling-stackframes", "_id": "HH3S5b5wm0-iwMj6LUThxAAAAAAAAAAy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3BlobContainer.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3BlobContainer.writeBlob(java.lang.String, boolean, boolean, org.elasticsearch.core.CheckedConsumer)"], "Stackframe.line.number": [211], "Stackframe.function.offset": [72]} +{"create": {"_index": "profiling-stackframes", "_id": "gvYbo0YBmE65VwrpTWYalQAAAAAAAAB2"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndicesClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.indices.cluster.IndicesClusterStateService.applyClusterState(org.elasticsearch.cluster.ClusterChangedEvent)"], "Stackframe.line.number": [231], "Stackframe.function.offset": [31]} +{"create": {"_index": "profiling-stackframes", "_id": "mYNktaRFRY0plnRBTkItUgAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["StandardDirectoryReader.java"], "Stackframe.function.name": ["org.apache.lucene.index.DirectoryReader org.apache.lucene.index.StandardDirectoryReader.doOpenIfChanged(org.apache.lucene.index.IndexCommit)"], "Stackframe.line.number": [355], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "fo33a47SQa_ikvx14Wo6GwAAAAAAAAAx"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractChannelHandlerContext.java"], "Stackframe.function.name": ["void io.netty.channel.AbstractChannelHandlerContext.invokeFlush0()"], "Stackframe.line.number": [925], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "nGPKMkILxfXIjt9GRbMXwAAAAAAAAABY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractPipeline.java"], "Stackframe.function.name": ["java.lang.Object java.util.stream.AbstractPipeline.evaluate(java.util.stream.TerminalOp)"], "Stackframe.line.number": [234], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "V2OKkEZQbgFjMzVwaTkytwAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService$UpdateTask.run()"], "Stackframe.line.number": [154], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "2i9E7A7hICqdU2K3ciAqMgAAAAAAAAFL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndicesAliasesRequestInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.IndicesAliasesRequestInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [124], "Stackframe.function.offset": [65]} +{"create": {"_index": "profiling-stackframes", "_id": "WE9wZHWs5kC-tWUiTpjMyQAAAAAAAAA2"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileParser._finishNumberToken(int)"], "Stackframe.line.number": [2058], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "HTWghk6sgAJ14g_dZIR1agAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LockValidatingDirectoryWrapper.java"], "Stackframe.function.name": ["void org.apache.lucene.store.LockValidatingDirectoryWrapper.deleteFile(java.lang.String)"], "Stackframe.line.number": [37], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "wRQstrBYZ0ShmJnYV-ADrQAAAAAAAAFL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.runTask(java.lang.String, java.util.function.Function, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [428], "Stackframe.function.offset": [44]} +{"create": {"_index": "profiling-stackframes", "_id": "KTge2icH_VE2kyqCeDCWXgAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService$AuthorizationResultListener.onResponse(java.lang.Object)"], "Stackframe.line.number": [952], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "agUnXAr2_hsl_qyYsSOrqQAAAAAAAABc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexTemplateRegistry.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.core.template.IndexTemplateRegistry.addComposableTemplatesIfMissing(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [294], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "kxggmTmW48eX63B51-tv4wAAAAAAAAEE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexingChain.java"], "Stackframe.function.name": ["org.apache.lucene.index.Sorter$DocMap org.apache.lucene.index.IndexingChain.flush(org.apache.lucene.index.SegmentWriteState)"], "Stackframe.line.number": [281], "Stackframe.function.offset": [40]} +{"create": {"_index": "profiling-stackframes", "_id": "PQiF41WruwXqpcujdafqogAAAAAAAAAl"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InflaterInputStream.java"], "Stackframe.function.name": ["int java.util.zip.InflaterInputStream.read(byte[], int, int)"], "Stackframe.line.number": [153], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "5jDiO_Lgv2e10LojWCULCwAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixChannelFactory.java"], "Stackframe.function.name": ["java.nio.channels.FileChannel sun.nio.fs.UnixChannelFactory.newFileChannel(sun.nio.fs.UnixPath, java.util.Set, int)"], "Stackframe.line.number": [146], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALpP6"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["do_filp_open"]} +{"create": {"_index": "profiling-stackframes", "_id": "NW0MvXdcfyNB_rjxH_JHXgAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLHandshake.java"], "Stackframe.function.name": ["byte[] sun.security.ssl.SSLHandshake.produce(sun.security.ssl.ConnectionContext, sun.security.ssl.SSLHandshake$HandshakeMessage)"], "Stackframe.line.number": [440], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "VlTLE-LbC-qAF2rlk-BxLQAAAAAAAABI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLEngineOutputRecord.java"], "Stackframe.function.name": ["sun.security.ssl.Ciphertext sun.security.ssl.SSLEngineOutputRecord.encode(java.nio.ByteBuffer[], int, int, java.nio.ByteBuffer[], int, int)"], "Stackframe.line.number": [189], "Stackframe.function.offset": [16]} +{"create": {"_index": "profiling-stackframes", "_id": "3X9PCd1tVPhzrMiwigfodgAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Unsafe.java"], "Stackframe.function.name": ["void jdk.internal.misc.Unsafe.unpark(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "CGeCGw7geerub3AylJt7SwAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["JsonXContentParser.java"], "Stackframe.function.name": ["java.lang.String org.elasticsearch.xcontent.provider.json.JsonXContentParser.text()"], "Stackframe.line.number": [106], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "0vAOoRRxsQcS4vDapC3-mwAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LinkedTransferQueue.java"], "Stackframe.function.name": ["boolean java.util.concurrent.LinkedTransferQueue$Node.tryMatch(java.lang.Object, java.lang.Object)"], "Stackframe.line.number": [448], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "SPBSRBf5-JQ_B0HnmSeY4AAAAAAAAACL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["org.apache.lucene.index.DirectoryReader org.apache.lucene.index.IndexWriter.getReader(boolean, boolean)"], "Stackframe.line.number": [570], "Stackframe.function.offset": [71]} +{"create": {"_index": "profiling-stackframes", "_id": "x88iiyZQ8B1lQnhTbyrEkwAAAAAAAACN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexWriter.finishCommit()"], "Stackframe.line.number": [4097], "Stackframe.function.offset": [32]} +{"create": {"_index": "profiling-stackframes", "_id": "bt6Ho82qwuZ3iDZme91J3QAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DeflaterOutputStream.java"], "Stackframe.function.name": ["void java.util.zip.DeflaterOutputStream.deflate()"], "Stackframe.line.number": [259], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "wnSBrxGSumHiAQQABJeNtQAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReplicationTracker.java"], "Stackframe.function.name": ["void org.elasticsearch.index.seqno.ReplicationTracker.updateReplicationGroupAndNotify()"], "Stackframe.line.number": [992], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "45faa5L8TTwA9zJYlmGKywAAAAAAAAAU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Invokers$Holder"], "Stackframe.function.name": ["java.lang.Object java.lang.invoke.Invokers$Holder.invokeExact_MT(java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "TfYUtN-rf_0uYWSYAHKfDQAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.callClusterStateListeners(org.elasticsearch.cluster.ClusterChangedEvent, org.elasticsearch.cluster.service.ClusterApplierRecordingService$Recorder)"], "Stackframe.line.number": [559], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "eC3OrXu_7VLA9EdCZMRh6AAAAAAAAAGd"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexWriter.startCommit(org.apache.lucene.index.SegmentInfos)"], "Stackframe.line.number": [5518], "Stackframe.function.offset": [80]} +{"create": {"_index": "profiling-stackframes", "_id": "PvkUluf_nN5PcG5RTjFmTgAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CodecUtil.java"], "Stackframe.function.name": ["void org.apache.lucene.codecs.CodecUtil.writeBELong(org.apache.lucene.store.DataOutput, long)"], "Stackframe.line.number": [661], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "i1t0pdr_xnHfk_UsEnpDiwAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLSocketImpl.java"], "Stackframe.function.name": ["int sun.security.ssl.SSLSocketImpl.readHandshakeRecord()"], "Stackframe.line.number": [1425], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "7wrMCA1tiwccLQqKJr9yEwAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.lambda$write$0(org.elasticsearch.xcontent.ToXContent, java.lang.String, boolean, java.util.Map, java.io.OutputStream)"], "Stackframe.line.number": [292], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "iM0ISMJkW1PbadMrIfDN5gAAAAAAAAAZ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["HttpRequestExecutor.java"], "Stackframe.function.name": ["org.apache.http.HttpResponse org.apache.http.protocol.HttpRequestExecutor.execute(org.apache.http.HttpRequest, org.apache.http.HttpClientConnection, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [123], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "0Pih7Ldp-yyaIKsINSnPvQAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileParser._finishLong()"], "Stackframe.line.number": [2170], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "ySPx-89oJ6TfXYn-uir7mQAAAAAAAABc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadPoolExecutor.java"], "Stackframe.function.name": ["void java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)"], "Stackframe.line.number": [1136], "Stackframe.function.offset": [20]} +{"create": {"_index": "profiling-stackframes", "_id": "AtOKW4aoAdMhmd7GyZi8TAAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ElasticsearchReaderManager.java"], "Stackframe.function.name": ["org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader org.elasticsearch.index.engine.ElasticsearchReaderManager.refreshIfNeeded(org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader)"], "Stackframe.line.number": [48], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "C-DRD-J_UvsXUc59TOzOVAAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.cluster.coordination.Coordinator$$Lambda$4627+0x000000080196c3c8.apply(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "DP-R410tjxklQGgyNbSilQAAAAAAAADK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["HashMap.java"], "Stackframe.function.name": ["java.lang.Object java.util.HashMap.computeIfAbsent(java.lang.Object, java.util.function.Function)"], "Stackframe.line.number": [1228], "Stackframe.function.offset": [31]} +{"create": {"_index": "profiling-stackframes", "_id": "TC2QqZAb8KnizCTsnAuCGgAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassLoader.java"], "Stackframe.function.name": ["java.net.URL java.net.URLClassLoader.findResource(java.lang.String)"], "Stackframe.line.number": [626], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAKgJ7"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["kmem_cache_alloc"]} +{"create": {"_index": "profiling-stackframes", "_id": "3Opn5CZ2iYMPW1WE6HYQfAAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Collections.java"], "Stackframe.function.name": ["boolean java.util.Collections$UnmodifiableCollection$1.hasNext()"], "Stackframe.line.number": [1053], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "luqtQRsIjOPWY7wdgiD4bAAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.lambda$write$0(java.lang.Object, java.lang.String, boolean, java.util.Map, java.io.OutputStream)"], "Stackframe.line.number": [346], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "fhsEKXDuxJ-jIJrZpdRuSAAAAAAAAEJ_"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["./deflate.c"], "Stackframe.function.name": ["deflate_fast"], "Stackframe.line.number": [1902]} +{"create": {"_index": "profiling-stackframes", "_id": "OkkiwQ9aM2OL-oHoBjLvhAAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReferenceManager.java"], "Stackframe.function.name": ["void org.apache.lucene.search.ReferenceManager.maybeRefreshBlocking()"], "Stackframe.line.number": [240], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "q5h54Kb3U85YSjTzRyaF_gAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReferenceManager.java"], "Stackframe.function.name": ["void org.apache.lucene.search.ReferenceManager.doMaybeRefresh()"], "Stackframe.line.number": [167], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "EBqXZsoPGK2F0IF3gcVlzwAAAAAAAAAa"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ActionListener.java"], "Stackframe.function.name": ["void org.elasticsearch.action.ActionListener$MappedActionListener.onResponse(java.lang.Object)"], "Stackframe.line.number": [101], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALpFF"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["user_path_at_empty"]} +{"create": {"_index": "profiling-stackframes", "_id": "eOepQqcpphM0NTbieX0wIgAAAAAAAAE4"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Coordinator.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishWithJoinResponse org.elasticsearch.cluster.coordination.Coordinator.handlePublishRequest(org.elasticsearch.cluster.coordination.PublishRequest)"], "Stackframe.line.number": [459], "Stackframe.function.offset": [46]} +{"create": {"_index": "profiling-stackframes", "_id": "7z0vK5_lN4w4BvVehfj4wQAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FSDirectory.java"], "Stackframe.function.name": ["void org.apache.lucene.store.FSDirectory.fsync(java.lang.String)"], "Stackframe.line.number": [331], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ReKfR-5xbP5LM4-DWXoUigAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DeflaterOutputStream.java"], "Stackframe.function.name": ["void java.util.zip.DeflaterOutputStream.deflate()"], "Stackframe.line.number": [253], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "7pNlbTUIFAGF4ew35tQ1mAAAAAAAAABA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.runRequestInterceptors(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [597], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "1XoENb4qIQOXPbHVVVKpqgAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexShard.java"], "Stackframe.function.name": ["void org.elasticsearch.index.shard.IndexShard.lambda$wrapPrimaryOperationPermitListener$25(org.elasticsearch.action.ActionListener, org.elasticsearch.core.Releasable)"], "Stackframe.line.number": [3354], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "id_Nq5V40zsa5EMiEnSMswAAAAAAAAA0"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NioSocketImpl.java"], "Stackframe.function.name": ["void sun.nio.ch.NioSocketImpl.write(byte[], int, int)"], "Stackframe.line.number": [437], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "QkrVHFzrrNPXJryM_udmYQAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SocketAccess.java"], "Stackframe.function.name": ["java.lang.Void org.elasticsearch.repositories.s3.SocketAccess.lambda$doPrivilegedVoid$0(java.lang.Runnable)"], "Stackframe.line.number": [46], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "zY8oFom0Mq0j4EO6ho2wLAAAAAAAAAME"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["long org.apache.lucene.index.IndexWriter.prepareCommitInternal()"], "Stackframe.line.number": [3715], "Stackframe.function.offset": [143]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAMRyc"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["do_fsync"]} +{"create": {"_index": "profiling-stackframes", "_id": "5TLUwI_khEb2-zaj4FK_ZQAAAAAAAAAa"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DigestBase.java"], "Stackframe.function.name": ["void sun.security.provider.DigestBase.engineUpdate(byte[], int, int)"], "Stackframe.line.number": [111], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALW4J"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__x64_sys_write"]} +{"create": {"_index": "profiling-stackframes", "_id": "sE8Y8IrvW4YwnRePcOob7gAAAAAAAADr"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.authorizeAction(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, java.lang.String, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [476], "Stackframe.function.offset": [53]} +{"create": {"_index": "profiling-stackframes", "_id": "CgEj6Rp3n26KLxmeHzazOAAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixFileAttributes.java"], "Stackframe.function.name": ["sun.nio.fs.UnixFileAttributes sun.nio.fs.UnixFileAttributes.get(sun.nio.fs.UnixPath, boolean)"], "Stackframe.line.number": [72], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "l4dVCOTlRJdBiyQ88Tx8OQAAAAAAAABR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["int com.fasterxml.jackson.dataformat.smile.SmileParser._fourBytesToIntSlow()"], "Stackframe.line.number": [2237], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "4jtFwV4I0VFxZ633CiXAEQAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IntegerPolynomialP256.java"], "Stackframe.function.name": ["void sun.security.util.math.intpoly.IntegerPolynomialP256.carryReduce0(long[], long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long)"], "Stackframe.line.number": [170], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "HbnRCcPqpeCpFx8D1c3dbgAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FilterDirectoryReader.java"], "Stackframe.function.name": ["org.apache.lucene.index.DirectoryReader org.apache.lucene.index.FilterDirectoryReader.doOpenIfChanged()"], "Stackframe.line.number": [112], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "NWk7GQCtDStIuwl83iGqTwAAAAAAAAAv"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PersistedClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.PersistedClusterStateService$Writer.commit(long, long, org.elasticsearch.Version)"], "Stackframe.line.number": [941], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "acnJzC_C1bsmTPMH_pcsCAAAAAAAAABd"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexTemplateRegistry.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.core.template.IndexTemplateRegistry.clusterChanged(org.elasticsearch.cluster.ClusterChangedEvent)"], "Stackframe.line.number": [187], "Stackframe.function.offset": [25]} +{"create": {"_index": "profiling-stackframes", "_id": "UMvK1DDwEUhXYDkQsZey_gAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportShardRefreshAction.java"], "Stackframe.function.name": ["void org.elasticsearch.action.admin.indices.refresh.TransportShardRefreshAction.shardOperationOnPrimary(org.elasticsearch.action.support.replication.BasicReplicationRequest, org.elasticsearch.index.shard.IndexShard, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [90], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "a9pOxt0fqv_QTtBRBlV_8QAAAAAAAAAl"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InternalEngine.java"], "Stackframe.function.name": ["org.elasticsearch.index.engine.Engine$RefreshResult org.elasticsearch.index.engine.InternalEngine.refresh(java.lang.String, org.elasticsearch.index.engine.Engine$SearcherScope, boolean)"], "Stackframe.line.number": [1860], "Stackframe.function.offset": [13]} +{"create": {"_index": "profiling-stackframes", "_id": "yTY-WlmajV5YhETL2qC3jgAAAAAAAAAd"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexWriter$IndexWriterMergeSource.merge(org.apache.lucene.index.MergePolicy$OneMerge)"], "Stackframe.line.number": [6432], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "EIxfgHbDbI5dElFzd3Ha-QAAAAAAAAAZ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PendingReplicationActions.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.PendingReplicationActions.accept(org.elasticsearch.index.shard.ReplicationGroup)"], "Stackframe.line.number": [71], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "TyZK4BUMsPg3P3wu6EYhkAAAAAAAAAB6"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PersistedClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.PersistedClusterStateService$MetadataIndexWriter.prepareCommit(java.lang.String, long, long, org.elasticsearch.Version)"], "Stackframe.line.number": [799], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "3V-8FLy1GH8nVRceMDeaiwAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PrioritizedEsThreadPoolExecutor.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.runAndClean(java.lang.Runnable)"], "Stackframe.line.number": [262], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Unm7y6YLik7WuESK-9BXmgAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileChannel.java"], "Stackframe.function.name": ["java.nio.channels.FileChannel java.nio.channels.FileChannel.open(java.nio.file.Path, java.util.Set, java.nio.file.attribute.FileAttribute[])"], "Stackframe.line.number": [304], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "zZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadPoolExecutor.java"], "Stackframe.function.name": ["void java.util.concurrent.ThreadPoolExecutor$Worker.run()"], "Stackframe.line.number": [635], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "5IpWfunjiW7HmwJPoVX8WQAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["MatchOps.java"], "Stackframe.function.name": ["java.lang.Boolean java.util.stream.MatchOps$MatchOp.evaluateSequential(java.util.stream.PipelineHelper, java.util.Spliterator)"], "Stackframe.line.number": [230], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ytcfMbv--KUIhYho462CJAAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Thread.java"], "Stackframe.function.name": ["void java.lang.Thread.run()"], "Stackframe.line.number": [1589], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "eWbRBA5HjtSejwiG_37xQwAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["GaloisCounterMode.java"], "Stackframe.function.name": ["int com.sun.crypto.provider.GaloisCounterMode.engineGetOutputSize(int)"], "Stackframe.line.number": [204], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "bzjbwk-dGTAEYzErTwuuOQAAAAAAAADg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BulkShardRequestInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.BulkShardRequestInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [85], "Stackframe.function.offset": [34]} +{"create": {"_index": "profiling-stackframes", "_id": "N1GpN_XYj7AoDd4ql2RyHgAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots org.elasticsearch.repositories.blobstore.BlobStoreRepository.lambda$static$3(java.lang.String, org.elasticsearch.xcontent.XContentParser)"], "Stackframe.line.number": [303], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "S271rVNO8pQHfSRu8D2-7gAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DirectoryReader.java"], "Stackframe.function.name": ["org.apache.lucene.index.DirectoryReader org.apache.lucene.index.DirectoryReader.openIfChanged(org.apache.lucene.index.DirectoryReader)"], "Stackframe.line.number": [170], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "yhAKi3MXCBa6QbMykpz9ogAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ActionRunnable.java"], "Stackframe.function.name": ["void org.elasticsearch.action.ActionRunnable$3.doRun()"], "Stackframe.line.number": [72], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "KtHBJ2R0vuuTX50F8kgGAgAAAAAAAAAh"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileGenerator.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileGenerator._flushBuffer()"], "Stackframe.line.number": [2585], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "YIa5zXbUOkdcfY3BuS_SLwAAAAAAAAAK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.TransportReplicationAction$$Lambda$5077+0x00000008020bf4b8.messageReceived(org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.tasks.Task)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "q8pCDW17i5MKeUo5LLv96QAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassLoader.java"], "Stackframe.function.name": ["java.lang.Object java.net.URLClassLoader$2.run()"], "Stackframe.line.number": [627], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "PiSwsMqyjn24GnxXwBvfuQAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["long org.apache.lucene.index.IndexWriter.prepareCommit()"], "Stackframe.line.number": [3535], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "GabeEr60puRs20AAHoWA0AAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReferencePipeline.java"], "Stackframe.function.name": ["boolean java.util.stream.ReferencePipeline.anyMatch(java.util.function.Predicate)"], "Stackframe.line.number": [632], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "TrTEyZn7B30gBbM8MLULSQAAAAAAAADA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IOUtils.java"], "Stackframe.function.name": ["void org.apache.lucene.util.IOUtils.fsync(java.nio.file.Path, boolean)"], "Stackframe.line.number": [483], "Stackframe.function.offset": [22]} +{"create": {"_index": "profiling-stackframes", "_id": "FERPJCfrzkqs-7XhU53jhwAAAAAAAAFt"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLConnectionSocketFactory.java"], "Stackframe.function.name": ["java.net.Socket org.apache.http.conn.ssl.SSLConnectionSocketFactory.createLayeredSocket(java.net.Socket, java.lang.String, int, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [436], "Stackframe.function.offset": [43]} +{"create": {"_index": "profiling-stackframes", "_id": "1PIPTBbtPHUrg0d3N2nuygAAAAAAAABB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SdkTLSSocketFactory.java"], "Stackframe.function.name": ["java.net.Socket com.amazonaws.http.conn.ssl.SdkTLSSocketFactory.connectSocket(int, java.net.Socket, org.apache.http.HttpHost, java.net.InetSocketAddress, java.net.InetSocketAddress, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [142], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "aBWoLwWk5u6kqgD3gwKyagAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat$$Lambda$10706+0x0000000802977208.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "-lMJxP-0bZiF5XB2gMdbVwAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["java.lang.Object com.amazonaws.services.s3.AmazonS3Client.access$300(com.amazonaws.services.s3.AmazonS3Client, com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, java.lang.String, java.lang.String)"], "Stackframe.line.number": [394], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "duOfgYTG-Pz3qxfosjBOKgAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SocketAccess.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.s3.SocketAccess.doPrivileged(java.security.PrivilegedAction)"], "Stackframe.line.number": [31], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "BjycB5yXrYLMnUcjhbPq2gAAAAAAAAAn"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedInputStream.java"], "Stackframe.function.name": ["int java.io.BufferedInputStream.read1(byte[], int, int)"], "Stackframe.line.number": [308], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "7F9XBiRMgggoDE1ZbLLQRgAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["MessageDigest.java"], "Stackframe.function.name": ["void java.security.MessageDigest$Delegate.engineUpdate(byte[], int, int)"], "Stackframe.line.number": [654], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "qzJXUPlBwNnFY2QMV7wpwgAAAAAAAACF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Deflater.java"], "Stackframe.function.name": ["int java.util.zip.Deflater.deflate(byte[], int, int, int)"], "Stackframe.line.number": [582], "Stackframe.function.offset": [25]} +{"create": {"_index": "profiling-stackframes", "_id": "6TUXIWC2WcF-_43xKp-wtQAAAAAAAAET"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CoordinationState.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishResponse org.elasticsearch.cluster.coordination.CoordinationState.handlePublishRequest(org.elasticsearch.cluster.coordination.PublishRequest)"], "Stackframe.line.number": [413], "Stackframe.function.offset": [36]} +{"create": {"_index": "profiling-stackframes", "_id": "tXdLdOQIv2CEGExsyL_epQAAAAAAAAC4"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedInputStream.java"], "Stackframe.function.name": ["void java.io.BufferedInputStream.fill()"], "Stackframe.line.number": [244], "Stackframe.function.offset": [32]} +{"create": {"_index": "profiling-stackframes", "_id": "8iRqFHNeeA-Vq5mM-jaESgAAAAAAAAAR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassPath.java"], "Stackframe.function.name": ["java.net.URL jdk.internal.loader.URLClassPath.findResource(java.lang.String, boolean)"], "Stackframe.line.number": [299], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "yaSy1RHb0tl4hOkUXXLXQQAAAAAAAAAz"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["org.elasticsearch.core.Tuple org.elasticsearch.repositories.blobstore.BlobStoreRepository.buildBlobStoreIndexShardSnapshots(java.util.Set, org.elasticsearch.common.blobstore.BlobContainer, org.elasticsearch.repositories.ShardGeneration)"], "Stackframe.line.number": [3439], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "Z737A4mvoWuasLPvz8LMuAAAAAAAAABD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.warnAboutSlowTaskIfNeeded(org.elasticsearch.core.TimeValue, java.lang.String, org.elasticsearch.cluster.service.ClusterApplierRecordingService$Recorder)"], "Stackframe.line.number": [615], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "-FiXU0afkg-sRwubj2WiAwAAAAAAAmE0"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../../../dbus/dbus-mainloop.c"], "Stackframe.function.name": ["_dbus_loop_run"], "Stackframe.line.number": [888]} +{"create": {"_index": "profiling-stackframes", "_id": "eG5s_qI4F4I51_yuNS46mwAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.lambda$authorizeAction$11(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, java.lang.String, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AsyncSupplier, org.elasticsearch.cluster.metadata.Metadata, org.elasticsearch.action.ActionListener, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$IndexAuthorizationResult)"], "Stackframe.line.number": [483], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "4iWyc_l7zIMt9gi9uNbDxgAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InternalEngine.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.index.engine.InternalEngine$ExternalReaderManager.refreshIfNeeded(java.lang.Object)"], "Stackframe.line.number": [367], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "5-kXfhOMZeJHGZEEqUgZKAAAAAAAAADS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ResizeRequestInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.ResizeRequestInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [98], "Stackframe.function.offset": [47]} +{"create": {"_index": "profiling-stackframes", "_id": "2B-rbS8KhbB8ePRnpVf7_QAAAAAAAAAU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECDHKeyExchange.java"], "Stackframe.function.name": ["void sun.security.ssl.ECDHKeyExchange$ECDHEPossession.(sun.security.ssl.NamedGroup, java.security.SecureRandom)"], "Stackframe.line.number": [119], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "im4CSpLxZ-7jo4vfOXMUNAAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.index.shard.IndexShard$$Lambda$8959+0x00000008027eb9c0.accept(java.lang.Object, java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "HWeYMQJlcaP0GMe8ZUABEgAAAAAAAAAu"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixPath.java"], "Stackframe.function.name": ["java.lang.String sun.nio.fs.UnixPath.normalizeAndCheck(java.lang.String)"], "Stackframe.line.number": [80], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "fhsEKXDuxJ-jIJrZpdRuSAAAAAAAADZ4"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["", "", "./deflate.c", "./deflate.c", "./deflate.c", "./deflate.c"], "Stackframe.function.name": ["fill_window", "fill_window", "fill_window", "slide_hash", "slide_hash", "slide_hash"], "Stackframe.line.number": [0, 0, 1580, 1580, 1580, 226]} +{"create": {"_index": "profiling-stackframes", "_id": "6YIeLAztuVSewvuGh8XKXgAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LinkedTransferQueue.java"], "Stackframe.function.name": ["boolean java.util.concurrent.LinkedTransferQueue.tryTransfer(java.lang.Object)"], "Stackframe.line.number": [1241], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "pKAwMR-9-3afO3tccFzRKwAAAAAAAABw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECOperations.java"], "Stackframe.function.name": ["void sun.security.ec.ECOperations.setDouble(sun.security.ec.point.ProjectivePoint$Mutable, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP, sun.security.util.math.MutableIntegerModuloP)"], "Stackframe.line.number": [302], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "TrTEyZn7B30gBbM8MLULSQAAAAAAAABB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IOUtils.java"], "Stackframe.function.name": ["void org.apache.lucene.util.IOUtils.fsync(java.nio.file.Path, boolean)"], "Stackframe.line.number": [471], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "cBTlmvRfiK_eaWEDctBQ5wAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService$$Lambda$9093+0x00000008020d9700.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "5t8GiksjQcuLCsu8jyRsaAAAAAAAAAA2"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshots.java"], "Stackframe.function.name": ["org.elasticsearch.xcontent.XContentBuilder org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots.toXContent(org.elasticsearch.xcontent.XContentBuilder, org.elasticsearch.xcontent.ToXContent$Params)"], "Stackframe.line.number": [221], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "LY5nQliSvp7OkEEaJEKLkwAAAAAAAABB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ServerTransportFilter.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.ServerTransportFilter.lambda$inbound$1(java.lang.String, org.elasticsearch.TransportVersion, org.elasticsearch.transport.TransportRequest, org.elasticsearch.action.ActionListener, org.elasticsearch.xpack.core.security.authc.Authentication)"], "Stackframe.line.number": [113], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "3SU4n_eVvbLOxW0zu9gonAAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient.execute(com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, com.amazonaws.http.HttpResponseHandler, com.amazonaws.http.ExecutionContext)"], "Stackframe.line.number": [541], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "RQiOAUVYPrJNPqfM54o9YwAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ShardSnapshotTaskRunner.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ShardSnapshotTaskRunner$ShardSnapshotTask.doRun()"], "Stackframe.line.number": [77], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "2HSuy-oZigDpg5cKC7dkqwAAAAAAAACX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixPath.java"], "Stackframe.function.name": ["void sun.nio.fs.UnixPath.initOffsets()"], "Stackframe.line.number": [204], "Stackframe.function.offset": [31]} +{"create": {"_index": "profiling-stackframes", "_id": "weat5ldYNeQ9Grv2_1-CAwAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixFileSystemProvider.java"], "Stackframe.function.name": ["boolean sun.nio.fs.UnixFileSystemProvider.implDelete(java.nio.file.Path, boolean)"], "Stackframe.line.number": [231], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "-FiXU0afkg-sRwubj2WiAwAAAAAAAZdL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../../../bus/dispatch.c"], "Stackframe.function.name": ["bus_dispatch_matches"], "Stackframe.line.number": [179]} +{"create": {"_index": "profiling-stackframes", "_id": "dkAubCDgL7khnJPkNdUwRwAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InternalEngine.java"], "Stackframe.function.name": ["org.elasticsearch.index.engine.Engine$RefreshResult org.elasticsearch.index.engine.InternalEngine.refresh(java.lang.String)"], "Stackframe.line.number": [1836], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "hfMR5YVLU22vPk_NOG-TgAAAAAAAAAAX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECKeyPairGenerator.java"], "Stackframe.function.name": ["byte[] sun.security.ec.ECKeyPairGenerator.generatePrivateScalar(java.security.SecureRandom, sun.security.ec.ECOperations, int)"], "Stackframe.line.number": [168], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "ClvIuFYr5ArIOKSAaLGqggAAAAAAAAGv"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper()"], "Stackframe.line.number": [1157], "Stackframe.function.offset": [61]} +{"create": {"_index": "profiling-stackframes", "_id": "y3hT9dbM11gdF2x7xJH8oQAAAAAAAAAR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParserBase.java"], "Stackframe.function.name": ["long com.fasterxml.jackson.dataformat.smile.SmileParserBase.getLongValue()"], "Stackframe.line.number": [538], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "C-hFb4flwkETp_lahExtrwAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportService.java"], "Stackframe.function.name": ["void org.elasticsearch.transport.TransportService$6.doRun()"], "Stackframe.line.number": [976], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ZaLM93n34r2A8Tcv4UQ0mgAAAAAAAABC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportReplicationAction.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.TransportReplicationAction.handlePrimaryRequest(org.elasticsearch.action.support.replication.TransportReplicationAction$ConcreteShardRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.tasks.Task)"], "Stackframe.line.number": [355], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "YcHA99kMtpR5AAkygC-qMAAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CodecUtil.java"], "Stackframe.function.name": ["void org.apache.lucene.codecs.CodecUtil.writeCRC(org.apache.lucene.store.IndexOutput)"], "Stackframe.line.number": [648], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAAEM2"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["do_syscall_64"]} +{"create": {"_index": "profiling-stackframes", "_id": "sxq3K3ppB_xqEWb8y-RbYwAAAAAAAAC8"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3BlobContainer.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3BlobContainer.executeSingleUpload(org.elasticsearch.repositories.s3.S3BlobStore, java.lang.String, java.io.InputStream, long)"], "Stackframe.line.number": [490], "Stackframe.function.offset": [18]} +{"create": {"_index": "profiling-stackframes", "_id": "vs-DizK6zOgT7PLk55Hi8wAAAAAAAABg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexFileDeleter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexFileDeleter.decRef(java.util.Collection)"], "Stackframe.line.number": [656], "Stackframe.function.offset": [14]} +{"create": {"_index": "profiling-stackframes", "_id": "FhP6JufLZYYhuADwCLvRKAAAAAAAAACV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute()"], "Stackframe.line.number": [814], "Stackframe.function.offset": [20]} +{"create": {"_index": "profiling-stackframes", "_id": "mSb0V8ANbBWpi7TZTXcPNwAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FSDirectory.java"], "Stackframe.function.name": ["void org.apache.lucene.store.FSDirectory.fsync(java.lang.String)"], "Stackframe.line.number": [303], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "HoqCWZ471w3YYcVCfd06hAAAAAAAAACh"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessController.java"], "Stackframe.function.name": ["void java.security.AccessController.checkPermission(java.security.Permission)"], "Stackframe.line.number": [1068], "Stackframe.function.offset": [30]} +{"create": {"_index": "profiling-stackframes", "_id": "QTXwwuqN0mLb-8lqckiaTQAAAAAAAWoj"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["xfs_attr_get"]} +{"create": {"_index": "profiling-stackframes", "_id": "u-D_h2osKIeJ8uQjOobu_AAAAAAAAAHy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReplicationTracker.java"], "Stackframe.function.name": ["void org.elasticsearch.index.seqno.ReplicationTracker.updateFromMaster(long, java.util.Set, org.elasticsearch.cluster.routing.IndexShardRoutingTable)"], "Stackframe.line.number": [1198], "Stackframe.function.offset": [47]} +{"create": {"_index": "profiling-stackframes", "_id": "HNWspokmSU3mmXH9ENGQygAAAAAAAABV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FilePermission.java"], "Stackframe.function.name": ["boolean java.io.FilePermissionCollection.implies(java.security.Permission)"], "Stackframe.line.number": [1202], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "5ajLrlzgZ5NJn1Ifyl4tjwAAAAAAAACK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FilePermission.java"], "Stackframe.function.name": ["boolean java.io.FilePermission.impliesIgnoreMask(java.io.FilePermission)"], "Stackframe.line.number": [611], "Stackframe.function.offset": [23]} +{"create": {"_index": "profiling-stackframes", "_id": "s1ge7c8paVPaLi3Du1Dh1QAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SocketDispatcher.java"], "Stackframe.function.name": ["int sun.nio.ch.SocketDispatcher.write(java.io.FileDescriptor, long, int)"], "Stackframe.line.number": [62], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "VDYfO2UIwBGcuJQmu3YTtwAAAAAAAAAU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedOutputStream.java"], "Stackframe.function.name": ["void java.io.BufferedOutputStream.write(byte[], int, int)"], "Stackframe.line.number": [123], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "Ryec96xv7uLWO7iiTNqYKgAAAAAAAAAm"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractPipeline.java"], "Stackframe.function.name": ["boolean java.util.stream.AbstractPipeline.copyIntoWithCancel(java.util.stream.Sink, java.util.Spliterator)"], "Stackframe.line.number": [528], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "TnhL_a5gaR0GbLDTAKZU8gAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InternalEngine.java"], "Stackframe.function.name": ["org.elasticsearch.index.engine.Engine$IndexCommitRef org.elasticsearch.index.engine.InternalEngine.acquireLastIndexCommit(boolean)"], "Stackframe.line.number": [2224], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAMCXY"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__vfs_getxattr"]} +{"create": {"_index": "profiling-stackframes", "_id": "HwASysJunE30XUonp_SYPwAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ByteSizeCachingDirectory.java"], "Stackframe.function.name": ["void org.elasticsearch.index.store.ByteSizeCachingDirectory$2.writeLong(long)"], "Stackframe.line.number": [162], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Ohy0c1RhzpJIxIHtdNOBrgAAAAAAAABe"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InputStreamEntity.java"], "Stackframe.function.name": ["void org.apache.http.entity.InputStreamEntity.writeTo(java.io.OutputStream)"], "Stackframe.line.number": [144], "Stackframe.function.offset": [18]} +{"create": {"_index": "profiling-stackframes", "_id": "m_DK0nAVx1tMIimjFYyQPAAAAAAAAAAS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.execute()"], "Stackframe.line.number": [755], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "yqhsLQpHnDkAN8Nbjjx5JQAAAAAAAABp"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["RequestHandlerRegistry.java"], "Stackframe.function.name": ["void org.elasticsearch.transport.RequestHandlerRegistry.processMessageReceived(org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel)"], "Stackframe.line.number": [71], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "jojK5E5-Tpf57iS2qx9_2QAAAAAAAAA2"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshot.java"], "Stackframe.function.name": ["void org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo.toXContent(org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo, org.elasticsearch.xcontent.XContentBuilder, org.elasticsearch.xcontent.ToXContent$Params)"], "Stackframe.line.number": [259], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "8A-e4vMGSQm_8vtVRynoJAAAAAAAAADL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLSocketImpl.java"], "Stackframe.function.name": ["void sun.security.ssl.SSLSocketImpl$AppOutputStream.write(byte[], int, int)"], "Stackframe.line.number": [1298], "Stackframe.function.offset": [35]} +{"create": {"_index": "profiling-stackframes", "_id": "vzzPIR5tUnMkJ7d_ITdQRgAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["EsExecutors.java"], "Stackframe.function.name": ["boolean org.elasticsearch.common.util.concurrent.EsExecutors$ExecutorScalingQueue.offer(java.lang.Object)"], "Stackframe.line.number": [363], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "5iLGjLJFjZ3vVn5PbnDFiAAAAAAAAAAv"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassPath.java"], "Stackframe.function.name": ["jdk.internal.loader.Resource jdk.internal.loader.URLClassPath$JarLoader.checkResource(java.lang.String, boolean, java.util.jar.JarEntry)"], "Stackframe.line.number": [867], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "tq710yUF9bzkGXjQwM8_6QAAAAAAAADW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ECOperations.java"], "Stackframe.function.name": ["sun.security.ec.point.MutablePoint sun.security.ec.ECOperations.multiply(sun.security.ec.point.AffinePoint, byte[])"], "Stackframe.line.number": [271], "Stackframe.function.offset": [32]} +{"create": {"_index": "profiling-stackframes", "_id": "15Kl_LNlY1wkBO4-dVuptAAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["long org.apache.lucene.index.IndexWriter.commit()"], "Stackframe.line.number": [4005], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAADSU7"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["try_to_wake_up"]} +{"create": {"_index": "profiling-stackframes", "_id": "u1a5bpPCc1mWwQaCPFMnewAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SearchRequestInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.SearchRequestInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [21], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "SdeiuUk4wJykJDjmt_gy5AAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["OneMergeWrappingMergePolicy.java"], "Stackframe.function.name": ["org.apache.lucene.index.MergePolicy$MergeSpecification org.apache.lucene.index.OneMergeWrappingMergePolicy.findMerges(org.apache.lucene.index.MergeTrigger, org.apache.lucene.index.SegmentInfos, org.apache.lucene.index.MergePolicy$MergeContext)"], "Stackframe.line.number": [48], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "1kYL5y-AD3sdPCqn6ivZowAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots$$Lambda$8712+0x0000000802140000.apply(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "QpI4ER3R8tI6upvquCghPgAAAAAAAABG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["GroupedActionListener.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.GroupedActionListener.onResponse(java.lang.Object)"], "Stackframe.line.number": [55], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAFBxM"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["futex_wake"]} +{"create": {"_index": "profiling-stackframes", "_id": "AFikCbtP_Dm7iUthjnlnEgAAAAAAAAGt"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.applyChanges(org.elasticsearch.cluster.ClusterState, org.elasticsearch.cluster.ClusterState, java.lang.String, org.elasticsearch.cluster.service.ClusterApplierRecordingService$Recorder)"], "Stackframe.line.number": [519], "Stackframe.function.offset": [41]} +{"create": {"_index": "profiling-stackframes", "_id": "f_uYAlUc4Mk4nBdvFkWUPgAAAAAAAAME"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["long org.apache.lucene.index.IndexWriter.prepareCommitInternal()"], "Stackframe.line.number": [3460], "Stackframe.function.offset": [130]} +{"create": {"_index": "profiling-stackframes", "_id": "77CMGAnm82a_x5oOgINbKgAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Permissions.java"], "Stackframe.function.name": ["boolean java.security.Permissions.implies(java.security.Permission)"], "Stackframe.line.number": [177], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "3D_A_sI8XDgpK1lgEmfWZgAAAAAAAAC_"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["HandshakeContext.java"], "Stackframe.function.name": ["void sun.security.ssl.HandshakeContext.dispatch(byte, sun.security.ssl.Plaintext)"], "Stackframe.line.number": [458], "Stackframe.function.offset": [66]} +{"create": {"_index": "profiling-stackframes", "_id": "cn7jRZ-ZvDDBUMM5Rer9wAAAAAAAAAAD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["MatchOps.java"], "Stackframe.function.name": ["java.lang.Object java.util.stream.MatchOps$MatchOp.evaluateSequential(java.util.stream.PipelineHelper, java.util.Spliterator)"], "Stackframe.line.number": [196], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "rJNvWVtt3dJDIQfHVkT3mAAAAAAAAAAk"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService$$Lambda$7229+0x000000080251cea0.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "eBJz8rf8VBkiNSfy4HLlcwAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CodecUtil.java"], "Stackframe.function.name": ["void org.apache.lucene.codecs.CodecUtil.writeFooter(org.apache.lucene.store.IndexOutput)"], "Stackframe.line.number": [411], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "eJ7mihRgQcJvejyU6kLDVAAAAAAAAAAc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService$$Lambda$10692+0x00000008029728f0.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "KjF0JPI3hCXDTCc6Z7ofdQAAAAAAAAFL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.runTask(java.lang.String, java.util.function.Function, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [428], "Stackframe.function.offset": [44]} +{"create": {"_index": "profiling-stackframes", "_id": "h4dwrMYlRFRjyfsvjXt4tgAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PrioritizedEsThreadPoolExecutor.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor$TieBreakingPrioritizedRunnable.run()"], "Stackframe.line.number": [225], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALjan"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["lookup_fast"]} +{"create": {"_index": "profiling-stackframes", "_id": "vM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PrioritizedThrottledTaskRunner.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.PrioritizedThrottledTaskRunner$TaskWrapper.onResponse(org.elasticsearch.core.Releasable)"], "Stackframe.line.number": [51], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "wRG5B2klLAxXtt0B9lVulAAAAAAAAADS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.serialize(java.lang.Object, java.lang.String, boolean, java.util.Map, java.io.OutputStream)"], "Stackframe.line.number": [389], "Stackframe.function.offset": [29]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALpAY"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["filename_lookup"]} +{"create": {"_index": "profiling-stackframes", "_id": "X-q4xdk6PYPLAgZGMSlgiwAAAAAAAAAZ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.write(org.elasticsearch.xcontent.ToXContent, org.elasticsearch.common.blobstore.BlobContainer, java.lang.String, boolean, java.util.Map)"], "Stackframe.line.number": [292], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "nMqxSWeol-RN_p8hXE5oZQAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexShard.java"], "Stackframe.function.name": ["org.elasticsearch.index.engine.Engine$IndexCommitRef org.elasticsearch.index.shard.IndexShard.acquireIndexCommitForSnapshot()"], "Stackframe.line.number": [1443], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "gQv1haKZmRIe2g2DBTzDVAAAAAAAAAAX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["org.elasticsearch.xcontent.ToXContent org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.read(java.lang.String, org.elasticsearch.common.blobstore.BlobContainer, java.lang.String, org.elasticsearch.xcontent.NamedXContentRegistry)"], "Stackframe.line.number": [89], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "TvbloCY34U08znkUfz6GbAAAAAAAAAAU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["com.amazonaws.services.s3.model.ObjectMetadata com.amazonaws.services.s3.AmazonS3Client$PutObjectStrategy.invokeServiceCall(com.amazonaws.Request)"], "Stackframe.line.number": [5942], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "WOsBhYd9IbFldGZWVw03hgAAAAAAAAAV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ProtectionDomain.java"], "Stackframe.function.name": ["boolean java.security.ProtectionDomain.implies(java.security.Permission)"], "Stackframe.line.number": [325], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "PKIQdknzONDqr2S2uRbbjQAAAAAAAACC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.authorize(org.elasticsearch.xpack.core.security.authc.Authentication, java.lang.String, org.elasticsearch.transport.TransportRequest, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [250], "Stackframe.function.offset": [37]} +{"create": {"_index": "profiling-stackframes", "_id": "uBDO9BMjtjTmzHTlTfDLzAAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots$$Lambda$9714+0x00000008028a2440.apply(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "5VGPPNmqbrejwy3AxU1wmQAAAAAAAAB5"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLCipher.java"], "Stackframe.function.name": ["int sun.security.ssl.SSLCipher$T13GcmWriteCipherGenerator$GcmWriteCipher.encrypt(byte, java.nio.ByteBuffer)"], "Stackframe.line.number": [2101], "Stackframe.function.offset": [21]} +{"create": {"_index": "profiling-stackframes", "_id": "r4QCFwkajSWVBltX9dOYSwAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Sink.java"], "Stackframe.function.name": ["void java.util.stream.Sink.end()"], "Stackframe.line.number": [138], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "TrTEyZn7B30gBbM8MLULSQAAAAAAAAA5"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IOUtils.java"], "Stackframe.function.name": ["void org.apache.lucene.util.IOUtils.fsync(java.nio.file.Path, boolean)"], "Stackframe.line.number": [469], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "Oo9QFBPQY4ypFVPfKrisIQAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ConjunctionScorer.java"], "Stackframe.function.name": ["void org.apache.lucene.search.ConjunctionScorer.(org.apache.lucene.search.Weight, java.util.Collection, java.util.Collection)"], "Stackframe.line.number": [37], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAwACL"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["entry_SYSCALL_64_after_hwframe"]} +{"create": {"_index": "profiling-stackframes", "_id": "smN6GZa9-v-IcDUWnhvNkAAAAAAAAABL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexFileDeleter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexFileDeleter.deleteCommits()"], "Stackframe.line.number": [437], "Stackframe.function.offset": [14]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALSp_"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__x64_sys_openat"]} +{"create": {"_index": "profiling-stackframes", "_id": "COij2gsV-4lHDIB8-jTbygAAAAAAAACG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SnapshotShardsService.java"], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService.lambda$snapshot$3(org.elasticsearch.index.snapshots.IndexShardSnapshotStatus, org.elasticsearch.index.shard.ShardId, org.elasticsearch.snapshots.Snapshot, org.elasticsearch.repositories.IndexId, org.elasticsearch.Version, long, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [371], "Stackframe.function.offset": [21]} +{"create": {"_index": "profiling-stackframes", "_id": "Jy9mUBVZJITLUNGFd8QS0QAAAAAAAAFJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NioEventLoop.java"], "Stackframe.function.name": ["void io.netty.channel.nio.NioEventLoop.run()"], "Stackframe.line.number": [569], "Stackframe.function.offset": [64]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAATgeq"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["blkdev_issue_flush"]} +{"create": {"_index": "profiling-stackframes", "_id": "TF7R8j8eszhmy8WskJZ2NAAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.RBACEngine$$Lambda$7207+0x0000000802516a90.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "fhsEKXDuxJ-jIJrZpdRuSAAAAAAAAI9-"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["", "", "./inffast.c"], "Stackframe.function.name": ["inflate_fast", "inflate_fast", "inflate_fast"], "Stackframe.line.number": [0, 0, 113]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALUfw"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["vfs_write"]} +{"create": {"_index": "profiling-stackframes", "_id": "plZYNE8qhJOfCM58L5mfyQAAAAAAAAAd"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessController.java"], "Stackframe.function.name": ["java.lang.Object java.security.AccessController.executePrivileged(java.security.PrivilegedExceptionAction, java.security.AccessControlContext, java.lang.Class)"], "Stackframe.line.number": [807], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "yScC9Npb_ya8Br_QJ7EUZAAAAAAAAAB7"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Class.java"], "Stackframe.function.name": ["java.io.InputStream java.lang.Class.getResourceAsStream(java.lang.String)"], "Stackframe.line.number": [2882], "Stackframe.function.offset": [36]} +{"create": {"_index": "profiling-stackframes", "_id": "ZVMl7SvWgt8_FxFL79kGYgAAAAAAAAHm"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLEngineOutputRecord.java"], "Stackframe.function.name": ["sun.security.ssl.Ciphertext sun.security.ssl.SSLEngineOutputRecord.encode(java.nio.ByteBuffer[], int, int, java.nio.ByteBuffer)"], "Stackframe.line.number": [287], "Stackframe.function.offset": [92]} +{"create": {"_index": "profiling-stackframes", "_id": "DVxwSo4eOEycJY2lxrVA3wAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["JsonXContentParser.java"], "Stackframe.function.name": ["java.lang.String org.elasticsearch.xcontent.json.JsonXContentParser.text()"], "Stackframe.line.number": [104], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALSjZ"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["do_sys_open"]} +{"create": {"_index": "profiling-stackframes", "_id": "2V2dRNf-iGCAuWhm4oOfRAAAAAAAAABE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient.execute(com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, com.amazonaws.http.HttpResponseHandler, com.amazonaws.http.ExecutionContext, com.amazonaws.RequestConfig)"], "Stackframe.line.number": [544], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALb-c"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__do_sys_newlstat"]} +{"create": {"_index": "profiling-stackframes", "_id": "CNW3Uw4RnzkMwEIUegsYPwAAAAAAAACF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Deflater.java"], "Stackframe.function.name": ["int java.util.zip.Deflater.deflate(byte[], int, int, int)"], "Stackframe.line.number": [582], "Stackframe.function.offset": [25]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAC6oJ"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["worker_thread"]} +{"create": {"_index": "profiling-stackframes", "_id": "7z0vK5_lN4w4BvVehfj4wQAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FSDirectory.java"], "Stackframe.function.name": ["void org.apache.lucene.store.FSDirectory.fsync(java.lang.String)"], "Stackframe.line.number": [331], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "jLRjRKJujxpIzkv5EAU8CAAAAAAAAADE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SnapshotShardsService.java"], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService.lambda$startNewShards$1(org.elasticsearch.cluster.SnapshotsInProgress$Entry, java.util.Map)"], "Stackframe.line.number": [267], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "tMqbgEmfZJ47YRogSA-gKgAAAAAAAADl"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndicesClusterStateService.java"], "Stackframe.function.name": ["void org.elasticsearch.indices.cluster.IndicesClusterStateService.createOrUpdateShards(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [556], "Stackframe.function.offset": [18]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAApYft"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["copy_user_enhanced_fast_string"]} +{"create": {"_index": "profiling-stackframes", "_id": "zvnrbT9TB1hdLy7i_u1ShgAAAAAAAAAl"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LoggingHandler.java"], "Stackframe.function.name": ["void io.netty.handler.logging.LoggingHandler.flush(io.netty.channel.ChannelHandlerContext)"], "Stackframe.line.number": [304], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "XxvzEXolyPiyR3WZkqmNewAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SessionOutputBufferImpl.java"], "Stackframe.function.name": ["void org.apache.http.impl.io.SessionOutputBufferImpl.streamWrite(byte[], int, int)"], "Stackframe.line.number": [124], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "NDsPouY3NiBzl33zaU7fMgAAAAAAAAA4"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService$AuthorizationResultListener.onResponse(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationResult)"], "Stackframe.line.number": [986], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "5wGUH60A7iQqyWZA8jj7kwAAAAAAAAAd"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessController.java"], "Stackframe.function.name": ["java.lang.Object java.security.AccessController.executePrivileged(java.security.PrivilegedAction, java.security.AccessControlContext, java.lang.Class)"], "Stackframe.line.number": [776], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "nxbylxGsMi-XNEec__mC2gAAAAAAAAAP"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLEngineImpl.java"], "Stackframe.function.name": ["javax.net.ssl.SSLEngineResult sun.security.ssl.SSLEngineImpl.wrap(java.nio.ByteBuffer[], int, int, java.nio.ByteBuffer)"], "Stackframe.line.number": [123], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "radvx3ka7U7YJiCSRTJ3kwAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.ServerTransportFilter$NodeProfile$$Lambda$6206+0x0000000801c16d08.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "2bJMveL01ajlzm-wdxWvUAAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.cluster.coordination.Coordinator$$Lambda$4541+0x00000008018321a8.apply(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "7tZRpCUjHjYLEjfN8TUeQAAAAAAAAABw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SearchRequestCacheDisablingInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.SearchRequestCacheDisablingInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [53], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "wh1zX_oqIoKNefuSr1ngvgAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LambdaForm$DMH"], "Stackframe.function.name": ["void java.lang.invoke.LambdaForm$DMH+0x00000008026fc000.invokeInterface(java.lang.Object, java.lang.Object, java.lang.Object, java.lang.Object, int, java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "FSvf6nmXZySKV-TedM62ngAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileParser._finishLongSlow()"], "Stackframe.line.number": [2196], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAFw9n"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["audit_copy_inode"]} +{"create": {"_index": "profiling-stackframes", "_id": "3-Nyd5iyxL31wUMeSpG3wAAAAAAAAAAK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PendingDeletes.java"], "Stackframe.function.name": ["int org.apache.lucene.index.PendingDeletes.numDeletesToMerge(org.apache.lucene.index.MergePolicy, org.apache.lucene.util.IOSupplier)"], "Stackframe.line.number": [238], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALmmh"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["path_openat"]} +{"create": {"_index": "profiling-stackframes", "_id": "48AhUVUmIzx4uY_a_nqGPwAAAAAAAAAe"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CodecUtil.java"], "Stackframe.function.name": ["void org.apache.lucene.codecs.CodecUtil.writeBEInt(org.apache.lucene.store.DataOutput, int)"], "Stackframe.line.number": [656], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "KO6VsFDaVRMvLVZz6LumQAAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixPath.java"], "Stackframe.function.name": ["java.nio.file.Path sun.nio.fs.UnixPath.getName(int)"], "Stackframe.line.number": [43], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "8AUzmHhqUZOHWH8Bccl_sQAAAAAAAAHc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Coordinator.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishWithJoinResponse org.elasticsearch.cluster.coordination.Coordinator.handlePublishRequest(org.elasticsearch.cluster.coordination.PublishRequest)"], "Stackframe.line.number": [437], "Stackframe.function.offset": [51]} +{"create": {"_index": "profiling-stackframes", "_id": "wEIfl0pGBuepuUcTwdKc2gAAAAAAAKGd"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["./inflate.c"], "Stackframe.function.name": ["inflate"], "Stackframe.line.number": [1077]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAAEIG"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["do_syscall_64"]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAimtb"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["sock_sendmsg"]} +{"create": {"_index": "profiling-stackframes", "_id": "ip4jveo5OesPxqJEnXrM0wAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLSocketImpl.java"], "Stackframe.function.name": ["sun.security.ssl.Plaintext sun.security.ssl.SSLSocketImpl.decode(java.nio.ByteBuffer)"], "Stackframe.line.number": [1510], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "cDTcKVGzeMo1oRYERHbXHwAAAAAAAAB5"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReplicationTracker.java"], "Stackframe.function.name": ["org.elasticsearch.index.shard.ReplicationGroup org.elasticsearch.index.seqno.ReplicationTracker.calculateReplicationGroup()"], "Stackframe.line.number": [1007], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "4pUL3Hn4ak1IOxEqUVFkHAAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["RBACEngine.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.RBACEngine.lambda$resolveAuthorizationInfo$0(org.elasticsearch.action.ActionListener, org.elasticsearch.core.Tuple)"], "Stackframe.line.number": [149], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAFBtp"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["futex_wake"]} +{"create": {"_index": "profiling-stackframes", "_id": "KxHEggHQvpMo7NXk9m-aKgAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void com.amazonaws.http.conn.$Proxy62.connect(org.apache.http.HttpClientConnection, org.apache.http.conn.routing.HttpRoute, int, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ovSFLiAX5U3zPkxnICf4iQAAAAAAAAAR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SnapshotShardsService.java"], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService.snapshot(org.elasticsearch.index.shard.ShardId, org.elasticsearch.snapshots.Snapshot, org.elasticsearch.repositories.IndexId, org.elasticsearch.index.snapshots.IndexShardSnapshotStatus, org.elasticsearch.Version, long, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [349], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "IvW6glp0EMhdH986Hu8MmgAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IntegerPolynomialP256.java"], "Stackframe.function.name": ["void sun.security.util.math.intpoly.IntegerPolynomialP256.carryReduce(long[], long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long)"], "Stackframe.line.number": [81], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "S3fVfiwq94i14_VfWgkFiAAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ActionRunnable.java"], "Stackframe.function.name": ["void org.elasticsearch.action.ActionRunnable$2.accept(org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [50], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "RrXasmojz0ojjKVFAz-eswAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessController.java"], "Stackframe.function.name": ["java.security.AccessControlContext java.security.AccessController.getStackAccessControlContext()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "HBnk-r2jmt48gRTNgMOf8wAAAAAAAAAl"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService$1.onResponse(java.lang.Void)"], "Stackframe.line.number": [601], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "4P1DMygHaI4yHD8SKHtROwAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["URLClassPath.java"], "Stackframe.function.name": ["void jdk.internal.loader.URLClassPath.check(java.net.URL)"], "Stackframe.line.number": [559], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "zUqGzEtASB77lr6F9wlBfAAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["JsonXContentParser.java"], "Stackframe.function.name": ["long org.elasticsearch.xcontent.provider.json.JsonXContentParser.doLongValue()"], "Stackframe.line.number": [223], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "4Wda4DEj07iGWB9mKOvWMAAAAAAAAABo"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DirectMethodHandleAccessor.java"], "Stackframe.function.name": ["java.lang.Object jdk.internal.reflect.DirectMethodHandleAccessor.invokeImpl(java.lang.Object, java.lang.Object[])"], "Stackframe.line.number": [158], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "kbApFhDbueqDTdsjbbOMUQAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FSDirectory.java"], "Stackframe.function.name": ["void org.apache.lucene.store.FSDirectory.sync(java.util.Collection)"], "Stackframe.line.number": [286], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "Y2eGzIpiKa-QBMAAKXZA_QAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["OutputStreamIndexOutput.java"], "Stackframe.function.name": ["void org.apache.lucene.store.OutputStreamIndexOutput.writeLong(long)"], "Stackframe.line.number": [77], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "fMf02bFW9UOsHXaY1ju3UAAAAAAAAAEL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["com.amazonaws.services.s3.model.S3Object com.amazonaws.services.s3.AmazonS3Client.getObject(com.amazonaws.services.s3.model.GetObjectRequest)"], "Stackframe.line.number": [1524], "Stackframe.function.offset": [40]} +{"create": {"_index": "profiling-stackframes", "_id": "56q5trA0bAF1B-Um6L_rqwAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.callClusterStateAppliers(org.elasticsearch.cluster.ClusterChangedEvent, org.elasticsearch.cluster.service.ClusterApplierRecordingService$Recorder)"], "Stackframe.line.number": [539], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "XJAoY26CmEBCHOgy-kaybgAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityManager.java"], "Stackframe.function.name": ["void java.lang.SecurityManager.checkWrite(java.lang.String)"], "Stackframe.line.number": [842], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "sNHGncE46bxEYqLkiTNaMgAAAAAAAADJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportReplicationAction.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.TransportReplicationAction$AsyncPrimaryAction.doRun()"], "Stackframe.line.number": [411], "Stackframe.function.offset": [29]} +{"create": {"_index": "profiling-stackframes", "_id": "mafpkLzxyAOwBxGkDhiVNAAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["InboundHandler.java"], "Stackframe.function.name": ["void org.elasticsearch.transport.InboundHandler$1.doRun()"], "Stackframe.line.number": [260], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Z3m1S883R8QNRDHePf66cAAAAAAAAAAD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractFileSystemProvider.java"], "Stackframe.function.name": ["void sun.nio.fs.AbstractFileSystemProvider.delete(java.nio.file.Path)"], "Stackframe.line.number": [105], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "NIZuECXcBG2pC3vk7GF71gAAAAAAAAAe"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FSDirectory.java"], "Stackframe.function.name": ["void org.apache.lucene.store.FSDirectory.deleteFile(java.lang.String)"], "Stackframe.line.number": [311], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "83pPtFJ0Pe4c5bSskvCkMAAAAAAAAABn"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ReplicationOperation.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.ReplicationOperation.execute()"], "Stackframe.line.number": [125], "Stackframe.function.offset": [18]} +{"create": {"_index": "profiling-stackframes", "_id": "lt4DCYuCTF_qGLbdRnIhjAAAAAAAAADD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLConnectionSocketFactory.java"], "Stackframe.function.name": ["java.net.Socket org.apache.http.conn.ssl.SSLConnectionSocketFactory.connectSocket(int, java.net.Socket, org.apache.http.HttpHost, java.net.InetSocketAddress, java.net.InetSocketAddress, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [384], "Stackframe.function.offset": [29]} +{"create": {"_index": "profiling-stackframes", "_id": "NaokQDqfrb6dyCBpANIkNwAAAAAAAAEv"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLSocketOutputRecord.java"], "Stackframe.function.name": ["void sun.security.ssl.SSLSocketOutputRecord.deliver(byte[], int, int)"], "Stackframe.line.number": [345], "Stackframe.function.offset": [57]} +{"create": {"_index": "profiling-stackframes", "_id": "tGxEGlv356rmSfx4IkmLcQAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractChannelHandlerContext.java"], "Stackframe.function.name": ["io.netty.channel.ChannelHandlerContext io.netty.channel.AbstractChannelHandlerContext.flush()"], "Stackframe.line.number": [893], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "Re_dvjV79ezcmCpSsdxZuAAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PhantomCleanable.java"], "Stackframe.function.name": ["void jdk.internal.ref.PhantomCleanable.clean()"], "Stackframe.line.number": [133], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "8__nCHyreLVA7r_BF4RklwAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3BlobContainer$$Lambda$8687+0x0000000802083390.run()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "2yAqVPrM2wZm88ulDRIkCgAAAAAAAACE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.serialize(org.elasticsearch.xcontent.ToXContent, java.lang.String, boolean, java.util.Map, java.io.OutputStream)"], "Stackframe.line.number": [332], "Stackframe.function.offset": [26]} +{"create": {"_index": "profiling-stackframes", "_id": "akj2EbMcV1BY1x-2V-R6VQAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NamedGroup.java"], "Stackframe.function.name": ["sun.security.ssl.SSLPossession sun.security.ssl.NamedGroup$NamedGroupSpec.createPossession(sun.security.ssl.NamedGroup, java.security.SecureRandom)"], "Stackframe.line.number": [520], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "wEIfl0pGBuepuUcTwdKc2gAAAAAAADW4"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["", "", "./deflate.c", "./deflate.c", "./deflate.c", "./deflate.c"], "Stackframe.function.name": ["fill_window", "fill_window", "fill_window", "slide_hash", "slide_hash", "slide_hash"], "Stackframe.line.number": [0, 0, 1580, 1580, 1580, 226]} +{"create": {"_index": "profiling-stackframes", "_id": "EiWa0n_9SgJD34zKIyVJDAAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Path.java"], "Stackframe.function.name": ["java.nio.file.Path java.nio.file.Path.resolve(java.lang.String)"], "Stackframe.line.number": [515], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "h6nIgR7I9-N28cV-Wv3K3AAAAAAAAAFB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixChannelFactory.java"], "Stackframe.function.name": ["java.io.FileDescriptor sun.nio.fs.UnixChannelFactory.open(int, sun.nio.fs.UnixPath, java.lang.String, sun.nio.fs.UnixChannelFactory$Flags, int)"], "Stackframe.line.number": [258], "Stackframe.function.offset": [71]} +{"create": {"_index": "profiling-stackframes", "_id": "bOTvPcD8pPP1fG6-CUmg6wAAAAAAAAGv"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeHelper()"], "Stackframe.line.number": [1139], "Stackframe.function.offset": [61]} +{"create": {"_index": "profiling-stackframes", "_id": "IQvpHpp20NHD-0mZNf95oAAAAAAAAABp"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LinkedTransferQueue.java"], "Stackframe.function.name": ["java.lang.Object java.util.concurrent.LinkedTransferQueue.xfer(java.lang.Object, boolean, int, long)"], "Stackframe.line.number": [605], "Stackframe.function.offset": [10]} +{"create": {"_index": "profiling-stackframes", "_id": "wvgldxPJJj8FcIi5eLlQvgAAAAAAAAAm"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileParser.java"], "Stackframe.function.name": ["java.lang.String com.fasterxml.jackson.dataformat.smile.SmileParser.getText()"], "Stackframe.line.number": [996], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "-FiXU0afkg-sRwubj2WiAwAAAAAAAiUs"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["", "../../../bus/signals.c", "../../../bus/signals.c", "../../../bus/signals.c"], "Stackframe.function.name": ["get_recipients_from_list", "get_recipients_from_list", "match_rule_matches", "match_rule_matches"], "Stackframe.line.number": [0, 2093, 2093, 1797]} +{"create": {"_index": "profiling-stackframes", "_id": "Pu2JWw096K4wEF2usuCt-AAAAAAAAABc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PoolingHttpClientConnectionManager.java"], "Stackframe.function.name": ["void org.apache.http.impl.conn.PoolingHttpClientConnectionManager.connect(org.apache.http.HttpClientConnection, org.apache.http.conn.routing.HttpRoute, int, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [376], "Stackframe.function.offset": [13]} +{"create": {"_index": "profiling-stackframes", "_id": "MRjz7ydeslXKi2GbhltLmwAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PendingSoftDeletes.java"], "Stackframe.function.name": ["int org.apache.lucene.index.PendingSoftDeletes.numDeletesToMerge(org.apache.lucene.index.MergePolicy, org.apache.lucene.util.IOSupplier)"], "Stackframe.line.number": [195], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "rXmzLUA4HH3snMulWKOyNgAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SessionOutputBufferImpl.java"], "Stackframe.function.name": ["void org.apache.http.impl.io.SessionOutputBufferImpl.flushBuffer()"], "Stackframe.line.number": [136], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "Td592FDxi26zl_r4mKuaXAAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NativePRNG.java"], "Stackframe.function.name": ["void sun.security.provider.NativePRNG.engineNextBytes(byte[])"], "Stackframe.line.number": [221], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAimwC"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["sock_write_iter"]} +{"create": {"_index": "profiling-stackframes", "_id": "rFCnpw1umelBJZQ8bfB_gwAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SnapshotShardsService.java"], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService.lambda$newShardSnapshotTask$2(org.elasticsearch.index.shard.ShardId, org.elasticsearch.snapshots.Snapshot, org.elasticsearch.repositories.IndexId, org.elasticsearch.index.snapshots.IndexShardSnapshotStatus, org.elasticsearch.Version, long)"], "Stackframe.line.number": [276], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "H1mK93wmygFvNYC57fovfwAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileChannel.java"], "Stackframe.function.name": ["java.nio.channels.FileChannel java.nio.channels.FileChannel.open(java.nio.file.Path, java.nio.file.OpenOption[])"], "Stackframe.line.number": [363], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "U3XGmp9XAuDQti4O3QZItgAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedOutputStream.java"], "Stackframe.function.name": ["void java.io.BufferedOutputStream.implWrite(byte[], int, int)"], "Stackframe.line.number": [216], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "OYpTnRQxT_yM-wTPYnG49wAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileDispatcherImpl.java"], "Stackframe.function.name": ["int sun.nio.ch.FileDispatcherImpl.force(java.io.FileDescriptor, boolean)"], "Stackframe.line.number": [82], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "RooPjsQ01TWhvqGm4rD4pAAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["NativePRNG.java"], "Stackframe.function.name": ["void sun.security.provider.NativePRNG$RandomIO.implNextBytes(byte[])"], "Stackframe.line.number": [537], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "DIud9aCnVVpiy1IhYas8ZwAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FilterDirectory.java"], "Stackframe.function.name": ["void org.apache.lucene.store.FilterDirectory.deleteFile(java.lang.String)"], "Stackframe.line.number": [65], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "_2Sq1jeupJuJ-QB3TmIqcwAAAAAAAADX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["java.lang.Object com.amazonaws.services.s3.AmazonS3Client.invoke(com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, java.lang.String, java.lang.String, boolean)"], "Stackframe.line.number": [5054], "Stackframe.function.offset": [46]} +{"create": {"_index": "profiling-stackframes", "_id": "AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8q"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["StubRoutines (1)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "5tsfFOiB0XQGtxfawg0JiQAAAAAAAAGP"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.executeOneRequest(com.amazonaws.http.AmazonHttpClient$RequestExecutor$ExecOneRequestParams)"], "Stackframe.line.number": [1346], "Stackframe.function.offset": [63]} +{"create": {"_index": "profiling-stackframes", "_id": "nVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadContext.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingRunnable.run()"], "Stackframe.line.number": [718], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "P4ZjI7g37fWz45cghUwTxQAAAAAAAAA5"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IOUtils.java"], "Stackframe.function.name": ["void org.apache.lucene.util.IOUtils.fsync(java.nio.file.Path, boolean)"], "Stackframe.line.number": [465], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "KPNYkONEetadZox7ZPj-_wAAAAAAAACa"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SnapshotShardsService.java"], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService.snapshot(org.elasticsearch.index.shard.ShardId, org.elasticsearch.snapshots.Snapshot, org.elasticsearch.repositories.IndexId, java.util.Map, org.elasticsearch.index.snapshots.IndexShardSnapshotStatus, org.elasticsearch.Version, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [377], "Stackframe.function.offset": [26]} +{"create": {"_index": "profiling-stackframes", "_id": "lLBPRDZhOsSZ0SPWLgVYwQAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLEngineImpl.java"], "Stackframe.function.name": ["sun.security.ssl.Ciphertext sun.security.ssl.SSLEngineImpl.encode(java.nio.ByteBuffer[], int, int, java.nio.ByteBuffer[], int, int)"], "Stackframe.line.number": [305], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "OX6pd1GZ1KHmZAikJOZD-wAAAAAAAABo"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixChannelFactory.java"], "Stackframe.function.name": ["java.nio.channels.FileChannel sun.nio.fs.UnixChannelFactory.newFileChannel(int, sun.nio.fs.UnixPath, java.lang.String, java.util.Set, int)"], "Stackframe.line.number": [133], "Stackframe.function.offset": [17]} +{"create": {"_index": "profiling-stackframes", "_id": "a_D_KA1yUInCMlw0HKKJXgAAAAAAAABX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadContext.java"], "Stackframe.function.name": ["org.elasticsearch.common.util.concurrent.ThreadContext$StoredContext org.elasticsearch.common.util.concurrent.ThreadContext.newTraceContext()"], "Stackframe.line.number": [163], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "FOYm6EawRDxcLqPYr2TlYwAAAAAAAACT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SequenceNumbers.java"], "Stackframe.function.name": ["org.elasticsearch.index.seqno.SequenceNumbers$CommitInfo org.elasticsearch.index.seqno.SequenceNumbers.loadSeqNoInfoFromLuceneCommit(java.lang.Iterable)"], "Stackframe.line.number": [53], "Stackframe.function.offset": [12]} +{"create": {"_index": "profiling-stackframes", "_id": "gA6euexXfLm-c-5bdW_pngAAAAAAAAAx"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CompositeRolesStore.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.store.CompositeRolesStore.lambda$getRoles$1(org.elasticsearch.xpack.core.security.authc.Authentication, org.elasticsearch.action.ActionListener, org.elasticsearch.xpack.core.security.authz.permission.Role)"], "Stackframe.line.number": [201], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "ZQ0AxrpV-kvXnFTqilFT7wAAAAAAAAAf"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileChannel.java"], "Stackframe.function.name": ["java.nio.channels.FileChannel java.nio.channels.FileChannel.open(java.nio.file.Path, java.nio.file.OpenOption[])"], "Stackframe.line.number": [357], "Stackframe.function.offset": [6]} +{"create": {"_index": "profiling-stackframes", "_id": "vweqY124jM8xgz5HvRfnbAAAAAAAAAA9"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadPoolExecutor.java"], "Stackframe.function.name": ["void java.util.concurrent.ThreadPoolExecutor.execute(java.lang.Runnable)"], "Stackframe.line.number": [1357], "Stackframe.function.offset": [28]} +{"create": {"_index": "profiling-stackframes", "_id": "eNITicG0MvFr9HQHk70FLAAAAAAAAAAI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["EsThreadPoolExecutor.java"], "Stackframe.function.name": ["void org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor.execute(java.lang.Runnable)"], "Stackframe.line.number": [95], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "PeFsynzCKRZawnN184w4uwAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractChannelHandlerContext.java"], "Stackframe.function.name": ["void io.netty.channel.AbstractChannelHandlerContext.invokeWriteAndFlush(java.lang.Object, io.netty.channel.ChannelPromise)"], "Stackframe.line.number": [941], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "xhzTSPDm8vP1CtKm1pHa_AAAAAAAAADX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonS3Client.java"], "Stackframe.function.name": ["java.lang.Object com.amazonaws.services.s3.AmazonS3Client.invoke(com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, java.lang.String, java.lang.String, boolean)"], "Stackframe.line.number": [5456], "Stackframe.function.offset": [45]} +{"create": {"_index": "profiling-stackframes", "_id": "bFStJdQtuBtluMM9Zd1OwQAAAAAAAABc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshots.java"], "Stackframe.function.name": ["org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots.findPhysicalIndexFile(org.elasticsearch.index.store.StoreFileMetadata)"], "Stackframe.line.number": [142], "Stackframe.function.offset": [7]} +{"create": {"_index": "profiling-stackframes", "_id": "dfk3O1V70lDk9OWRJ4O2zgAAAAAAAABC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["OutputRecord.java"], "Stackframe.function.name": ["long sun.security.ssl.OutputRecord.encrypt(sun.security.ssl.SSLCipher$SSLWriteCipher, byte, java.nio.ByteBuffer, int, int, int, sun.security.ssl.ProtocolVersion)"], "Stackframe.line.number": [341], "Stackframe.function.offset": [13]} +{"create": {"_index": "profiling-stackframes", "_id": "0dBKXDWNMiejfPcFa2J8WwAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileDescriptor.java"], "Stackframe.function.name": ["void java.io.FileDescriptor.close()"], "Stackframe.line.number": [297], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "fbCakSHDHoBMTyuJNDB7CwAAAAAAAAAx"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedInputStream.java"], "Stackframe.function.name": ["int java.io.BufferedInputStream.read(byte[], int, int)"], "Stackframe.line.number": [343], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "q3uedK83ZBGp21KqIIcD7QAAAAAAAACW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LambdaForm$MH"], "Stackframe.function.name": ["java.lang.Object java.lang.invoke.LambdaForm$MH+0x0000000802a2a000.invoke(java.lang.Object, java.lang.Object, java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Brq2tMeYvo-7GC1rkF8BOwAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.snapshots.SnapshotShardsService$$Lambda$10696+0x00000008029733b8.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "CQI8nGGXTZm2Anxc2rVzlAAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient.execute(com.amazonaws.Request, com.amazonaws.http.HttpResponseHandler, com.amazonaws.http.HttpResponseHandler, com.amazonaws.http.ExecutionContext)"], "Stackframe.line.number": [524], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "oGQ2OUdACCpDvWCPN-t91QAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexTemplateRegistry.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.core.template.IndexTemplateRegistry.addTemplatesIfMissing(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [204], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "V_QMdmt4RxKxn4ZNgdvkJwAAAAAAAAAR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PendingReplicationActions.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.PendingReplicationActions.cancelActions(java.util.ArrayList, java.lang.String)"], "Stackframe.line.number": [108], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "W2G52eIffs6rSMZaDokh0AAAAAAAAAAT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexTemplateConfig.java"], "Stackframe.function.name": ["byte[] org.elasticsearch.xpack.core.template.IndexTemplateConfig.loadBytes()"], "Stackframe.line.number": [91], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "FHxMx1_4SqiAeAbrZHJ7sQAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileGenerator.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileGenerator.writeStringField(java.lang.String, java.lang.String)"], "Stackframe.line.number": [534], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "nfWxBNCjXXCl3tAETF7T8wAAAAAAAAAU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["XContentBuilder.java"], "Stackframe.function.name": ["org.elasticsearch.xcontent.XContentBuilder org.elasticsearch.xcontent.XContentBuilder.field(java.lang.String, java.lang.String)"], "Stackframe.line.number": [730], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "Mrj6Z8YTvyj7p7UD3GGpMgAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Deflater.java"], "Stackframe.function.name": ["int java.util.zip.Deflater.deflate(byte[], int, int)"], "Stackframe.line.number": [462], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "C7Gr60ZG8XU9_R3ucXNk1wAAAAAAAAEr"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.deserialize(java.lang.String, org.elasticsearch.xcontent.NamedXContentRegistry, java.io.InputStream)"], "Stackframe.line.number": [165], "Stackframe.function.offset": [35]} +{"create": {"_index": "profiling-stackframes", "_id": "ImSzkW6pD8jhY2FJThRJ8QAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ActionRunnable.java"], "Stackframe.function.name": ["void org.elasticsearch.action.ActionRunnable$2.accept(java.lang.Object)"], "Stackframe.line.number": [47], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "VdDRdK9-3VxhlLY5l0wZhQAAAAAAAAGh"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLEngineImpl.java"], "Stackframe.function.name": ["javax.net.ssl.SSLEngineResult sun.security.ssl.SSLEngineImpl.writeRecord(java.nio.ByteBuffer[], int, int, java.nio.ByteBuffer[], int, int)"], "Stackframe.line.number": [251], "Stackframe.function.offset": [83]} +{"create": {"_index": "profiling-stackframes", "_id": "kRX6qZbTW9c5D6S8BjNVUQAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexModule.java"], "Stackframe.function.name": ["org.elasticsearch.index.engine.Engine$IndexCommitRef org.elasticsearch.index.IndexModule.lambda$static$1(org.elasticsearch.index.engine.Engine)"], "Stackframe.line.number": [626], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "GKHjZXGfLxTKi_lPdb3kRgAAAAAAAABV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileChannelImpl.java"], "Stackframe.function.name": ["void sun.nio.ch.FileChannelImpl.force(boolean)"], "Stackframe.line.number": [531], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "ds9yTvk8UzKAgDSjs6pNkgAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3BlobContainer.java"], "Stackframe.function.name": ["java.io.InputStream org.elasticsearch.repositories.s3.S3BlobContainer.readBlob(java.lang.String)"], "Stackframe.line.number": [95], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "k4geTYpM8uDz6HvI1jT73gAAAAAAAAAZ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ChecksumBlobStoreFormat.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.write(java.lang.Object, org.elasticsearch.common.blobstore.BlobContainer, java.lang.String, boolean, java.util.Map)"], "Stackframe.line.number": [346], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "ebuxNWQVKJLKmMPt8wkOJwAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixNativeDispatcher.java"], "Stackframe.function.name": ["void sun.nio.fs.UnixNativeDispatcher.lstat(sun.nio.fs.UnixPath, sun.nio.fs.UnixFileAttributes)"], "Stackframe.line.number": [356], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "e_qwG8gpgd38JnWYrQTbhgAAAAAAAABT"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileGenerator.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileGenerator.writeString(java.lang.String)"], "Stackframe.line.number": [1025], "Stackframe.function.offset": [27]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAADtss"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["complete"]} +{"create": {"_index": "profiling-stackframes", "_id": "YkaRTKRhsYL87aTctpzuoQAAAAAAAACw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["void org.apache.lucene.index.IndexWriter.merge(org.apache.lucene.index.MergePolicy$OneMerge)"], "Stackframe.line.number": [4704], "Stackframe.function.offset": [38]} +{"create": {"_index": "profiling-stackframes", "_id": "Vo9H_8YeBmc0_i3CIsMkrwAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixNativeDispatcher.java"], "Stackframe.function.name": ["int sun.nio.fs.UnixNativeDispatcher.open0(long, int, int)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "xpXYHQL7vamCXb04_96hlgAAAAAAAAAL"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DigestBase.java"], "Stackframe.function.name": ["void sun.security.provider.DigestBase.engineReset()"], "Stackframe.line.number": [175], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "cMPxRfIRSUAO9TVG-M8JpAAAAAAAAAB0"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository.snapshotFile(org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo, org.elasticsearch.repositories.IndexId, org.elasticsearch.index.shard.ShardId, org.elasticsearch.snapshots.SnapshotId, org.elasticsearch.index.snapshots.IndexShardSnapshotStatus, org.elasticsearch.index.store.Store)"], "Stackframe.line.number": [3493], "Stackframe.function.offset": [31]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAp1q0"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__lock_text_start"]} +{"create": {"_index": "profiling-stackframes", "_id": "c2BpwrDKQx3haZVDD56LgAAAAAAAAABs"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SdkHttpRequestExecutor.java"], "Stackframe.function.name": ["org.apache.http.HttpResponse com.amazonaws.http.protocol.SdkHttpRequestExecutor.doSendRequest(org.apache.http.HttpRequest, org.apache.http.HttpClientConnection, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [63], "Stackframe.function.offset": [19]} +{"create": {"_index": "profiling-stackframes", "_id": "4j8yS0qsC_6XfTfMdPp5KQAAAAAAAAAQ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService$UpdateTask.run()"], "Stackframe.line.number": [154], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "UJmIMEc6K8gJ39NHYaD0WwAAAAAAAhyG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["../csu/libc-start.c"], "Stackframe.function.name": ["__libc_start_main"], "Stackframe.line.number": [310]} +{"create": {"_index": "profiling-stackframes", "_id": "9j0yGbd8eQNwdRhHZ159OQAAAAAAAAA9"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadPoolExecutor.java"], "Stackframe.function.name": ["void java.util.concurrent.ThreadPoolExecutor.execute(java.lang.Runnable)"], "Stackframe.line.number": [1357], "Stackframe.function.offset": [28]} +{"create": {"_index": "profiling-stackframes", "_id": "VTe2bxq-Jr5s7wFSht9_vgAAAAAAAABq"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["PublicationTransportHandler.java"], "Stackframe.function.name": ["org.elasticsearch.cluster.coordination.PublishWithJoinResponse org.elasticsearch.cluster.coordination.PublicationTransportHandler.acceptState(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [245], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "jxIGTrGHMnImsEvjAP6NuwAAAAAAAAEY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Boolean2ScorerSupplier.java"], "Stackframe.function.name": ["org.apache.lucene.search.Scorer org.apache.lucene.search.Boolean2ScorerSupplier.req(java.util.Collection, java.util.Collection, long)"], "Stackframe.line.number": [210], "Stackframe.function.offset": [47]} +{"create": {"_index": "profiling-stackframes", "_id": "IvW6glp0EMhdH986Hu8MmgAAAAAAAAN7"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IntegerPolynomialP256.java"], "Stackframe.function.name": ["void sun.security.util.math.intpoly.IntegerPolynomialP256.carryReduce(long[], long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long, long)"], "Stackframe.line.number": [165], "Stackframe.function.offset": [84]} +{"create": {"_index": "profiling-stackframes", "_id": "uaLlXo_jail-f0Vd8f8fCwAAAAAAAACD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["GatewayMetaState.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.GatewayMetaState$LucenePersistedState.setLastAcceptedState(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [593], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "n_CesQ5pN3dpPkarM3dScAAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["TransportReplicationAction.java"], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.TransportReplicationAction$AsyncPrimaryAction.lambda$doRun$0(org.elasticsearch.index.shard.IndexShard, org.elasticsearch.core.Releasable)"], "Stackframe.line.number": [414], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "S4xo2K-7S9Psi5KVtkSU0AAAAAAAAAAO"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["CloseableHttpClient.java"], "Stackframe.function.name": ["org.apache.http.client.methods.CloseableHttpResponse org.apache.http.impl.client.CloseableHttpClient.execute(org.apache.http.client.methods.HttpUriRequest, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [83], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "OoUyVxHN2rYPjLirZK4ihQAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["long org.apache.lucene.index.IndexWriter.prepareCommit()"], "Stackframe.line.number": [3293], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "8NHfQtcyI14VYOjZC2v_gwAAAAAAAABc"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadPoolExecutor.java"], "Stackframe.function.name": ["void java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)"], "Stackframe.line.number": [1144], "Stackframe.function.offset": [20]} +{"create": {"_index": "profiling-stackframes", "_id": "9oBnE4xnAvOiOv1q-LbApgAAAAAAAAAE"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierService.access$000(org.elasticsearch.cluster.service.ClusterApplierService, java.lang.String, java.util.function.Function, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [56], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "LeaqrhFv22rb1wzUlEMwAgAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["LockValidatingDirectoryWrapper.java"], "Stackframe.function.name": ["void org.apache.lucene.store.LockValidatingDirectoryWrapper.sync(java.util.Collection)"], "Stackframe.line.number": [68], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "L4TTplFq0_54QvTmQIK7TgAAAAAAAAAS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreRepository.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.blobstore.BlobStoreRepository.lambda$executeOneFileSnapshot$78(org.elasticsearch.index.store.Store, org.elasticsearch.index.snapshots.IndexShardSnapshotStatus, org.elasticsearch.index.shard.ShardId, org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo, org.elasticsearch.repositories.IndexId, org.elasticsearch.snapshots.SnapshotId, java.util.concurrent.BlockingQueue, java.util.concurrent.Executor, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [2938], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "ihMpsHacOmWht-v_3Vk1hQAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BuiltinClassLoader.java"], "Stackframe.function.name": ["java.net.URL jdk.internal.loader.BuiltinClassLoader.lambda$findResourceOnClassPath$0(java.lang.String)"], "Stackframe.line.number": [539], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "O57ePcWMLnVMB2c1JYP_SgAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshot.java"], "Stackframe.function.name": ["void org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo.toXContent(org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot$FileInfo, org.elasticsearch.xcontent.XContentBuilder, org.elasticsearch.xcontent.ToXContent$Params)"], "Stackframe.line.number": [256], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "KACWjtDKKsY3WzryMLMuQgAAAAAAAACN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.doExecute()"], "Stackframe.line.number": [796], "Stackframe.function.offset": [19]} +{"create": {"_index": "profiling-stackframes", "_id": "NJPFmv3cYB4fnHHuI2lrYQAAAAAAAABb"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3RetryingInputStream.java"], "Stackframe.function.name": ["void org.elasticsearch.repositories.s3.S3RetryingInputStream.(org.elasticsearch.repositories.s3.S3BlobStore, java.lang.String, long, long)"], "Stackframe.line.number": [76], "Stackframe.function.offset": [23]} +{"create": {"_index": "profiling-stackframes", "_id": "P7LRUTUpfWcPCMm1bmRUrgAAAAAAAAAY"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.ServerTransportFilter$$Lambda$6914+0x0000000802433568.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "ezRRC2sXv2hHIZbqwpv5zQAAAAAAAADS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexShard.java"], "Stackframe.function.name": ["void org.elasticsearch.index.shard.IndexShard.updateShardState(org.elasticsearch.cluster.routing.ShardRouting, long, java.util.function.BiConsumer, long, java.util.Set, org.elasticsearch.cluster.routing.IndexShardRoutingTable)"], "Stackframe.line.number": [535], "Stackframe.function.offset": [24]} +{"create": {"_index": "profiling-stackframes", "_id": "hqzAzTCpj-Nsh96skuh10wAAAAAAAABJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["HandshakeContext.java"], "Stackframe.function.name": ["void sun.security.ssl.HandshakeContext.dispatch(byte, java.nio.ByteBuffer)"], "Stackframe.line.number": [480], "Stackframe.function.offset": [15]} +{"create": {"_index": "profiling-stackframes", "_id": "IY3kOFBxddrJzoZrzargTAAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FileDescriptor.java"], "Stackframe.function.name": ["void java.io.FileDescriptor.close0()"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "HJHMeViEmhpbdveA5zS4zgAAAAAAAADU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ServerHelloDone.java"], "Stackframe.function.name": ["void sun.security.ssl.ServerHelloDone$ServerHelloDoneConsumer.consume(sun.security.ssl.ConnectionContext, java.nio.ByteBuffer)"], "Stackframe.line.number": [182], "Stackframe.function.offset": [48]} +{"create": {"_index": "profiling-stackframes", "_id": "lTaOxl3xTWrzN2-LwynoLwAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SHA.java"], "Stackframe.function.name": ["void sun.security.provider.SHA.implReset()"], "Stackframe.line.number": [85], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "Lhf7WGU0lxCaIsujYZjafAAAAAAAAAAS"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedOutputStream.java"], "Stackframe.function.name": ["void java.io.BufferedOutputStream.write(byte[], int, int)"], "Stackframe.line.number": [199], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "LqiOes3kysZtBc8totbsBAAAAAAAAAAA"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["UnixNativeDispatcher.java"], "Stackframe.function.name": ["void sun.nio.fs.UnixNativeDispatcher.lstat0(long, sun.nio.fs.UnixFileAttributes)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "_gZdvwQP6FQ5RcMsyYjAXwAAAAAAAAFD"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["DefaultHttpClientConnectionOperator.java"], "Stackframe.function.name": ["void org.apache.http.impl.conn.DefaultHttpClientConnectionOperator.connect(org.apache.http.conn.ManagedHttpClientConnection, org.apache.http.HttpHost, java.net.InetSocketAddress, int, org.apache.http.config.SocketConfig, org.apache.http.protocol.HttpContext)"], "Stackframe.line.number": [142], "Stackframe.function.offset": [36]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAMRzj"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["__x64_sys_fsync"]} +{"create": {"_index": "profiling-stackframes", "_id": "8T1GTOmmtW2zNfDAiA-RCgAAAAAAAADH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler.messageReceived(org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.tasks.Task)"], "Stackframe.line.number": [415], "Stackframe.function.offset": [48]} +{"create": {"_index": "profiling-stackframes", "_id": "FpTtdOGq0KA_LZU93WcT2gAAAAAAAAAs"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BufferedInputStream.java"], "Stackframe.function.name": ["int java.io.BufferedInputStream.read1(byte[], int, int)"], "Stackframe.line.number": [284], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "FC-FZ1oAaXNFzXvQCjtC_AAAAAAAAAAU"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLEngine.java"], "Stackframe.function.name": ["javax.net.ssl.SSLEngineResult javax.net.ssl.SSLEngine.wrap(java.nio.ByteBuffer[], java.nio.ByteBuffer)"], "Stackframe.line.number": [564], "Stackframe.function.offset": [3]} +{"create": {"_index": "profiling-stackframes", "_id": "RJOpyLKTKlv2enKt_xrNSwAAAAAAAAA7"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.authorizeSystemUser(org.elasticsearch.xpack.core.security.authc.Authentication, java.lang.String, java.lang.String, org.elasticsearch.transport.TransportRequest, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [620], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "aSV2qTlMpiialisnhZW9PgAAAAAAAAAN"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AccessController.java"], "Stackframe.function.name": ["java.lang.Object java.security.AccessController.doPrivileged(java.security.PrivilegedAction, java.security.AccessControlContext)"], "Stackframe.line.number": [399], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "becvgDELr5B5ZYyfqmyRzQAAAAAAAACq"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler.messageReceived(org.elasticsearch.transport.TransportRequest, org.elasticsearch.transport.TransportChannel, org.elasticsearch.tasks.Task)"], "Stackframe.line.number": [601], "Stackframe.function.offset": [48]} +{"create": {"_index": "profiling-stackframes", "_id": "rjyp8aib7sZjDzshnnXJQQAAAAAAAABB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ContentLengthOutputStream.java"], "Stackframe.function.name": ["void org.apache.http.impl.io.ContentLengthOutputStream.write(byte[], int, int)"], "Stackframe.line.number": [113], "Stackframe.function.offset": [9]} +{"create": {"_index": "profiling-stackframes", "_id": "3ITJpMVvGWxtAJg8iCjlKgAAAAAAAAAh"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SmileGenerator.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.dataformat.smile.SmileGenerator._flushBuffer()"], "Stackframe.line.number": [2647], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "6aHBr7IoSaYRBHN60TsSvgAAAAAAAACC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["long org.apache.lucene.index.IndexWriter.commitInternal(org.apache.lucene.index.MergePolicy)"], "Stackframe.line.number": [4051], "Stackframe.function.offset": [25]} +{"create": {"_index": "profiling-stackframes", "_id": "3HIAZ7KMFykIwp4WdvYF6gAAAAAAAAEV"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IndexWriter.java"], "Stackframe.function.name": ["org.apache.lucene.index.MergePolicy$MergeSpecification org.apache.lucene.index.IndexWriter.updatePendingMerges(org.apache.lucene.index.MergePolicy, org.apache.lucene.index.MergeTrigger, int)"], "Stackframe.line.number": [2349], "Stackframe.function.offset": [45]} +{"create": {"_index": "profiling-stackframes", "_id": "uE354xX1-6pq5qEqEPux5gAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ConcurrentMergeScheduler.java"], "Stackframe.function.name": ["void org.apache.lucene.index.ConcurrentMergeScheduler.doMerge(org.apache.lucene.index.MergeScheduler$MergeSource, org.apache.lucene.index.MergePolicy$OneMerge)"], "Stackframe.line.number": [639], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "35BPGLqYcVRpRMhVtTI1-AAAAAAAAAAJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ActionListener.java"], "Stackframe.function.name": ["void org.elasticsearch.action.ActionListener$DelegatingFailureActionListener.onResponse(java.lang.Object)"], "Stackframe.line.number": [250], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "CKOgldhpJkxI6GxtBnkMFwAAAAAAAABI"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ClusterApplierRecordingService.java"], "Stackframe.function.name": ["void org.elasticsearch.cluster.service.ClusterApplierRecordingService.updateStats(org.elasticsearch.cluster.service.ClusterApplierRecordingService$Recorder)"], "Stackframe.line.number": [52], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "nrAGsit1vNwpXNBDsxW-CAAAAAAAAAAF"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SecurityServerTransportInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.transport.SecurityServerTransportInterceptor$ProfileSecuredRequestHandler$3.onResponse(java.lang.Object)"], "Stackframe.line.number": [579], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "jqNXsTtyxyukYQAy6-O-GwAAAAAAAAFy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthorizationService.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.AuthorizationService.handleIndexActionAuthorizationResult(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$IndexAuthorizationResult, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, java.lang.String, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AsyncSupplier, org.elasticsearch.cluster.metadata.Metadata, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [582], "Stackframe.function.offset": [64]} +{"create": {"_index": "profiling-stackframes", "_id": "UI6vy2YkxZpNhPhBVD9zrQAAAAAAAAAB"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AmazonHttpClient.java"], "Stackframe.function.name": ["com.amazonaws.Response com.amazonaws.http.AmazonHttpClient$RequestExecutor.access$500(com.amazonaws.http.AmazonHttpClient$RequestExecutor)"], "Stackframe.line.number": [698], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "JA3L5oTr7kSGnFgD6CVpBwAAAAAAAAAM"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": [""], "Stackframe.function.name": ["void org.elasticsearch.action.support.replication.TransportReplicationAction$AsyncPrimaryAction$$Lambda$8956+0x00000008027eb370.accept(java.lang.Object)"], "Stackframe.line.number": [0], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "WAsi5KS1LX4-mmePSPN4qwAAAAAAAABz"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["GatewayMetaState.java"], "Stackframe.function.name": ["void org.elasticsearch.gateway.GatewayMetaState$LucenePersistedState.setLastAcceptedState(org.elasticsearch.cluster.ClusterState)"], "Stackframe.line.number": [506], "Stackframe.function.offset": [11]} +{"create": {"_index": "profiling-stackframes", "_id": "087ujtUEat__kjA2FRsgeQAAAAAAAAAK"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FSDirectory.java"], "Stackframe.function.name": ["void org.apache.lucene.store.FSDirectory.privateDeleteFile(java.lang.String, boolean)"], "Stackframe.line.number": [346], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "DFSw6fr_Dyh89ExFUOBWuwAAAAAAAABm"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AuthenticatorChain.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authc.AuthenticatorChain.authenticateAsync(org.elasticsearch.xpack.security.authc.Authenticator$Context, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [93], "Stackframe.function.offset": [21]} +{"create": {"_index": "profiling-stackframes", "_id": "zP5HzbxY7goeRVwj37voeQAAAAAAAAAC"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLSocketImpl.java"], "Stackframe.function.name": ["void sun.security.ssl.SSLSocketImpl.startHandshake()"], "Stackframe.line.number": [426], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "tupDcGbWHEIVe-Q0MXArXgAAAAAAAAAG"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ShardSearchRequestInterceptor.java"], "Stackframe.function.name": ["void org.elasticsearch.xpack.security.authz.interceptor.ShardSearchRequestInterceptor.intercept(org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$RequestInfo, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine, org.elasticsearch.xpack.core.security.authz.AuthorizationEngine$AuthorizationInfo, org.elasticsearch.action.ActionListener)"], "Stackframe.line.number": [24], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "BUXwXn0jTEljgr89rND6CgAAAAAAAAAw"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["IntegerPolynomial.java"], "Stackframe.function.name": ["sun.security.util.math.intpoly.IntegerPolynomial$MutableElement sun.security.util.math.intpoly.IntegerPolynomial$MutableElement.setProduct(sun.security.util.math.IntegerModuloP)"], "Stackframe.line.number": [698], "Stackframe.function.offset": [2]} +{"create": {"_index": "profiling-stackframes", "_id": "7SvDCQFg8noiOfIrUqyslQAAAAAAAAAJ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["HashMap.java"], "Stackframe.function.name": ["java.lang.Object java.util.HashMap.put(java.lang.Object, java.lang.Object)"], "Stackframe.line.number": [618], "Stackframe.function.offset": [0]} +{"create": {"_index": "profiling-stackframes", "_id": "XqZ8EmLPraAc01S4qiJfWgAAAAAAAAAZ"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["ThreadContext.java"], "Stackframe.function.name": ["org.elasticsearch.common.util.concurrent.ThreadContext$StoredContext org.elasticsearch.common.util.concurrent.ThreadContext.stashContext()"], "Stackframe.line.number": [114], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "wUUkbH4v-_I9fBBV0L_DSwAAAAAAAAAg"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BuiltinClassLoader.java"], "Stackframe.function.name": ["java.net.URL jdk.internal.loader.BuiltinClassLoader.findResourceOnClassPath(java.lang.String)"], "Stackframe.line.number": [540], "Stackframe.function.offset": [5]} +{"create": {"_index": "profiling-stackframes", "_id": "w-t0BUQhbl_nV8h8YiyiEQAAAAAAAAA1"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SSLEngineImpl.java"], "Stackframe.function.name": ["javax.net.ssl.SSLEngineResult sun.security.ssl.SSLEngineImpl.wrap(java.nio.ByteBuffer[], int, int, java.nio.ByteBuffer[], int, int)"], "Stackframe.line.number": [146], "Stackframe.function.offset": [15]} +{"create": {"_index": "profiling-stackframes", "_id": "V-zEfMlNdgG4SvC62RGySgAAAAAAAAAX"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["AbstractInterruptibleChannel.java"], "Stackframe.function.name": ["void java.nio.channels.spi.AbstractInterruptibleChannel.close()"], "Stackframe.line.number": [112], "Stackframe.function.offset": [4]} +{"create": {"_index": "profiling-stackframes", "_id": "j7E103DHicBcnyrMzev20gAAAAAAAAAx"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["Cipher.java"], "Stackframe.function.name": ["int javax.crypto.Cipher.getOutputSize(int)"], "Stackframe.line.number": [1030], "Stackframe.function.offset": [8]} +{"create": {"_index": "profiling-stackframes", "_id": "MDZqk-bznCkkPcSl3Z06LwAAAAAAAACR"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["FieldsIndexWriter.java"], "Stackframe.function.name": ["void org.apache.lucene.codecs.lucene90.compressing.FieldsIndexWriter.finish(int, long, org.apache.lucene.store.IndexOutput)"], "Stackframe.line.number": [121], "Stackframe.function.offset": [14]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAAAEAT"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["syscall_slow_exit_work"]} +{"create": {"_index": "profiling-stackframes", "_id": "92n0tgLpue4WLe7eq6vpuwAAAAAAAAAW"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["S3BlobContainer.java"], "Stackframe.function.name": ["java.lang.Object org.elasticsearch.repositories.s3.S3BlobContainer.lambda$writeBlob$1(long, java.lang.String, java.io.InputStream)"], "Stackframe.line.number": [128], "Stackframe.function.offset": [1]} +{"create": {"_index": "profiling-stackframes", "_id": "lHp5_WAgpLy2alrUVab6HAAAAAAALTpU"}} +{"ecs.version": "1.12.0", "Stackframe.function.name": ["new_sync_write"]} +{"create": {"_index": "profiling-stackframes", "_id": "KZn_I-ZtnojJ7fIDZHHneQAAAAAAAADy"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["BlobStoreIndexShardSnapshots.java"], "Stackframe.function.name": ["org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots.fromXContent(org.elasticsearch.xcontent.XContentParser)"], "Stackframe.line.number": [294], "Stackframe.function.offset": [34]} +{"create": {"_index": "profiling-stackframes", "_id": "c8gmQiPjvAnokd-rAm4_OQAAAAAAAABH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["SessionOutputBufferImpl.java"], "Stackframe.function.name": ["void org.apache.http.impl.io.SessionOutputBufferImpl.write(byte[], int, int)"], "Stackframe.line.number": [167], "Stackframe.function.offset": [17]} +{"create": {"_index": "profiling-stackframes", "_id": "hWcFfmcayZnqxV8VaktDEwAAAAAAAAAH"}} +{"ecs.version": "1.12.0", "Stackframe.file.name": ["JsonGenerator.java"], "Stackframe.function.name": ["void com.fasterxml.jackson.core.JsonGenerator.writeStringField(java.lang.String, java.lang.String)"], "Stackframe.line.number": [2187], "Stackframe.function.offset": [1]} diff --git a/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-stacktraces.ndjson b/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-stacktraces.ndjson new file mode 100644 index 0000000000000..23774e9b2b43b --- /dev/null +++ b/x-pack/plugin/profiling/src/internalClusterTest/resources/data/profiling-stacktraces.ndjson @@ -0,0 +1,88 @@ +{"create": {"_index": "profiling-stacktraces", "_id": "p0FpNwvinkonmYTrP_m8AQ"}} +{"Stacktrace": {"frame": {"ids": "SA0fsx010mzWc4t8ff1-lAAAAAAABCaZUJmIMEc6K8gJ39NHYaD0WwAAAAAAAhyGSA0fsx010mzWc4t8ff1-lAAAAAAABBdGSA0fsx010mzWc4t8ff1-lAAAAAAABWWLSA0fsx010mzWc4t8ff1-lAAAAAAABO5fSA0fsx010mzWc4t8ff1-lAAAAAAACbHCSA0fsx010mzWc4t8ff1-lAAAAAAABFnNVNoyxdugL4O4pEtN-lEJQwAAAAAADKVLVNoyxdugL4O4pEtN-lEJQwAAAAAADKAFVNoyxdugL4O4pEtN-lEJQwAAAAAADHogVNoyxdugL4O4pEtN-lEJQwAAAAAAD0DTVNoyxdugL4O4pEtN-lEJQwAAAAAADtnd", "types": "DAM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "V-R5JWu8vu2g_Vodg6Kd6w"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABck8v4vce5CPjZaoKS8wiSZwAAAAAAAAAVBqEVAd8MPrtkYhJgO4GhZAAAAAAAAAAEepNUCeEe4JglhIlXIVE76AAAAAAAAAAG7TfJIA5ukUl_URVlhByP5QAAAAAAAAAu7MyQRgXoynWLAIL3MMBcgwAAAAAAAAAEeJ7mihRgQcJvejyU6kLDVAAAAAAAAAAcrFCnpw1umelBJZQ8bfB_gwAAAAAAAAAWovSFLiAX5U3zPkxnICf4iQAAAAAAAAARbnYc3RJmtKwpcpofJJsHfwAAAAAAAAACBrq2tMeYvo-7GC1rkF8BOwAAAAAAAAAgCOij2gsV-4lHDIB8-jTbygAAAAAAAACGnMqxSWeol-RN_p8hXE5oZQAAAAAAAAAQ7HvWKt-xH9wdfCiPfTMv1gAAAAAAAAAIYRdREBgpcH5AEXcyjzcstgAAAAAAAAABkRX6qZbTW9c5D6S8BjNVUQAAAAAAAAACTnhL_a5gaR0GbLDTAKZU8gAAAAAAAAATYhErxbFc2y82dI3FdqHUOwAAAAAAAADs0vu-8-EP1KjTNjFoZrojbgAAAAAAAAAh15Kl_LNlY1wkBO4-dVuptAAAAAAAAAAM6aHBr7IoSaYRBHN60TsSvgAAAAAAAACCx88iiyZQ8B1lQnhTbyrEkwAAAAAAAACNr21mW5wIwvlwHfvj4Leh8wAAAAAAAACosmN6GZa9-v-IcDUWnhvNkAAAAAAAAABLvs-DizK6zOgT7PLk55Hi8wAAAAAAAABgkBc-zardyidmU9M2FtS_7wAAAAAAAACUpT78-BAaZ59fFXwdquC9KQAAAAAAAAAFHTWghk6sgAJ14g_dZIR1agAAAAAAAAAMu6cCfBpCKfnm1TY7pHx6ZgAAAAAAAAAEayyYx9DIcjy4NGNqpq88GQAAAAAAAAAPDIud9aCnVVpiy1IhYas8ZwAAAAAAAAAFRO5fkjSwhs-w1RFeE1q0WQAAAAAAAAACDIud9aCnVVpiy1IhYas8ZwAAAAAAAAAFNIZuECXcBG2pC3vk7GF71gAAAAAAAAAe087ujtUEat__kjA2FRsgeQAAAAAAAAAKBJKRobhikTEVTM_TJ5G21wAAAAAAAAAFZ3m1S883R8QNRDHePf66cAAAAAAAAAADweat5ldYNeQ9Grv2_1-CAwAAAAAAAAAOCgEj6Rp3n26KLxmeHzazOAAAAAAAAAAWebuxNWQVKJLKmMPt8wkOJwAAAAAAAAAOLqiOes3kysZtBc8totbsBAAAAAAAAAAASVqTovndX3vgWwiNPDrUZAAAAAAAAM4TqjB_amR59Lv13Nhle_NwbwAAAAAAENV5lHp5_WAgpLy2alrUVab6HAAAAAAAwACLlHp5_WAgpLy2alrUVab6HAAAAAAAAEIGlHp5_WAgpLy2alrUVab6HAAAAAAALb_llHp5_WAgpLy2alrUVab6HAAAAAAALb-clHp5_WAgpLy2alrUVab6HAAAAAAALbjVlHp5_WAgpLy2alrUVab6HAAAAAAALpFFlHp5_WAgpLy2alrUVab6HAAAAAAALpAYlHp5_WAgpLy2alrUVab6HAAAAAAAF0JUlHp5_WAgpLy2alrUVab6HAAAAAAAFw9nlHp5_WAgpLy2alrUVab6HAAAAAAARLUslHp5_WAgpLy2alrUVab6HAAAAAAAMCXYQTXwwuqN0mLb-8lqckiaTQAAAAAACHlyQTXwwuqN0mLb-8lqckiaTQAAAAAAAWojQTXwwuqN0mLb-8lqckiaTQAAAAAAB4DjQTXwwuqN0mLb-8lqckiaTQAAAAAAB4A6lHp5_WAgpLy2alrUVab6HAAAAAAApz47", "types": "CAMsBQIDEAQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "evah_aPn6ywh3ieU3e4MDQ"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAACuyaSy1RHb0tl4hOkUXXLXQQAAAAAAAAAzGUwipo03N4glyS6268EEmgAAAAAAAAAXC7Gr60ZG8XU9_R3ucXNk1wAAAAAAAAErrLErn8W8_gFwO7zvGO-LBwAAAAAAAAAIOUMeXZZ6fO-t9RTzPCiRtQAAAAAAAAABKZn_I-ZtnojJ7fIDZHHneQAAAAAAAACFzV-LQYJm41zGMkfXUp2AHgAAAAAAAAF_QzYQ9wb1jS08S2BufSVqWAAAAAAAAAAC0GU5IksNSYIYre39EnRVtQAAAAAAAAAczUqGzEtASB77lr6F9wlBfAAAAAAAAAAEy3hT9dbM11gdF2x7xJH8oQAAAAAAAAARiZ3mYpUZ_vDzUycli_tCcQAAAAAAAAAuWE9wZHWs5kC-tWUiTpjMyQAAAAAAAAA20Pih7Ldp-yyaIKsINSnPvQAAAAAAAAAfFSvf6nmXZySKV-TedM62ngAAAAAAAAABt5nOOkOTN6QC2_E5A_LICgAAAAAAAAAQl4dVCOTlRJdBiyQ88Tx8OQAAAAAAAABR-FIPITFlgylPOavKEh5SEQAAAAAAAAAB4w-7ZRIHk2tM9RfD6v6f4gAAAAAAAAAVI9AUeiUn_SK_Dgdkbb3mdAAAAAAAAAASnTSx0380z3H9OoIBkTGK2wAAAAAAAAAxBjycB5yXrYLMnUcjhbPq2gAAAAAAAAAnU2mkS0m3fBX-awRgTvpBLAAAAAAAAAAlCsbq98qEMjLfznv89zxcQwAAAAAAAABASlb8MNVKIFfE9HJqtN3BhwAAAAAAAAAAhrCIM-i5cm0sSd-oVTZrsgAAAAAAAC8kwEIfl0pGBuepuUcTwdKc2gAAAAAAAKGdwEIfl0pGBuepuUcTwdKc2gAAAAAAAJF2", "types": "CAMmBQMD"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "NPliZYuvPv2y5wnRJfOutw"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAACuyaSy1RHb0tl4hOkUXXLXQQAAAAAAAAAzGUwipo03N4glyS6268EEmgAAAAAAAAAKds9yTvk8UzKAgDSjs6pNkgAAAAAAAAANzVmgO3uV6bZtx2XUjAkXcgAAAAAAAAAHNJPFmv3cYB4fnHHuI2lrYQAAAAAAAABbsh1T4lVaqf9yZbH1lhPrZwAAAAAAAACNduOfgYTG-Pz3qxfosjBOKgAAAAAAAAAEFTD_DMq9LCr2rMh4XAWQegAAAAAAAAAF5wGUH60A7iQqyWZA8jj7kwAAAAAAAAAdDPKZG86WCe-rkFTx_IAdagAAAAAAAAAIhCl4jaZsLnxk3C0-qCjENwAAAAAAAAAFfMf02bFW9UOsHXaY1ju3UAAAAAAAAAELie51PKalSPz0oTw25hOjdgAAAAAAAAAHxhzTSPDm8vP1CtKm1pHa_AAAAAAAAADX3SU4n_eVvbLOxW0zu9gonAAAAAAAAAATBULXlXluMevM23wGn-scGgAAAAAAAABIX3YYuA6U4OM2ZjQu8fVwYwAAAAAAAABLSNDYMaokxZpNhjFZnC0cqQAAAAAAAAABm_DK0nAVx1tMIimjFYyQPAAAAAAAAAAS_B6c9784WlLvFF9uk3pSswAAAAAAAAAcFhP6JufLZYYhuADwCLvRKAAAAAAAAACVClvIuFYr5ArIOKSAaLGqggAAAAAAAAGv5tsfFOiB0XQGtxfawg0JiQAAAAAAAAGPe8ARrqT9JmZcuLd6G8ZTMgAAAAAAAAAGV6Xv8yqxVPgM9JZWMvD_ogAAAAAAAAADS4xo2K-7S9Psi5KVtkSU0AAAAAAAAAAONNX6LKmc60fQ-V_Nw2_9cAAAAAAAAACwVGJu-bYJBqeYrugO5COQdQAAAAAAAAHGAxJNlcXBvrgVOpR8a1WEYgAAAAAAAAG2n0mWG92K0zbUz5S-_ixlIAAAAAAAAABrKxHEggHQvpMo7NXk9m-aKgAAAAAAAAAghvijQK1a5iiWkFoVp40y5AAAAAAAAAAGvRwDdjzKXpreYxi90MDGQQAAAAAAAABms-5AzRQS0AyjARcxs2D2DQAAAAAAAAAX4Wda4DEj07iGWB9mKOvWMAAAAAAAAABo45faa5L8TTwA9zJYlmGKywAAAAAAAAAUq3uedK83ZBGp21KqIIcD7QAAAAAAAACWwh1zX_oqIoKNefuSr1ngvgAAAAAAAAAfPu2JWw096K4wEF2usuCt-AAAAAAAAABc_gZdvwQP6FQ5RcMsyYjAXwAAAAAAAAFD1PIPTBbtPHUrg0d3N2nuygAAAAAAAABBlt4DCYuCTF_qGLbdRnIhjAAAAAAAAADDFERPJCfrzkqs-7XhU53jhwAAAAAAAAFtzP5HzbxY7goeRVwj37voeQAAAAAAAAACy87oVbGEsnZxcfXcdrLYswAAAAAAAAB6i1t0pdr_xnHfk_UsEnpDiwAAAAAAAAAMip4jveo5OesPxqJEnXrM0wAAAAAAAAAOrDli4oCQ8GWnljRH2_1bLAAAAAAAAAEZH2XMIGvXG_73d5XzBvFSxAAAAAAAAAEB3D_A_sI8XDgpK1lgEmfWZgAAAAAAAAC_hqzAzTCpj-Nsh96skuh10wAAAAAAAABJibtF619pQHGNKblUfb9MKwAAAAAAAAANHJHMeViEmhpbdveA5zS4zgAAAAAAAADUNW0MvXdcfyNB_rjxH_JHXgAAAAAAAAAN16j8HtYPNjd3qdRoXnMi0AAAAAAAAABaO4wSxfOE-c6Sft1wmb_6fgAAAAAAAABpoKMpPO_rrguOLYx8VBTGhgAAAAAAAAAGakj2EbMcV1BY1x-2V-R6VQAAAAAAAAANqND66z7potQhxSJ1H4ezHgAAAAAAAAAG2B-rbS8KhbB8ePRnpVf7_QAAAAAAAAAU89ogc5uTbOqh2aLJlzHKmAAAAAAAAAAX9ccfl0YdabzwSUY2PdYA0QAAAAAAAAATDT06D5A7Yxiqx6IJ7Kvu0wAAAAAAAACCtq710yUF9bzkGXjQwM8_6QAAAAAAAADWH7Tmwg9rKZuxl8eAfe0cQwAAAAAAAAATpKAwMR-9-3afO3tccFzRKwAAAAAAAAIXbYAsS36pm68v9ebHURyhdAAAAAAAAAACBUXwXn0jTEljgr89rND6CgAAAAAAAAAwR_ZB9SjojSrHSQ8dwIq_WQAAAAAAAAOrIvW6glp0EMhdH986Hu8MmgAAAAAAAAAA", "types": "CANTBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "VxcDhJCIVlMC063NXB1S1Q"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAANGUEIf0h4CmbF3WXaliniYgAAAAAAAAAL6bg5h6o_GLUtahiXyigwlQAAAAAAAAAsJy9mUBVZJITLUNGFd8QS0QAAAAAAAAFJX5HtFQ851Qd03wmhWrZR4QAAAAAAAAAqMs8C2PsOPTwJ_BWU-NqKjwAAAAAAAAABXC-En-mvMkLnH_EQYbHTPwAAAAAAAAABDsEcu-FxsA-5RKaWB1grdAAAAAAAAAApPeFsynzCKRZawnN184w4uwAAAAAAAAAOfo33a47SQa_ikvx14Wo6GwAAAAAAAAAmTF5EMbS9G90c1yPz7xfKpAAAAAAAAAAChEBKqh2xU-RorWUVO4ewtQAAAAAAAADttGxEGlv356rmSfx4IkmLcQAAAAAAAAAWtqJQIOovDd8UazKau6rQMwAAAAAAAAAIfo33a47SQa_ikvx14Wo6GwAAAAAAAAAmzvnrbT9TB1hdLy7i_u1ShgAAAAAAAAAltGxEGlv356rmSfx4IkmLcQAAAAAAAAAWtqJQIOovDd8UazKau6rQMwAAAAAAAAAIfo33a47SQa_ikvx14Wo6GwAAAAAAAAAxDjDYSMBm5cQnwKCRH0dlgwAAAAAAAAA0obEloCxQqTOnivGYEKZEEAAAAAAAAAAuAE1Q7tv3S3rfQh4l5ABt6AAAAAAAAABnB8SjIPdc3ZwlcMQD5LrUqAAAAAAAAACOFC-FZ1oAaXNFzXvQCjtC_AAAAAAAAAAUnxbylxGsMi-XNEec__mC2gAAAAAAAAAPw-t0BUQhbl_nV8h8YiyiEQAAAAAAAAA1VdDRdK9-3VxhlLY5l0wZhQAAAAAAAAGhlLBPRDZhOsSZ0SPWLgVYwQAAAAAAAAAQVlTLE-LbC-qAF2rlk-BxLQAAAAAAAABIZVMl7SvWgt8_FxFL79kGYgAAAAAAAAHmdfk3O1V70lDk9OWRJ4O2zgAAAAAAAABCT2JZOvbQqWZFxul4uMvPzgAAAAAAAABw5VGPPNmqbrejwy3AxU1wmQAAAAAAAAB5j7E103DHicBcnyrMzev20gAAAAAAAAAxeWbRBA5HjtSejwiG_37xQwAAAAAAAAABv5Q60hHLUFC78bfvz1WyQQAAAAAAAAAA", "types": "CAMlBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "-lrLeM0iQSsI3WnVH08B3w"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAANVk4geTYpM8uDz6HvI1jT73gAAAAAAAAAZHc4fbH8wnl-aZPA1_7gB-wAAAAAAAAAyaBWoLwWk5u6kqgD3gwKyagAAAAAAAAAYluqtQRsIjOPWY7wdgiD4bAAAAAAAAAAIwRG5B2klLAxXtt0B9lVulAAAAAAAAADSeBJz8rf8VBkiNSfy4HLlcwAAAAAAAAAMYcHA99kMtpR5AAkygC-qMAAAAAAAAAAgPvkUluf_nN5PcG5RTjFmTgAAAAAAAAAG48AhUVUmIzx4uY_a_nqGPwAAAAAAAAAeXjeOrvBx98FSmUHFqJE5JAAAAAAAAAAA", "types": "CAMXBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "XF9MchOwpePfa6_hYy-vZQ"}} +{"Stacktrace": {"frame": {"ids": "634wiWh6F21tPpXr0Zz3mgAAAAAAEfFi8NlMClggx8jaziUTJXlmWAAAAAAAAIYIZSkKN3zNxr0HYuO2pqe5hQAAAAAAwcBwZSkKN3zNxr0HYuO2pqe5hQAAAAAA5ECvZSkKN3zNxr0HYuO2pqe5hQAAAAAA4_9_ZSkKN3zNxr0HYuO2pqe5hQAAAAAAj7b5ZSkKN3zNxr0HYuO2pqe5hQAAAAAAgwXKZSkKN3zNxr0HYuO2pqe5hQAAAAAAgu3UAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABch4dwrMYlRFRjyfsvjXt4tgAAAAAAAAAg3V-8FLy1GH8nVRceMDeaiwAAAAAAAAABnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAV4j8yS0qsC_6XfTfMdPp5KQAAAAAAAAAQ9oBnE4xnAvOiOv1q-LbApgAAAAAAAAAEwRQstrBYZ0ShmJnYV-ADrQAAAAAAAAFLAFikCbtP_Dm7iUthjnlnEgAAAAAAAAGtTfYUtN-rf_0uYWSYAHKfDQAAAAAAAAAH4O-ovD5bG6j4SStsVkgzcQAAAAAAAABHacnJzC_C1bsmTPMH_pcsCAAAAAAAAABdoGQ2OUdACCpDvWCPN-t91QAAAAAAAAAMagUnXAr2_hsl_qyYsSOrqQAAAAAAAABcIP-4kS9K2z9dstopMSzdEAAAAAAAAAAKW2G52eIffs6rSMZaDokh0AAAAAAAAAATZUblhfsP3eghXbFQlRo7WwAAAAAAAAABmwuQyMHK5RYUe_aaboQONgAAAAAAAAADyScC9Npb_ya8Br_QJ7EUZAAAAAAAAAB7Sm149Q4n4lGGxDiZNf94FQAAAAAAAAAHPUPItXL9ODmL0kwnalRigwAAAAAAAAARPUPItXL9ODmL0kwnalRigwAAAAAAAAARPUPItXL9ODmL0kwnalRigwAAAAAAAAAj77BcdPidwQf8JepqsYlnCAAAAAAAAACPwUUkbH4v-_I9fBBV0L_DSwAAAAAAAAAgFTD_DMq9LCr2rMh4XAWQegAAAAAAAAAF5wGUH60A7iQqyWZA8jj7kwAAAAAAAAAdIGvGOb8CKDtqOos6VI5kCAAAAAAAAAAIihMpsHacOmWht-v_3Vk1hQAAAAAAAAAG8iRqFHNeeA-Vq5mM-jaESgAAAAAAAAAG", "types": "CAMgBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "SsxQM9_HdP1iq1X4sNsK_Q"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAACuyaSy1RHb0tl4hOkUXXLXQQAAAAAAAAAzGUwipo03N4glyS6268EEmgAAAAAAAAAKds9yTvk8UzKAgDSjs6pNkgAAAAAAAAANzVmgO3uV6bZtx2XUjAkXcgAAAAAAAAAHNJPFmv3cYB4fnHHuI2lrYQAAAAAAAABbsh1T4lVaqf9yZbH1lhPrZwAAAAAAAACNduOfgYTG-Pz3qxfosjBOKgAAAAAAAAAEFTD_DMq9LCr2rMh4XAWQegAAAAAAAAAF5wGUH60A7iQqyWZA8jj7kwAAAAAAAAAdDPKZG86WCe-rkFTx_IAdagAAAAAAAAAIhCl4jaZsLnxk3C0-qCjENwAAAAAAAAAFfMf02bFW9UOsHXaY1ju3UAAAAAAAAAELie51PKalSPz0oTw25hOjdgAAAAAAAAAHxhzTSPDm8vP1CtKm1pHa_AAAAAAAAADX3SU4n_eVvbLOxW0zu9gonAAAAAAAAAATBULXlXluMevM23wGn-scGgAAAAAAAABIX3YYuA6U4OM2ZjQu8fVwYwAAAAAAAABLSNDYMaokxZpNhjFZnC0cqQAAAAAAAAABm_DK0nAVx1tMIimjFYyQPAAAAAAAAAAS_B6c9784WlLvFF9uk3pSswAAAAAAAAAcFhP6JufLZYYhuADwCLvRKAAAAAAAAACVClvIuFYr5ArIOKSAaLGqggAAAAAAAAGv5tsfFOiB0XQGtxfawg0JiQAAAAAAAAGPe8ARrqT9JmZcuLd6G8ZTMgAAAAAAAAAGV6Xv8yqxVPgM9JZWMvD_ogAAAAAAAAADS4xo2K-7S9Psi5KVtkSU0AAAAAAAAAAONNX6LKmc60fQ-V_Nw2_9cAAAAAAAAACwVGJu-bYJBqeYrugO5COQdQAAAAAAAAHGAxJNlcXBvrgVOpR8a1WEYgAAAAAAAAG2n0mWG92K0zbUz5S-_ixlIAAAAAAAAABrKxHEggHQvpMo7NXk9m-aKgAAAAAAAAAghvijQK1a5iiWkFoVp40y5AAAAAAAAAAGvRwDdjzKXpreYxi90MDGQQAAAAAAAABms-5AzRQS0AyjARcxs2D2DQAAAAAAAAAX4Wda4DEj07iGWB9mKOvWMAAAAAAAAABo45faa5L8TTwA9zJYlmGKywAAAAAAAAAUq3uedK83ZBGp21KqIIcD7QAAAAAAAACWwh1zX_oqIoKNefuSr1ngvgAAAAAAAAAfPu2JWw096K4wEF2usuCt-AAAAAAAAABc_gZdvwQP6FQ5RcMsyYjAXwAAAAAAAAFD1PIPTBbtPHUrg0d3N2nuygAAAAAAAABBlt4DCYuCTF_qGLbdRnIhjAAAAAAAAADDFERPJCfrzkqs-7XhU53jhwAAAAAAAAFtzP5HzbxY7goeRVwj37voeQAAAAAAAAACy87oVbGEsnZxcfXcdrLYswAAAAAAAAB6i1t0pdr_xnHfk_UsEnpDiwAAAAAAAAAMip4jveo5OesPxqJEnXrM0wAAAAAAAAAOrDli4oCQ8GWnljRH2_1bLAAAAAAAAAEZH2XMIGvXG_73d5XzBvFSxAAAAAAAAAEB3D_A_sI8XDgpK1lgEmfWZgAAAAAAAAC_hqzAzTCpj-Nsh96skuh10wAAAAAAAABJibtF619pQHGNKblUfb9MKwAAAAAAAAANHJHMeViEmhpbdveA5zS4zgAAAAAAAADUNW0MvXdcfyNB_rjxH_JHXgAAAAAAAAAN16j8HtYPNjd3qdRoXnMi0AAAAAAAAABaO4wSxfOE-c6Sft1wmb_6fgAAAAAAAABpoKMpPO_rrguOLYx8VBTGhgAAAAAAAAAGakj2EbMcV1BY1x-2V-R6VQAAAAAAAAANqND66z7potQhxSJ1H4ezHgAAAAAAAAAG2B-rbS8KhbB8ePRnpVf7_QAAAAAAAAAU89ogc5uTbOqh2aLJlzHKmAAAAAAAAAAX9ccfl0YdabzwSUY2PdYA0QAAAAAAAAATDT06D5A7Yxiqx6IJ7Kvu0wAAAAAAAACCtq710yUF9bzkGXjQwM8_6QAAAAAAAADWH7Tmwg9rKZuxl8eAfe0cQwAAAAAAAAATpKAwMR-9-3afO3tccFzRKwAAAAAAAABwbYAsS36pm68v9ebHURyhdAAAAAAAAAACBUXwXn0jTEljgr89rND6CgAAAAAAAAAwR_ZB9SjojSrHSQ8dwIq_WQAAAAAAAAOrIvW6glp0EMhdH986Hu8MmgAAAAAAAAN74jtFwV4I0VFxZ633CiXAEQAAAAAAAAAA", "types": "CANUBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "AIcHF1g0UqebNhdXerG6rQ"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnyhAKi3MXCBa6QbMykpz9ogAAAAAAAAAIImSzkW6pD8jhY2FJThRJ8QAAAAAAAAAFS3fVfiwq94i14_VfWgkFiAAAAAAAAAAF86ShTOKEhMBNNZND3RBl5QAAAAAAAAABd8JoiANFYdN_HlsRPvshMwAAAAAAAAAxIz1h-lJR9hawlSRETyh1qQAAAAAAAAAIeOepQqcpphM0NTbieX0wIgAAAAAAAAE4ooplCYR951bf9t1YS6OUBQAAAAAAAACrWAsi5KS1LX4-mmePSPN4qwAAAAAAAABzUnU61_VzRQulrS0OJYkU7gAAAAAAAABaHJWDExCp8dmw9-CAF-WXfwAAAAAAAAAJoxSW321mwoDVUa0haFaP7QAAAAAAAAAuTyZK4BUMsPg3P3wu6EYhkAAAAAAAAAB6PiSwsMqyjn24GnxXwBvfuQAAAAAAAAAGzY8oFom0Mq0j4EO6ho2wLAAAAAAAAAMEeC3OrXu_7VLA9EdCZMRh6AAAAAAAAAGdLeaqrhFv22rb1wzUlEMwAgAAAAAAAAAM4FrN4VeCu4HJZNavjw0tmgAAAAAAAAAgmSb0V8ANbBWpi7TZTXcPNwAAAAAAAAALP4ZjI7g37fWz45cghUwTxQAAAAAAAAA_GKHjZXGfLxTKi_lPdb3kRgAAAAAAAABVOYpTnRQxT_yM-wTPYnG49wAAAAAAAAAC_OOeQuiXtvgynmE43QMbEAAAAAAAAAAASVqTovndX3vgWwiNPDrUZAAAAAAAAHzEvQ7EacG6CR4rzcFaNFEsTQAAAAAAATk6lHp5_WAgpLy2alrUVab6HAAAAAAAwACLlHp5_WAgpLy2alrUVab6HAAAAAAAAEIGlHp5_WAgpLy2alrUVab6HAAAAAAAMRzjlHp5_WAgpLy2alrUVab6HAAAAAAAMRyclHp5_WAgpLy2alrUVab6HAAAAAAAMRwHQTXwwuqN0mLb-8lqckiaTQAAAAAABml_QTXwwuqN0mLb-8lqckiaTQAAAAAACEMYlHp5_WAgpLy2alrUVab6HAAAAAAATgeqlHp5_WAgpLy2alrUVab6HAAAAAAATXEilHp5_WAgpLy2alrUVab6HAAAAAAAIgIAlHp5_WAgpLy2alrUVab6HAAAAAAAIgE0lHp5_WAgpLy2alrUVab6HAAAAAAAKgJ7", "types": "CAMdBQIDDAQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "Ce77w10WeIDow3kd1jowlA"}} +{"Stacktrace": {"frame": {"ids": "634wiWh6F21tPpXr0Zz3mgAAAAAAEfFi8NlMClggx8jaziUTJXlmWAAAAAAAAIYIZSkKN3zNxr0HYuO2pqe5hQAAAAAAwcBwZSkKN3zNxr0HYuO2pqe5hQAAAAAA5ECvZSkKN3zNxr0HYuO2pqe5hQAAAAAA4_9_ZSkKN3zNxr0HYuO2pqe5hQAAAAAAj7b5ZSkKN3zNxr0HYuO2pqe5hQAAAAAAgwXKZSkKN3zNxr0HYuO2pqe5hQAAAAAAgu3UAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAVcBTlmvRfiK_eaWEDctBQ5wAAAAAAAAAMjLRjRKJujxpIzkv5EAU8CAAAAAAAAADEKPNYkONEetadZox7ZPj-_wAAAAAAAACojPaG_bLbWLTHMVbsZ6KO3wAAAAAAAAPSX-q4xdk6PYPLAgZGMSlgiwAAAAAAAAAZHH3S5b5wm0-iwMj6LUThxAAAAAAAAAAyTqPaoMp2He0IHYVdVGyXigAAAAAAAAAY7wrMCA1tiwccLQqKJr9yEwAAAAAAAAAI2yAqVPrM2wZm88ulDRIkCgAAAAAAAACE5t8GiksjQcuLCsu8jyRsaAAAAAAAAAA2jojK5E5-Tpf57iS2qx9_2QAAAAAAAAA2rHKWkGh7pRChoM8BKsgaigAAAAAAAAAUwlgkZprTs7F_QEEM-7VvZwAAAAAAAAAGFHxMx1_4SqiAeAbrZHJ7sQAAAAAAAAAYi0MJsjg1nTMYg21Lh8j5pAAAAAAAAABTKtHBJ2R0vuuTX50F8kgGAgAAAAAAAAAhVDYfO2UIwBGcuJQmu3YTtwAAAAAAAAAU_eHurMk1AuWI20vPBelYUQAAAAAAAABRReKfR-5xbP5LM4-DWXoUigAAAAAAAAAOMrj6Z8YTvyj7p7UD3GGpMgAAAAAAAAAFCNW3Uw4RnzkMwEIUegsYPwAAAAAAAACFkpBi8LFR3RTjSJqFuljOuQAAAAAAAAAAGXH6S9Nv2Lf0omTz4cH4RAAAAAAAAClRfhsEKXDuxJ-jIJrZpdRuSAAAAAAAAFtjfhsEKXDuxJ-jIJrZpdRuSAAAAAAAAEJ_fhsEKXDuxJ-jIJrZpdRuSAAAAAAAADZ4", "types": "CAMbBQQD"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "JvISdnJ47BQ01489cwF9DA"}} +{"Stacktrace": {"frame": {"ids": "lW7fiQNIMpeUsn4R6s23pwAAAAAARq6glW7fiQNIMpeUsn4R6s23pwAAAAAAp3aplW7fiQNIMpeUsn4R6s23pwAAAAAApuPOlW7fiQNIMpeUsn4R6s23pwAAAAAApwWL", "types": "BAM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "ZZ003BQ7N2U_lm0xfG38Bw"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnC-hFb4flwkETp_lahExtrwAAAAAAAAAMyqhsLQpHnDkAN8Nbjjx5JQAAAAAAAABpbecvgDELr5B5ZYyfqmyRzQAAAAAAAACqS5_Ae_ABcHEi_tFSU_YdMgAAAAAAAAC9JwuJ_YrJlPmIg66vmsQiFAAAAAAAAAAzDFSw6fr_Dyh89ExFUOBWuwAAAAAAAABm5q05wRKHTvDR50L-9gxI_QAAAAAAAAAamFLCPRKYSR953-KUHmrTKQAAAAAAAAAFP7LRUTUpfWcPCMm1bmRUrgAAAAAAAAAYLY5nQliSvp7OkEEaJEKLkwAAAAAAAABBbigR-ShDu7V7qAzdPnM3tAAAAAAAAADaRp8oHoivvR1QPD15A2l2aQAAAAAAAAAeo5Yet5LouZ46IDsTOII1IgAAAAAAAAAbXlXfalHUp2vnq0b86Ske7QAAAAAAAABAIWt15Ku8-XXZGHssEyOb_gAAAAAAAAAw_NbGvNUAYnubBA2-cDqDrQAAAAAAAAAeOnLcYjZw6TA_n_91fXkLXAAAAAAAAAAMC5AvyP3bA4PjoVWj5P4reAAAAAAAAAADev9B8ljReSM94vfJilTEvgAAAAAAAAAMe4ya-skDHmxwpVLi-SiX7wAAAAAAAABwQpI4ER3R8tI6upvquCghPgAAAAAAAABGmFLCPRKYSR953-KUHmrTKQAAAAAAAAAFRoDaQXQoj9oJAPcF2bxG2gAAAAAAAAAIThW7aRtMu5otJJ-vTe9tmQAAAAAAAABMmFLCPRKYSR953-KUHmrTKQAAAAAAAAAFYfB7-hrLT6gRL7Zld6Lp8QAAAAAAAAAQgA6euexXfLm-c-5bdW_pngAAAAAAAAAxmFLCPRKYSR953-KUHmrTKQAAAAAAAAAFTF7R8j8eszhmy8WskJZ2NAAAAAAAAAAI4pUL3Hn4ak1IOxEqUVFkHAAAAAAAAAAWsfblkOdEaLUvfgvW-QAdjgAAAAAAAAASmFLCPRKYSR953-KUHmrTKQAAAAAAAAAFipWtYiD0L8zHlsu-KRlKIwAAAAAAAAAUDTQx8l7VJrszvdqL-9PytgAAAAAAAAARYgOo_ePEdS6xTt4Sk9uAVQAAAAAAAABpsE8Y8IrvW4YwnRePcOob7gAAAAAAAADrh7EwhN3qpUCOhT4f95Qz0gAAAAAAAAExsfblkOdEaLUvfgvW-QAdjgAAAAAAAAASKTge2icH_VE2kyqCeDCWXgAAAAAAAAAFNDsPouY3NiBzl33zaU7fMgAAAAAAAAA4rJNvWVtt3dJDIQfHVkT3mAAAAAAAAAAkeG5s_qI4F4I51_yuNS46mwAAAAAAAAAOjqNXsTtyxyukYQAy6-O-GwAAAAAAAAFy7pNlbTUIFAGF4ew35tQ1mAAAAAAAAABAtupDcGbWHEIVe-Q0MXArXgAAAAAAAAAGnpAgnwz74Cv0gdn19xaCuAAAAAAAAADyrQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAFHBnk-r2jmt48gRTNgMOf8wAAAAAAAAAlg6A5O5DE8lToSDEW-xkfTwAAAAAAAAESrQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAFHBnk-r2jmt48gRTNgMOf8wAAAAAAAAAl5-kXfhOMZeJHGZEEqUgZKAAAAAAAAADSrQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAFHBnk-r2jmt48gRTNgMOf8wAAAAAAAAAl2i9E7A7hICqdU2K3ciAqMgAAAAAAAAFLrQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAFHBnk-r2jmt48gRTNgMOf8wAAAAAAAAAlbzjbwk-dGTAEYzErTwuuOQAAAAAAAADgrQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAFHBnk-r2jmt48gRTNgMOf8wAAAAAAAAAlu1a5bpPCc1mWwQaCPFMnewAAAAAAAAAGnpAgnwz74Cv0gdn19xaCuAAAAAAAAADyrQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAFHBnk-r2jmt48gRTNgMOf8wAAAAAAAAAlBPKHF0uWLBGF-EDvSz50AwAAAAAAAAAGnpAgnwz74Cv0gdn19xaCuAAAAAAAAADyrQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAFHBnk-r2jmt48gRTNgMOf8wAAAAAAAAAl7tZRpCUjHjYLEjfN8TUeQAAAAAAAAABwrQ3u6x1hcLs9pHgHDEucRgAAAAAAAAAFHBnk-r2jmt48gRTNgMOf8wAAAAAAAAAynrAGsit1vNwpXNBDsxW-CAAAAAAAAAAFLewwMUXjDx8itumiJ5m9uAAAAAAAAAAOIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABhJgyhaKZ9eSJym5BMY7B2wAAAAAAAAATYIa5zXbUOkdcfY3BuS_SLwAAAAAAAAAKZaLM93n34r2A8Tcv4UQ0mgAAAAAAAABCIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABsNHGncE46bxEYqLkiTNaMgAAAAAAAADJmT2KIvEsDcA0I2RiMDHSuAAAAAAAAAAL2s9Z-1EblvqRm49b6lgFUQAAAAAAAAAy4wHidvPJpZTy7Q8lK21otgAAAAAAAAAfZ7ZAUH4Eu0mE1Lr-7VKuiwAAAAAAAACg35BPGLqYcVRpRMhVtTI1-AAAAAAAAAAJim4CSpLxZ-7jo4vfOXMUNAAAAAAAAAAM1XoENb4qIQOXPbHVVVKpqgAAAAAAAAAMmFLCPRKYSR953-KUHmrTKQAAAAAAAAAFJA3L5oTr7kSGnFgD6CVpBwAAAAAAAAAMn_CesQ5pN3dpPkarM3dScAAAAAAAAAAOizoIqZ2EJGa3SfYoWrHMNwAAAAAAAAFJ83pPtFJ0Pe4c5bSskvCkMAAAAAAAAABnkgFL8vDWeva5yNPPlsZUygAAAAAAAAA3etyLdtogTiVt3WXDEehDtAAAAAAAAAAHUMvK1DDwEUhXYDkQsZey_gAAAAAAAAAYjJKyCKnhVDSt_DYivEq7yQAAAAAAAAAidkAubCDgL7khnJPkNdUwRwAAAAAAAAAGa9pOxt0fqv_QTtBRBlV_8QAAAAAAAAAlOkkiwQ9aM2OL-oHoBjLvhAAAAAAAAAAOq5h54Kb3U85YSjTzRyaF_gAAAAAAAAAW4iWyc_l7zIMt9gi9uNbDxgAAAAAAAAAFhYgsx8HjLzhm9LYVIHuquwAAAAAAAAAEOkkiwQ9aM2OL-oHoBjLvhAAAAAAAAAAOq5h54Kb3U85YSjTzRyaF_gAAAAAAAAAWcnnfHiXpYZi_Xvw7GhRVowAAAAAAAAAFAtOKW4aoAdMhmd7GyZi8TAAAAAAAAAABS271rVNO8pQHfSRu8D2-7gAAAAAAAAABHbnRCcPqpeCpFx8D1c3dbgAAAAAAAAAF1AYJQILFnHXEV9oxyMp7FgAAAAAAAAAFmYNktaRFRY0plnRBTkItUgAAAAAAAAANWyzAcL0AWgIkl4A0YO5RQwAAAAAAAAAmSPBSRBf5-JQ_B0HnmSeY4AAAAAAAAACL92jCQ7FJd_qE8nezHXMXDQAAAAAAAACYySBUSdLpTE4kxptOyYACMwAAAAAAAACHGV9odb7eqAONOiRGAnVZsQAAAAAAAAFYkxggmTmW48eX63B51-tv4wAAAAAAAAEElMvGIQySgdUzptqnRUQpUAAAAAAAAAALeL-oIR1CC2R0ykcUAAfoowAAAAAAAABTMDZqk-bznCkkPcSl3Z06LwAAAAAAAACRHwASysJunE30XUonp_SYPwAAAAAAAAAFY2eGzIpiKa-QBMAAKXZA_QAAAAAAAAAA", "types": "CAN-BQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "p_qI8Ts1o4R4GHiu2MHhmg"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYIqvacTmpSpuAOHxTh9l_exAAAAAAAv8CwqvacTmpSpuAOHxTh9l_exAAAAAAA2XfPqvacTmpSpuAOHxTh9l_exAAAAAAA2TYhqvacTmpSpuAOHxTh9l_exAAAAAAAjXK5qvacTmpSpuAOHxTh9l_exAAAAAAAgN46qvacTmpSpuAOHxTh9l_exAAAAAAAgMZEAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABDJLJRrJNARIRyzxCnxa4WQAAAAAAAAAYmafpkLzxyAOwBxGkDhiVNAAAAAAAAAAMZJExOMsbBfy8WqRgcylCLwAAAAAAAABp8T1GTOmmtW2zNfDAiA-RCgAAAAAAAADHUu1pGUpLQ35aan6Rej1VEwAAAAAAAADJJwuJ_YrJlPmIg66vmsQiFAAAAAAAAAAz6KK4nudrIWi-u7NWSV5S7QAAAAAAAABnEBqXZsoPGK2F0IF3gcVlzwAAAAAAAAAaIunvE7sWPSz4YCgk0UVsFAAAAAAAAAAFradvx3ka7U7YJiCSRTJ3kwAAAAAAAAAYW80Dopg4GIca7JXLWHFwVAAAAAAAAAA6mZnKLpDGXH5r07376P6zgQAAAAAAAACCRJOpyLKTKlv2enKt_xrNSwAAAAAAAAA73-s2ANt1vNwpWiAFGYs2eAAAAAAAAAAFdWyxTJ01gHAXEF1z9sDOKwAAAAAAAAAOBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABJG_5wUb6Q1WN6hfRJFcqEwAAAAAAAAAT3r1MnCyPIgP6mYXLoLcz9AAAAAAAAAAKXrSW2Ro9Ws10jZmj20O0hQAAAAAAAAADHSftxLHu5xLDzJX5F8kBXAAAAAAAAAGVVTe2bxq-Jr5s7wFSht9_vgAAAAAAAABqC-DRD-J_UvsXUc59TOzOVAAAAAAAAAAI8AUzmHhqUZOHWH8Bccl_sQAAAAAAAAHc6TUXIWC2WcF-_43xKp-wtQAAAAAAAAETd0aX3KonLcPjMEO-H4cl5QAAAAAAAACDIir7eGdVZsjNItLfykxs3gAAAAAAAABaNWk7GQCtDStIuwl83iGqTwAAAAAAAAAv2CwcPQcRkPoyzD89B8CtBQAAAAAAAAB-OoUyVxHN2rYPjLirZK4ihQAAAAAAAAAGVWOUd0jMGfSYTm8Ns8_hDgAAAAAAAAMEGASd-GxTI3CnNP2dmtJ0gQAAAAAAAAHeLeaqrhFv22rb1wzUlEMwAgAAAAAAAAAMkbApFhDbueqDTdsjbbOMUQAAAAAAAAAg7z0vK5_lN4w4BvVehfj4wQAAAAAAAAALTrTEyZn7B30gBbM8MLULSQAAAAAAAADAV-zEfMlNdgG4SvC62RGySgAAAAAAAAAXFTndS9swIMm3K06ZEY-ZcgAAAAAAAACORe_dvjV79ezcmCpSsdxZuAAAAAAAAAAM6QvV8Ag8LiqGHdRYVptEnwAAAAAAAAAEGQWHl0s0fmnGMrrNRd7yDQAAAAAAAAAHp_sL8jeyDjPZI0awM1-lMQAAAAAAAAAB0dBKXDWNMiejfPcFa2J8WwAAAAAAAAAFIY3kOFBxddrJzoZrzargTAAAAAAAAAAA-6ARRafpA91HAsyIFfBmBQAAAAAAAWW_vQ7EacG6CR4rzcFaNFEsTQAAAAAAATRalHp5_WAgpLy2alrUVab6HAAAAAAAwACLlHp5_WAgpLy2alrUVab6HAAAAAAAAEM2lHp5_WAgpLy2alrUVab6HAAAAAAAAEATlHp5_WAgpLy2alrUVab6HAAAAAAAFz87lHp5_WAgpLy2alrUVab6HAAAAAAAFzaR", "types": "CAMwBQIDBQQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "iCDmSM5ZMerasUWJeO1RoA"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAACuyaSy1RHb0tl4hOkUXXLXQQAAAAAAAAAzGUwipo03N4glyS6268EEmgAAAAAAAAAKds9yTvk8UzKAgDSjs6pNkgAAAAAAAAANzVmgO3uV6bZtx2XUjAkXcgAAAAAAAAAHNJPFmv3cYB4fnHHuI2lrYQAAAAAAAABbsh1T4lVaqf9yZbH1lhPrZwAAAAAAAACNduOfgYTG-Pz3qxfosjBOKgAAAAAAAAAEFTD_DMq9LCr2rMh4XAWQegAAAAAAAAAF5wGUH60A7iQqyWZA8jj7kwAAAAAAAAAdDPKZG86WCe-rkFTx_IAdagAAAAAAAAAIhCl4jaZsLnxk3C0-qCjENwAAAAAAAAAFfMf02bFW9UOsHXaY1ju3UAAAAAAAAAELie51PKalSPz0oTw25hOjdgAAAAAAAAAHxhzTSPDm8vP1CtKm1pHa_AAAAAAAAADX3SU4n_eVvbLOxW0zu9gonAAAAAAAAAATBULXlXluMevM23wGn-scGgAAAAAAAABIX3YYuA6U4OM2ZjQu8fVwYwAAAAAAAABLSNDYMaokxZpNhjFZnC0cqQAAAAAAAAABm_DK0nAVx1tMIimjFYyQPAAAAAAAAAAS_B6c9784WlLvFF9uk3pSswAAAAAAAAAcFhP6JufLZYYhuADwCLvRKAAAAAAAAACVClvIuFYr5ArIOKSAaLGqggAAAAAAAAGv5tsfFOiB0XQGtxfawg0JiQAAAAAAAAGPe8ARrqT9JmZcuLd6G8ZTMgAAAAAAAAAGV6Xv8yqxVPgM9JZWMvD_ogAAAAAAAAADS4xo2K-7S9Psi5KVtkSU0AAAAAAAAAAONNX6LKmc60fQ-V_Nw2_9cAAAAAAAAACwVGJu-bYJBqeYrugO5COQdQAAAAAAAAHGAxJNlcXBvrgVOpR8a1WEYgAAAAAAAAG2n0mWG92K0zbUz5S-_ixlIAAAAAAAAABrKxHEggHQvpMo7NXk9m-aKgAAAAAAAAAghvijQK1a5iiWkFoVp40y5AAAAAAAAAAGvRwDdjzKXpreYxi90MDGQQAAAAAAAABms-5AzRQS0AyjARcxs2D2DQAAAAAAAAAX4Wda4DEj07iGWB9mKOvWMAAAAAAAAABo45faa5L8TTwA9zJYlmGKywAAAAAAAAAUq3uedK83ZBGp21KqIIcD7QAAAAAAAACWwh1zX_oqIoKNefuSr1ngvgAAAAAAAAAfPu2JWw096K4wEF2usuCt-AAAAAAAAABc_gZdvwQP6FQ5RcMsyYjAXwAAAAAAAAFD1PIPTBbtPHUrg0d3N2nuygAAAAAAAABBlt4DCYuCTF_qGLbdRnIhjAAAAAAAAADDFERPJCfrzkqs-7XhU53jhwAAAAAAAAFtzP5HzbxY7goeRVwj37voeQAAAAAAAAACy87oVbGEsnZxcfXcdrLYswAAAAAAAAB6i1t0pdr_xnHfk_UsEnpDiwAAAAAAAAAMip4jveo5OesPxqJEnXrM0wAAAAAAAAAOrDli4oCQ8GWnljRH2_1bLAAAAAAAAAEZH2XMIGvXG_73d5XzBvFSxAAAAAAAAAEB3D_A_sI8XDgpK1lgEmfWZgAAAAAAAAC_hqzAzTCpj-Nsh96skuh10wAAAAAAAABJibtF619pQHGNKblUfb9MKwAAAAAAAAANHJHMeViEmhpbdveA5zS4zgAAAAAAAADUNW0MvXdcfyNB_rjxH_JHXgAAAAAAAAAN16j8HtYPNjd3qdRoXnMi0AAAAAAAAABaO4wSxfOE-c6Sft1wmb_6fgAAAAAAAABpoKMpPO_rrguOLYx8VBTGhgAAAAAAAAAGakj2EbMcV1BY1x-2V-R6VQAAAAAAAAANqND66z7potQhxSJ1H4ezHgAAAAAAAAAG2B-rbS8KhbB8ePRnpVf7_QAAAAAAAAAU89ogc5uTbOqh2aLJlzHKmAAAAAAAAAAX9ccfl0YdabzwSUY2PdYA0QAAAAAAAAATDT06D5A7Yxiqx6IJ7Kvu0wAAAAAAAACCtq710yUF9bzkGXjQwM8_6QAAAAAAAADWH7Tmwg9rKZuxl8eAfe0cQwAAAAAAAAATpKAwMR-9-3afO3tccFzRKwAAAAAAAAEnbYAsS36pm68v9ebHURyhdAAAAAAAAAACBUXwXn0jTEljgr89rND6CgAAAAAAAAAwR_ZB9SjojSrHSQ8dwIq_WQAAAAAAAAOrIvW6glp0EMhdH986Hu8MmgAAAAAAAAAA", "types": "CANTBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "hm6pJJBFG1EoukWkIgUkSg"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAACuyaSy1RHb0tl4hOkUXXLXQQAAAAAAAAAzGUwipo03N4glyS6268EEmgAAAAAAAAAKds9yTvk8UzKAgDSjs6pNkgAAAAAAAAAA", "types": "CAMQBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "P4G4FrOCtB7UkFb-8UfLLQ"}} +{"Stacktrace": {"frame": {"ids": "z5k14PsCcDvFyUC1kmlV3QAAAAABGlXAz5k14PsCcDvFyUC1kmlV3QAAAAABSEXoz5k14PsCcDvFyUC1kmlV3QAAAAABSDNxz5k14PsCcDvFyUC1kmlV3QAAAAABSDbU", "types": "BAM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "L7kj7UvlKbT-vN73el4faQ"}} +{"Stacktrace": {"frame": {"ids": "634wiWh6F21tPpXr0Zz3mgAAAAAAEfFi8NlMClggx8jaziUTJXlmWAAAAAAAAIYIZSkKN3zNxr0HYuO2pqe5hQAAAAAAwcBwZSkKN3zNxr0HYuO2pqe5hQAAAAAA5ECvZSkKN3zNxr0HYuO2pqe5hQAAAAAA4_9_ZSkKN3zNxr0HYuO2pqe5hQAAAAAAj7b5ZSkKN3zNxr0HYuO2pqe5hQAAAAAAgwXKZSkKN3zNxr0HYuO2pqe5hQAAAAAAgu3UAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAVcBTlmvRfiK_eaWEDctBQ5wAAAAAAAAAMjLRjRKJujxpIzkv5EAU8CAAAAAAAAADEKPNYkONEetadZox7ZPj-_wAAAAAAAACaB0jzxgjaMAT2f_-sp0qKdQAAAAAAAAALFOYm6EawRDxcLqPYr2TlYwAAAAAAAACT", "types": "CAMKBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "8Nt8q4FpMS1YBSbb8JjA4Q"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYIqvacTmpSpuAOHxTh9l_exAAAAAAAv8CwqvacTmpSpuAOHxTh9l_exAAAAAAA2XfPqvacTmpSpuAOHxTh9l_exAAAAAAA2TYhqvacTmpSpuAOHxTh9l_exAAAAAAAjXK5qvacTmpSpuAOHxTh9l_exAAAAAAAgN46qvacTmpSpuAOHxTh9l_exAAAAAAAgMZEAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABch4dwrMYlRFRjyfsvjXt4tgAAAAAAAAAg3V-8FLy1GH8nVRceMDeaiwAAAAAAAAABnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAV4j8yS0qsC_6XfTfMdPp5KQAAAAAAAAAQ9oBnE4xnAvOiOv1q-LbApgAAAAAAAAAEKjF0JPI3hCXDTCc6Z7ofdQAAAAAAAAFLAFikCbtP_Dm7iUthjnlnEgAAAAAAAAEq56q5trA0bAF1B-Um6L_rqwAAAAAAAAAGgi_774C-EJhuJfyXXhzVgwAAAAAAAABEgvYbo0YBmE65VwrpTWYalQAAAAAAAAB2tMqbgEmfZJ47YRogSA-gKgAAAAAAAADlKz3aUSMwqIxIxaAplngd8AAAAAAAAACXezRRC2sXv2hHIZbqwpv5zQAAAAAAAADSu-D_h2osKIeJ8uQjOobu_AAAAAAAAAHywnSBrxGSumHiAQQABJeNtQAAAAAAAAAWcDTcKVGzeMo1oRYERHbXHwAAAAAAAAB5", "types": "CAMUBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "kU23IHsR5pRIb6YvFIRS3Q"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnyhAKi3MXCBa6QbMykpz9ogAAAAAAAAAIImSzkW6pD8jhY2FJThRJ8QAAAAAAAAAFS3fVfiwq94i14_VfWgkFiAAAAAAAAAAF86ShTOKEhMBNNZND3RBl5QAAAAAAAAABd8JoiANFYdN_HlsRPvshMwAAAAAAAAAxIz1h-lJR9hawlSRETyh1qQAAAAAAAAAIeOepQqcpphM0NTbieX0wIgAAAAAAAAE4ooplCYR951bf9t1YS6OUBQAAAAAAAACrWAsi5KS1LX4-mmePSPN4qwAAAAAAAABzUnU61_VzRQulrS0OJYkU7gAAAAAAAABaHJWDExCp8dmw9-CAF-WXfwAAAAAAAAAJoxSW321mwoDVUa0haFaP7QAAAAAAAAAuTyZK4BUMsPg3P3wu6EYhkAAAAAAAAAB6PiSwsMqyjn24GnxXwBvfuQAAAAAAAAAGzY8oFom0Mq0j4EO6ho2wLAAAAAAAAAMEeC3OrXu_7VLA9EdCZMRh6AAAAAAAAAGdLeaqrhFv22rb1wzUlEMwAgAAAAAAAAAM4FrN4VeCu4HJZNavjw0tmgAAAAAAAAAgmSb0V8ANbBWpi7TZTXcPNwAAAAAAAAALP4ZjI7g37fWz45cghUwTxQAAAAAAAAA5H1mK93wmygFvNYC57fovfwAAAAAAAAAfUnm7y6YLik7WuESK-9BXmgAAAAAAAAAOp_xoiXGCsXgpBNHrp24JKQAAAAAAAAAV5jDiO_Lgv2e10LojWCULCwAAAAAAAAAFOX6pd1GZ1KHmZAikJOZD-wAAAAAAAABoBoyt9QmW9-zzCwzKZ9LhFAAAAAAAAAEXXJAoY26CmEBCHOgy-kaybgAAAAAAAAALSWMbBt07f_PdUThaE5iWmwAAAAAAAAABHoqCWZ471w3YYcVCfd06hAAAAAAAAAAORrXasmojz0ojjKVFAz-eswAAAAAAAAAA8DDAjVJl721y2vxKHGSBegAAAAAAltV1", "types": "CAMkBQED"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "L3nUGoNiPS6jpV4lzI0HKw"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYIqvacTmpSpuAOHxTh9l_exAAAAAAAv8CwqvacTmpSpuAOHxTh9l_exAAAAAAA2XfPqvacTmpSpuAOHxTh9l_exAAAAAAA2TYhqvacTmpSpuAOHxTh9l_exAAAAAAAjXK5qvacTmpSpuAOHxTh9l_exAAAAAAAgN46qvacTmpSpuAOHxTh9l_exAAAAAAAgMZEAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAV5GqAUaYv86rgi1w0nJY6kwAAAAAAAAAMjLRjRKJujxpIzkv5EAU8CAAAAAAAAADEKPNYkONEetadZox7ZPj-_wAAAAAAAACoMH4ZLlkHvkfZTK_KWydMDgAAAAAAAAC4ZGKy7ArdiDfRv4uogAG0CQAAAAAAAAAzgQv1haKZmRIe2g2DBTzDVAAAAAAAAAAX6a5XWhjsuz26DnHoF0x5RwAAAAAAAABSwe1dsXqMIsi7fg7veaiJyQAAAAAAAAAIN1GpN_XYj7AoDd4ql2RyHgAAAAAAAAABGXzDmM-hgGlNDieJ9EUhGAAAAAAAAAEwzgecUfJLYoiYIxBtYNFbZwAAAAAAAAAo1kYL5y-AD3sdPCqn6ivZowAAAAAAAAAEDVxwSo4eOEycJY2lxrVA3wAAAAAAAAAOwvgldxPJJj8FcIi5eLlQvgAAAAAAAAAmnrTqwGPzO0uzQD3C7T948wAAAAAAAAAP9UBMSNV9JIwVE7bRWf7rNAAAAAAAAACIfbCakSHDHoBMTyuJNDB7CwAAAAAAAAAxFpTtdOGq0KA_LZU93WcT2gAAAAAAAAAstXdLdOQIv2CEGExsyL_epQAAAAAAAAC4PQiF41WruwXqpcujdafqogAAAAAAAAAlCsbq98qEMjLfznv89zxcQwAAAAAAAABASlb8MNVKIFfE9HJqtN3BhwAAAAAAAAAAqn_i-43Ca0UMIY4VcWF0AgAAAAAAAC8kfhsEKXDuxJ-jIJrZpdRuSAAAAAAAAKGdfhsEKXDuxJ-jIJrZpdRuSAAAAAAAAI9-", "types": "CAMbBQMD"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "YVZcVDoPQkcegaQYBCZkzw"}} +{"Stacktrace": {"frame": {"ids": "lHp5_WAgpLy2alrUVab6HAAAAAAAwAIUlHp5_WAgpLy2alrUVab6HAAAAAAADAxglHp5_WAgpLy2alrUVab6HAAAAAAAC6oJlHp5_WAgpLy2alrUVab6HAAAAAAAC6XulHp5_WAgpLy2alrUVab6HAAAAAAAC3tRlHp5_WAgpLy2alrUVab6HAAAAAAADtsslHp5_WAgpLy2alrUVab6HAAAAAAAp1q0", "types": "BwQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "14cFLjgoe-BTQd17mhedeA"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA76Af8DDAjVJl721y2vxKHGSBegAAAAAAdLJM8DDAjVJl721y2vxKHGSBegAAAAAAdFaK8DDAjVJl721y2vxKHGSBegAAAAAAdDbu8DDAjVJl721y2vxKHGSBegAAAAAAdJ0N8DDAjVJl721y2vxKHGSBegAAAAAAcin2", "types": "CgM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "_eitfgJr5SICw3IID0ieUw"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAfa_D_KA1yUInCMlw0HKKJXgAAAAAAAABX7SvDCQFg8noiOfIrUqyslQAAAAAAAAAJ", "types": "CAMIBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "5RX98LQ-TugfdPLcy0tTFA"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAANVk4geTYpM8uDz6HvI1jT73gAAAAAAAAAZHc4fbH8wnl-aZPA1_7gB-wAAAAAAAAAyaBWoLwWk5u6kqgD3gwKyagAAAAAAAAAYluqtQRsIjOPWY7wdgiD4bAAAAAAAAAAIwRG5B2klLAxXtt0B9lVulAAAAAAAAAB-Gjm3c7dzM21apoh-_mmOrwAAAAAAAAA2O57ePcWMLnVMB2c1JYP_SgAAAAAAAAAMnfWxBNCjXXCl3tAETF7T8wAAAAAAAAAUoi3QXV0PR2nGOV5pzW_MvAAAAAAAAAAGhWcFfmcayZnqxV8VaktDEwAAAAAAAAAHe_qwG8gpgd38JnWYrQTbhgAAAAAAAABT3ITJpMVvGWxtAJg8iCjlKgAAAAAAAAAhLhf7WGU0lxCaIsujYZjafAAAAAAAAAASU3XGmp9XAuDQti4O3QZItgAAAAAAAAAT_eHurMk1AuWI20vPBelYUQAAAAAAAABRbt6Ho82qwuZ3iDZme91J3QAAAAAAAAAOMrj6Z8YTvyj7p7UD3GGpMgAAAAAAAAAFqzJXUPlBwNnFY2QMV7wpwgAAAAAAAACFkpBi8LFR3RTjSJqFuljOuQAAAAAAAAAAhrCIM-i5cm0sSd-oVTZrsgAAAAAAACZBwEIfl0pGBuepuUcTwdKc2gAAAAAAAFtjwEIfl0pGBuepuUcTwdKc2gAAAAAAAEJ_wEIfl0pGBuepuUcTwdKc2gAAAAAAADW4", "types": "CAMgBQQD"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "lDY7ho5Q3VRiBNIA0utPkg"}} +{"Stacktrace": {"frame": {"ids": "634wiWh6F21tPpXr0Zz3mgAAAAAAEfFi8NlMClggx8jaziUTJXlmWAAAAAAAAIYIZSkKN3zNxr0HYuO2pqe5hQAAAAAAwcBwZSkKN3zNxr0HYuO2pqe5hQAAAAAA5ECvZSkKN3zNxr0HYuO2pqe5hQAAAAAA4_9_ZSkKN3zNxr0HYuO2pqe5hQAAAAAAj7b5ZSkKN3zNxr0HYuO2pqe5hQAAAAAAgwXKZSkKN3zNxr0HYuO2pqe5hQAAAAAAgu3UAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABch4dwrMYlRFRjyfsvjXt4tgAAAAAAAAAg3V-8FLy1GH8nVRceMDeaiwAAAAAAAAABnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAV4j8yS0qsC_6XfTfMdPp5KQAAAAAAAAAQ9oBnE4xnAvOiOv1q-LbApgAAAAAAAAAEwRQstrBYZ0ShmJnYV-ADrQAAAAAAAAFLAFikCbtP_Dm7iUthjnlnEgAAAAAAAAGtTfYUtN-rf_0uYWSYAHKfDQAAAAAAAAAH4O-ovD5bG6j4SStsVkgzcQAAAAAAAABHacnJzC_C1bsmTPMH_pcsCAAAAAAAAABdoGQ2OUdACCpDvWCPN-t91QAAAAAAAAAMagUnXAr2_hsl_qyYsSOrqQAAAAAAAABcIP-4kS9K2z9dstopMSzdEAAAAAAAAAAKW2G52eIffs6rSMZaDokh0AAAAAAAAAATZUblhfsP3eghXbFQlRo7WwAAAAAAAAABmwuQyMHK5RYUe_aaboQONgAAAAAAAAADyScC9Npb_ya8Br_QJ7EUZAAAAAAAAAB7Sm149Q4n4lGGxDiZNf94FQAAAAAAAAAHPUPItXL9ODmL0kwnalRigwAAAAAAAAAjTC2QqZAb8KnizCTsnAuCGgAAAAAAAAANaSV2qTlMpiialisnhZW9PgAAAAAAAAAN5wGUH60A7iQqyWZA8jj7kwAAAAAAAAAdq8pCDW17i5MKeUo5LLv96QAAAAAAAAABCW_r9nc-U1Fi4aU8XoGOXAAAAAAAAAAM8iRqFHNeeA-Vq5mM-jaESgAAAAAAAAARt5uE-GFMsmesNeLBgx_z4wAAAAAAAAADDnrzOZkhcnoTNvvkP1TdqAAAAAAAAAAi5iLGjLJFjZ3vVn5PbnDFiAAAAAAAAAAv4P1DMygHaI4yHD8SKHtROwAAAAAAAAAYSWMbBt07f_PdUThaE5iWmwAAAAAAAAABHoqCWZ471w3YYcVCfd06hAAAAAAAAAChYKWsiTTdS4chcntYalLpvQAAAAAAAAFDGwvNKsXwAWDPAp3O-Td6NAAAAAAAAAAXWOsBhYd9IbFldGZWVw03hgAAAAAAAAAVKOnuKkBYBgm4HtfyhrUS2wAAAAAAAAC777CMGAnm82a_x5oOgINbKgAAAAAAAAAWHNWspokmSU3mmXH9ENGQygAAAAAAAABV5ajLrlzgZ5NJn1Ifyl4tjwAAAAAAAACKX6aGRt4o2LlAY8FCZg8t_gAAAAAAAACWKO6VsFDaVRMvLVZz6LumQAAAAAAAAAACqOuY7z4cBuGpkQnMsu5M4QAAAAAAAAAB2HSuy-oZigDpg5cKC7dkqwAAAAAAAACX", "types": "CAMuBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "LpsjtOqp4gQNWANoCtMhpw"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qp0eo8IwDes1buf90_PyLCgAAAAAAAAAwVB_A4ihbc-HQ5QDUj7Wz0QAAAAAAAAB_uE354xX1-6pq5qEqEPux5gAAAAAAAAACyTY-WlmajV5YhETL2qC3jgAAAAAAAAAdYkaRTKRhsYL87aTctpzuoQAAAAAAAACw3HIAZ7KMFykIwp4WdvYF6gAAAAAAAAEVsCbyN1LnxDvKtK16chc1UAAAAAAAAAAHSdeiuUk4wJykJDjmt_gy5AAAAAAAAAAISdeiuUk4wJykJDjmt_gy5AAAAAAAAAAISdeiuUk4wJykJDjmt_gy5AAAAAAAAAAILqytSQO0xoKueTdBUmzm9gAAAAAAAAAc0iStdyhoG4WqQkJ1aFKZnQAAAAAAAAA1k8ZGDLdSxDSM2lL9n7Ll1AAAAAAAAAAfHbqGqBv_PKwJzIrNCTkRMwAAAAAAAAALMRjz7ydeslXKi2GbhltLmwAAAAAAAAAI3-Nyd5iyxL31wUMeSpG3wAAAAAAAAAAKggEpzKYUUToCZ_IIz775eQAAAAAAAAAHggEpzKYUUToCZ_IIz775eQAAAAAAAAAHSLSuJBznTE3BvHPDUjF7JgAAAAAAAABqrg1U8-_Qg7P4zN0uZwHMZAAAAAAAAAAh2sLCSYuHZvfeGvojnGEVfAAAAAAAAAAQr5B7woHUx7hmwtrjMRtOuwAAAAAAAAACLlon2cTU88EAb0xt_e8N6QAAAAAAAABBjxIGTrGHMnImsEvjAP6NuwAAAAAAAAEYOo9QFBPQY4ypFVPfKrisIQAAAAAAAAAfx9iJb0OhVB_kYAbvkT0IJAAAAAAAAABLw0BKBBEkfs3oopm7NSQhfQAAAAAAAADMxmj6c8GMqxu-eoFnl7D4qwAAAAAAAAAA", "types": "CAMdBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "hRqQI2CBPiapzgFG9jrmDA"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA76Af8DDAjVJl721y2vxKHGSBegAAAAAAdLJM8DDAjVJl721y2vxKHGSBegAAAAAAdFID8DDAjVJl721y2vxKHGSBegAAAAAAdJ0N8DDAjVJl721y2vxKHGSBegAAAAAAckc9", "types": "CQM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "Qn60Qbgyi9gMjkEiQhHG0w"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYIqvacTmpSpuAOHxTh9l_exAAAAAAAv8CwqvacTmpSpuAOHxTh9l_exAAAAAAA2XfPqvacTmpSpuAOHxTh9l_exAAAAAAA2TYhqvacTmpSpuAOHxTh9l_exAAAAAAAjXK5qvacTmpSpuAOHxTh9l_exAAAAAAAgN46qvacTmpSpuAOHxTh9l_exAAAAAAAgMZEAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABDJLJRrJNARIRyzxCnxa4WQAAAAAAAAAYmafpkLzxyAOwBxGkDhiVNAAAAAAAAAAMZJExOMsbBfy8WqRgcylCLwAAAAAAAABp8T1GTOmmtW2zNfDAiA-RCgAAAAAAAADHUu1pGUpLQ35aan6Rej1VEwAAAAAAAADJJwuJ_YrJlPmIg66vmsQiFAAAAAAAAAAz6KK4nudrIWi-u7NWSV5S7QAAAAAAAABnEBqXZsoPGK2F0IF3gcVlzwAAAAAAAAAaIunvE7sWPSz4YCgk0UVsFAAAAAAAAAAFradvx3ka7U7YJiCSRTJ3kwAAAAAAAAAYW80Dopg4GIca7JXLWHFwVAAAAAAAAAA6mZnKLpDGXH5r07376P6zgQAAAAAAAACCRJOpyLKTKlv2enKt_xrNSwAAAAAAAAA73-s2ANt1vNwpWiAFGYs2eAAAAAAAAAAFdWyxTJ01gHAXEF1z9sDOKwAAAAAAAAAOBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABJG_5wUb6Q1WN6hfRJFcqEwAAAAAAAAAT3r1MnCyPIgP6mYXLoLcz9AAAAAAAAAAKXrSW2Ro9Ws10jZmj20O0hQAAAAAAAAADHSftxLHu5xLDzJX5F8kBXAAAAAAAAAGVVTe2bxq-Jr5s7wFSht9_vgAAAAAAAABqC-DRD-J_UvsXUc59TOzOVAAAAAAAAAAI8AUzmHhqUZOHWH8Bccl_sQAAAAAAAAHc6TUXIWC2WcF-_43xKp-wtQAAAAAAAAETd0aX3KonLcPjMEO-H4cl5QAAAAAAAACDIir7eGdVZsjNItLfykxs3gAAAAAAAABaNWk7GQCtDStIuwl83iGqTwAAAAAAAAAv2CwcPQcRkPoyzD89B8CtBQAAAAAAAAB-OoUyVxHN2rYPjLirZK4ihQAAAAAAAAAGVWOUd0jMGfSYTm8Ns8_hDgAAAAAAAAMEGASd-GxTI3CnNP2dmtJ0gQAAAAAAAAHeLeaqrhFv22rb1wzUlEMwAgAAAAAAAAAMkbApFhDbueqDTdsjbbOMUQAAAAAAAAAg7z0vK5_lN4w4BvVehfj4wQAAAAAAAAAFEiWa0n_9SgJD34zKIyVJDAAAAAAAAAAMHo9Mi8DG7FUtIimhU2yBagAAAAAAAABqyzkMNxhTqdw8OfBH6KpVbgAAAAAAAAAEHWeYMQJlcaP0GMe8ZUABEgAAAAAAAAAu0Ju-cCCi3kHxLcKkTtnYSAAAAAAAAAAB", "types": "CAMsBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "7Se-2CaL4CbSRMgSyNxl5w"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAAIRbFStJdQtuBtluMM9Zd1OwQAAAAAAAABc", "types": "CAMOBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "P7SH4zhzkMyu8ClDXegYrw"}} +{"Stacktrace": {"frame": {"ids": "4rFFPORY8_4FvK9QaA3QUQAAAAAADLHJUJmIMEc6K8gJ39NHYaD0WwAAAAAAAhyG4rFFPORY8_4FvK9QaA3QUQAAAAAADLg44rFFPORY8_4FvK9QaA3QUQAAAAAAEcDs4rFFPORY8_4FvK9QaA3QUQAAAAAAEcbB4rFFPORY8_4FvK9QaA3QUQAAAAAAEhP-4rFFPORY8_4FvK9QaA3QUQAAAAAADxKo4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD4jP4rFFPORY8_4FvK9QaA3QUQAAAAAAD4jP4rFFPORY8_4FvK9QaA3QUQAAAAAAD4jP4rFFPORY8_4FvK9QaA3QUQAAAAAAD5YF4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD5YF4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD5YF4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD5YF4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD5Cd4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD5Cd4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD5YF4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD5YF4rFFPORY8_4FvK9QaA3QUQAAAAAADxmJ4rFFPORY8_4FvK9QaA3QUQAAAAAAD0IB4rFFPORY8_4FvK9QaA3QUQAAAAAADg7N", "types": "HQM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "S07KmaoGhvNte78xwwRbZQ"}} +{"Stacktrace": {"frame": {"ids": "634wiWh6F21tPpXr0Zz3mgAAAAAAEfFi8NlMClggx8jaziUTJXlmWAAAAAAAAIYIZSkKN3zNxr0HYuO2pqe5hQAAAAAAwcBwZSkKN3zNxr0HYuO2pqe5hQAAAAAA5ECvZSkKN3zNxr0HYuO2pqe5hQAAAAAA4_9_ZSkKN3zNxr0HYuO2pqe5hQAAAAAAj7b5ZSkKN3zNxr0HYuO2pqe5hQAAAAAAgwXKZSkKN3zNxr0HYuO2pqe5hQAAAAAAgu3UAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABch4dwrMYlRFRjyfsvjXt4tgAAAAAAAAAg3V-8FLy1GH8nVRceMDeaiwAAAAAAAAABnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAV4j8yS0qsC_6XfTfMdPp5KQAAAAAAAAAQ9oBnE4xnAvOiOv1q-LbApgAAAAAAAAAEwRQstrBYZ0ShmJnYV-ADrQAAAAAAAAFLAFikCbtP_Dm7iUthjnlnEgAAAAAAAAEq56q5trA0bAF1B-Um6L_rqwAAAAAAAAAGgi_774C-EJhuJfyXXhzVgwAAAAAAAABEgvYbo0YBmE65VwrpTWYalQAAAAAAAAB2tMqbgEmfZJ47YRogSA-gKgAAAAAAAADlCQUIxcdtvT35ZznMVnzc_AAAAAAAAACXN4c5sJszjyVzcx3AmWN8pwAAAAAAAADS_GFFImAT2VE6Ar5VgmaN7QAAAAAAAAHywnSBrxGSumHiAQQABJeNtQAAAAAAAAAkPK6VPfk6aJqBe-5Qji8O5gAAAAAAAAAFEIxfgHbDbI5dElFzd3Ha-QAAAAAAAAAZFq10nEfKWtXEt510UwEUUAAAAAAAAAB7V_QMdmt4RxKxn4ZNgdvkJwAAAAAAAAAReNITicG0MvFr9HQHk70FLAAAAAAAAAAI9j0yGbd8eQNwdRhHZ159OQAAAAAAAAA9vzzPIR5tUnMkJ7d_ITdQRgAAAAAAAAAC6YIeLAztuVSewvuGh8XKXgAAAAAAAAAFIQvpHpp20NHD-0mZNf95oAAAAAAAAABp0vAOoRRxsQcS4vDapC3-mwAAAAAAAAANqnvWBP24iZLcQ-Wi76ZDxQAAAAAAAAAI3X9PCd1tVPhzrMiwigfodgAAAAAAAAAAZSkKN3zNxr0HYuO2pqe5hQAAAAAA52Uf8NlMClggx8jaziUTJXlmWAAAAAAAAQEslHp5_WAgpLy2alrUVab6HAAAAAAAwACLlHp5_WAgpLy2alrUVab6HAAAAAAAAEIGlHp5_WAgpLy2alrUVab6HAAAAAAAFFQelHp5_WAgpLy2alrUVab6HAAAAAAAFErelHp5_WAgpLy2alrUVab6HAAAAAAAFBtp", "types": "CAMfBQIDBQQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "4tB_mGJrj1xVuMFbXVYwGA"}} +{"Stacktrace": {"frame": {"ids": "634wiWh6F21tPpXr0Zz3mgAAAAAAEfFi8NlMClggx8jaziUTJXlmWAAAAAAAAIYIZSkKN3zNxr0HYuO2pqe5hQAAAAAAwcBwZSkKN3zNxr0HYuO2pqe5hQAAAAAA5ECvZSkKN3zNxr0HYuO2pqe5hQAAAAAA4_9_ZSkKN3zNxr0HYuO2pqe5hQAAAAAAj7b5ZSkKN3zNxr0HYuO2pqe5hQAAAAAAgwXKZSkKN3zNxr0HYuO2pqe5hQAAAAAAgu3UAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABDJLJRrJNARIRyzxCnxa4WQAAAAAAAAAYv6rQl1rwEGF72NGhRc1i-wAAAAAAAAAMZJExOMsbBfy8WqRgcylCLwAAAAAAAABpLSgILL0Q8I6llgRQX49QDQAAAAAAAADHbUctjbWcypk0wJUtGkKmFgAAAAAAAADJJwuJ_YrJlPmIg66vmsQiFAAAAAAAAAAz6KK4nudrIWi-u7NWSV5S7QAAAAAAAABnEBqXZsoPGK2F0IF3gcVlzwAAAAAAAAAaIunvE7sWPSz4YCgk0UVsFAAAAAAAAAAFObVu4EglQ3fhG0H69eyyrwAAAAAAAAAYW80Dopg4GIca7JXLWHFwVAAAAAAAAAA6PKIQdknzONDqr2S2uRbbjQAAAAAAAACCRJOpyLKTKlv2enKt_xrNSwAAAAAAAAA73-s2ANt1vNwpWiAFGYs2eAAAAAAAAAAFdWyxTJ01gHAXEF1z9sDOKwAAAAAAAAAOBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABJG_5wUb6Q1WN6hfRJFcqEwAAAAAAAAATAo_nvps-se9i_u9kVJi0LQAAAAAAAAAKXrSW2Ro9Ws10jZmj20O0hQAAAAAAAAADHSftxLHu5xLDzJX5F8kBXAAAAAAAAAGVBASYhiEtRRk5wvHC9yWtCgAAAAAAAABV2bJMveL01ajlzm-wdxWvUAAAAAAAAAAIy6xtuPPjpVFeK4CXqSN6fQAAAAAAAAHc6TUXIWC2WcF-_43xKp-wtQAAAAAAAAETuaLlXo_jail-f0Vd8f8fCwAAAAAAAACDIir7eGdVZsjNItLfykxs3gAAAAAAAABaNWk7GQCtDStIuwl83iGqTwAAAAAAAAAv2CwcPQcRkPoyzD89B8CtBQAAAAAAAAB-OoUyVxHN2rYPjLirZK4ihQAAAAAAAAAGf_uYAlUc4Mk4nBdvFkWUPgAAAAAAAAMEGASd-GxTI3CnNP2dmtJ0gQAAAAAAAAHeLeaqrhFv22rb1wzUlEMwAgAAAAAAAAAMkbApFhDbueqDTdsjbbOMUQAAAAAAAAAg7z0vK5_lN4w4BvVehfj4wQAAAAAAAAALTrTEyZn7B30gBbM8MLULSQAAAAAAAAA5ZQ0AxrpV-kvXnFTqilFT7wAAAAAAAAAfK8SfjEwLlz1-kQVCzD5OrAAAAAAAAAAOp_xoiXGCsXgpBNHrp24JKQAAAAAAAAAV5jDiO_Lgv2e10LojWCULCwAAAAAAAAAFOX6pd1GZ1KHmZAikJOZD-wAAAAAAAABoh6nIgR7I9-N28cV-Wv3K3AAAAAAAAAFBCqi1_91FMYO-otI3ESGKwAAAAAAAAAALVo9H_8YeBmc0_i3CIsMkrwAAAAAAAAAAJsaw4aRwbsrSC4mkOwhwQQAAAAAAANJa8NlMClggx8jaziUTJXlmWAAAAAAAATrTlHp5_WAgpLy2alrUVab6HAAAAAAAwACLlHp5_WAgpLy2alrUVab6HAAAAAAAAEIGlHp5_WAgpLy2alrUVab6HAAAAAAALSp_lHp5_WAgpLy2alrUVab6HAAAAAAALSjZlHp5_WAgpLy2alrUVab6HAAAAAAALpP6lHp5_WAgpLy2alrUVab6HAAAAAAALmmhlHp5_WAgpLy2alrUVab6HAAAAAAALkVrlHp5_WAgpLy2alrUVab6HAAAAAAALkColHp5_WAgpLy2alrUVab6HAAAAAAALjanlHp5_WAgpLy2alrUVab6HAAAAAAAL0cjlHp5_WAgpLy2alrUVab6HAAAAAAAD8cD", "types": "CAMwBQIDCwQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "828nO5IsKeisTjrd1qZGjA"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYIqvacTmpSpuAOHxTh9l_exAAAAAAAv8CwqvacTmpSpuAOHxTh9l_exAAAAAAA2XfPqvacTmpSpuAOHxTh9l_exAAAAAAA2TYhqvacTmpSpuAOHxTh9l_exAAAAAAAjXK5qvacTmpSpuAOHxTh9l_exAAAAAAAgN46qvacTmpSpuAOHxTh9l_exAAAAAAAgMZEAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABch4dwrMYlRFRjyfsvjXt4tgAAAAAAAAAg3V-8FLy1GH8nVRceMDeaiwAAAAAAAAABnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAV4j8yS0qsC_6XfTfMdPp5KQAAAAAAAAAQ9oBnE4xnAvOiOv1q-LbApgAAAAAAAAAEKjF0JPI3hCXDTCc6Z7ofdQAAAAAAAAFLAFikCbtP_Dm7iUthjnlnEgAAAAAAAAEq56q5trA0bAF1B-Um6L_rqwAAAAAAAAAGgi_774C-EJhuJfyXXhzVgwAAAAAAAABEgvYbo0YBmE65VwrpTWYalQAAAAAAAAB2tMqbgEmfZJ47YRogSA-gKgAAAAAAAADlKz3aUSMwqIxIxaAplngd8AAAAAAAAACXezRRC2sXv2hHIZbqwpv5zQAAAAAAAADSu-D_h2osKIeJ8uQjOobu_AAAAAAAAAHywnSBrxGSumHiAQQABJeNtQAAAAAAAAAkPK6VPfk6aJqBe-5Qji8O5gAAAAAAAAAFEIxfgHbDbI5dElFzd3Ha-QAAAAAAAAAZFq10nEfKWtXEt510UwEUUAAAAAAAAAB7V_QMdmt4RxKxn4ZNgdvkJwAAAAAAAAAReNITicG0MvFr9HQHk70FLAAAAAAAAAAIvweqY124jM8xgz5HvRfnbAAAAAAAAAA9vzzPIR5tUnMkJ7d_ITdQRgAAAAAAAAAC6YIeLAztuVSewvuGh8XKXgAAAAAAAAAFIQvpHpp20NHD-0mZNf95oAAAAAAAAABp0vAOoRRxsQcS4vDapC3-mwAAAAAAAAANqnvWBP24iZLcQ-Wi76ZDxQAAAAAAAAAI3X9PCd1tVPhzrMiwigfodgAAAAAAAAAAqvacTmpSpuAOHxTh9l_exAAAAAAA3Hr-vQ7EacG6CR4rzcFaNFEsTQAAAAAAAQGMlHp5_WAgpLy2alrUVab6HAAAAAAAwACLlHp5_WAgpLy2alrUVab6HAAAAAAAAEIGlHp5_WAgpLy2alrUVab6HAAAAAAAFFQelHp5_WAgpLy2alrUVab6HAAAAAAAFErelHp5_WAgpLy2alrUVab6HAAAAAAAFBxMlHp5_WAgpLy2alrUVab6HAAAAAAADSlTlHp5_WAgpLy2alrUVab6HAAAAAAADSU7lHp5_WAgpLy2alrUVab6HAAAAAAAp1q0", "types": "CAMfBQIDCAQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "Cf-uAEYevmDh96dG8LiA7w"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYIqvacTmpSpuAOHxTh9l_exAAAAAAAv8CwqvacTmpSpuAOHxTh9l_exAAAAAAA2XfPqvacTmpSpuAOHxTh9l_exAAAAAAA2TYhqvacTmpSpuAOHxTh9l_exAAAAAAAjXK5qvacTmpSpuAOHxTh9l_exAAAAAAAgN46qvacTmpSpuAOHxTh9l_exAAAAAAAgMZEAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABDJLJRrJNARIRyzxCnxa4WQAAAAAAAAAYmafpkLzxyAOwBxGkDhiVNAAAAAAAAAAMZJExOMsbBfy8WqRgcylCLwAAAAAAAABp8T1GTOmmtW2zNfDAiA-RCgAAAAAAAADHUu1pGUpLQ35aan6Rej1VEwAAAAAAAADJJwuJ_YrJlPmIg66vmsQiFAAAAAAAAAAz6KK4nudrIWi-u7NWSV5S7QAAAAAAAABnEBqXZsoPGK2F0IF3gcVlzwAAAAAAAAAaIunvE7sWPSz4YCgk0UVsFAAAAAAAAAAFradvx3ka7U7YJiCSRTJ3kwAAAAAAAAAYW80Dopg4GIca7JXLWHFwVAAAAAAAAAA6mZnKLpDGXH5r07376P6zgQAAAAAAAACCRJOpyLKTKlv2enKt_xrNSwAAAAAAAAA73-s2ANt1vNwpWiAFGYs2eAAAAAAAAAAFdWyxTJ01gHAXEF1z9sDOKwAAAAAAAAAOBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABJG_5wUb6Q1WN6hfRJFcqEwAAAAAAAAAT3r1MnCyPIgP6mYXLoLcz9AAAAAAAAAAKXrSW2Ro9Ws10jZmj20O0hQAAAAAAAAADHSftxLHu5xLDzJX5F8kBXAAAAAAAAAGVVTe2bxq-Jr5s7wFSht9_vgAAAAAAAABqC-DRD-J_UvsXUc59TOzOVAAAAAAAAAAI8AUzmHhqUZOHWH8Bccl_sQAAAAAAAAHc6TUXIWC2WcF-_43xKp-wtQAAAAAAAAETd0aX3KonLcPjMEO-H4cl5QAAAAAAAACDIir7eGdVZsjNItLfykxs3gAAAAAAAABaNWk7GQCtDStIuwl83iGqTwAAAAAAAAAv2CwcPQcRkPoyzD89B8CtBQAAAAAAAAB-OoUyVxHN2rYPjLirZK4ihQAAAAAAAAAGVWOUd0jMGfSYTm8Ns8_hDgAAAAAAAAMEGASd-GxTI3CnNP2dmtJ0gQAAAAAAAAHeLeaqrhFv22rb1wzUlEMwAgAAAAAAAAAMkbApFhDbueqDTdsjbbOMUQAAAAAAAAAg7z0vK5_lN4w4BvVehfj4wQAAAAAAAAALTrTEyZn7B30gBbM8MLULSQAAAAAAAABBvtg0KUtDPlzazpv9gP6MtQAAAAAAAABQOYpTnRQxT_yM-wTPYnG49wAAAAAAAAAC_OOeQuiXtvgynmE43QMbEAAAAAAAAAAAEkX-Bbjdn5I_vHMz9xIc5gAAAAAAAHukvQ7EacG6CR4rzcFaNFEsTQAAAAAAATk7", "types": "CAMrBQID"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "TEeScu0pOzeesTxGaw4JAg"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAACuyaSy1RHb0tl4hOkUXXLXQQAAAAAAAAAzGUwipo03N4glyS6268EEmgAAAAAAAAAKds9yTvk8UzKAgDSjs6pNkgAAAAAAAAANzVmgO3uV6bZtx2XUjAkXcgAAAAAAAAAHNJPFmv3cYB4fnHHuI2lrYQAAAAAAAABbsh1T4lVaqf9yZbH1lhPrZwAAAAAAAACNduOfgYTG-Pz3qxfosjBOKgAAAAAAAAAEFTD_DMq9LCr2rMh4XAWQegAAAAAAAAAF5wGUH60A7iQqyWZA8jj7kwAAAAAAAAAdDPKZG86WCe-rkFTx_IAdagAAAAAAAAAIhCl4jaZsLnxk3C0-qCjENwAAAAAAAAAFfMf02bFW9UOsHXaY1ju3UAAAAAAAAAELie51PKalSPz0oTw25hOjdgAAAAAAAAAHxhzTSPDm8vP1CtKm1pHa_AAAAAAAAADX3SU4n_eVvbLOxW0zu9gonAAAAAAAAAATBULXlXluMevM23wGn-scGgAAAAAAAABIX3YYuA6U4OM2ZjQu8fVwYwAAAAAAAABLSNDYMaokxZpNhjFZnC0cqQAAAAAAAAABm_DK0nAVx1tMIimjFYyQPAAAAAAAAAAS_B6c9784WlLvFF9uk3pSswAAAAAAAAAcFhP6JufLZYYhuADwCLvRKAAAAAAAAACVClvIuFYr5ArIOKSAaLGqggAAAAAAAAGv5tsfFOiB0XQGtxfawg0JiQAAAAAAAAGPe8ARrqT9JmZcuLd6G8ZTMgAAAAAAAAAGV6Xv8yqxVPgM9JZWMvD_ogAAAAAAAAADS4xo2K-7S9Psi5KVtkSU0AAAAAAAAAAONNX6LKmc60fQ-V_Nw2_9cAAAAAAAAACwVGJu-bYJBqeYrugO5COQdQAAAAAAAAHGAxJNlcXBvrgVOpR8a1WEYgAAAAAAAAG2n0mWG92K0zbUz5S-_ixlIAAAAAAAAABrKxHEggHQvpMo7NXk9m-aKgAAAAAAAAAghvijQK1a5iiWkFoVp40y5AAAAAAAAAAGvRwDdjzKXpreYxi90MDGQQAAAAAAAABms-5AzRQS0AyjARcxs2D2DQAAAAAAAAAX4Wda4DEj07iGWB9mKOvWMAAAAAAAAABo45faa5L8TTwA9zJYlmGKywAAAAAAAAAUq3uedK83ZBGp21KqIIcD7QAAAAAAAACWwh1zX_oqIoKNefuSr1ngvgAAAAAAAAAfPu2JWw096K4wEF2usuCt-AAAAAAAAABc_gZdvwQP6FQ5RcMsyYjAXwAAAAAAAAFD1PIPTBbtPHUrg0d3N2nuygAAAAAAAABBlt4DCYuCTF_qGLbdRnIhjAAAAAAAAADDFERPJCfrzkqs-7XhU53jhwAAAAAAAAFtzP5HzbxY7goeRVwj37voeQAAAAAAAAACy87oVbGEsnZxcfXcdrLYswAAAAAAAAB6i1t0pdr_xnHfk_UsEnpDiwAAAAAAAAAMip4jveo5OesPxqJEnXrM0wAAAAAAAAAOrDli4oCQ8GWnljRH2_1bLAAAAAAAAAEZH2XMIGvXG_73d5XzBvFSxAAAAAAAAAEB3D_A_sI8XDgpK1lgEmfWZgAAAAAAAAC_hqzAzTCpj-Nsh96skuh10wAAAAAAAABJibtF619pQHGNKblUfb9MKwAAAAAAAAANHJHMeViEmhpbdveA5zS4zgAAAAAAAADUNW0MvXdcfyNB_rjxH_JHXgAAAAAAAAAN16j8HtYPNjd3qdRoXnMi0AAAAAAAAABaO4wSxfOE-c6Sft1wmb_6fgAAAAAAAABpoKMpPO_rrguOLYx8VBTGhgAAAAAAAAAGakj2EbMcV1BY1x-2V-R6VQAAAAAAAAANqND66z7potQhxSJ1H4ezHgAAAAAAAAAG2B-rbS8KhbB8ePRnpVf7_QAAAAAAAAAU89ogc5uTbOqh2aLJlzHKmAAAAAAAAAAX9ccfl0YdabzwSUY2PdYA0QAAAAAAAAATDT06D5A7Yxiqx6IJ7Kvu0wAAAAAAAABIhfMR5YVLU22vPk_NOG-TgAAAAAAAAAAXyknYMrbzZKxPQGFhtsDUFgAAAAAAAAAMTd592FDxi26zl_r4mKuaXAAAAAAAAAAERooPjsQ01TWhvqGm4rD4pAAAAAAAAAAFc6kGRam5UKx_l-9ot9811wAAAAAAAACL8KbuOHlqa1mTqGljNuuiVAAAAAAAAAAF7F9XBiRMgggoDE1ZbLLQRgAAAAAAAAAH5TLUwI_khEb2-zaj4FK_ZQAAAAAAAAAaxpXYHQL7vamCXb04_96hlgAAAAAAAAALlTaOxl3xTWrzN2-LwynoLwAAAAAAAAAA", "types": "CANWBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "qRkm6b_RZPI0741rrxfzIA"}} +{"Stacktrace": {"frame": {"ids": "cEvvhuguaGYCmGWrnWRz1QAAAAACFwqgcEvvhuguaGYCmGWrnWRz1QAAAAADhEEZcEvvhuguaGYCmGWrnWRz1QAAAAACsrSecEvvhuguaGYCmGWrnWRz1QAAAAACsroGcEvvhuguaGYCmGWrnWRz1QAAAAACsosccEvvhuguaGYCmGWrnWRz1QAAAAACso6VcEvvhuguaGYCmGWrnWRz1QAAAAACsp77cEvvhuguaGYCmGWrnWRz1QAAAAACXrhycEvvhuguaGYCmGWrnWRz1QAAAAACXrumcEvvhuguaGYCmGWrnWRz1QAAAAACXuSNcEvvhuguaGYCmGWrnWRz1QAAAAACXrumcEvvhuguaGYCmGWrnWRz1QAAAAACXtc2cEvvhuguaGYCmGWrnWRz1QAAAAACXssucEvvhuguaGYCmGWrnWRz1QAAAAACXtfdcEvvhuguaGYCmGWrnWRz1QAAAAACXvWEcEvvhuguaGYCmGWrnWRz1QAAAAACLjD5cEvvhuguaGYCmGWrnWRz1QAAAAACLjdJcEvvhuguaGYCmGWrnWRz1QAAAAACLj_YcEvvhuguaGYCmGWrnWRz1QAAAAACHPCM", "types": "EwM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "CT7Jn1_cfRttYWpYz0WvHA"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYIqvacTmpSpuAOHxTh9l_exAAAAAAAv8CwqvacTmpSpuAOHxTh9l_exAAAAAAA2XfPqvacTmpSpuAOHxTh9l_exAAAAAAA2TYhqvacTmpSpuAOHxTh9l_exAAAAAAAjXK5qvacTmpSpuAOHxTh9l_exAAAAAAAgN46qvacTmpSpuAOHxTh9l_exAAAAAAAgMZEAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABch4dwrMYlRFRjyfsvjXt4tgAAAAAAAAAg3V-8FLy1GH8nVRceMDeaiwAAAAAAAAABnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAV4j8yS0qsC_6XfTfMdPp5KQAAAAAAAAAQ9oBnE4xnAvOiOv1q-LbApgAAAAAAAAAEKjF0JPI3hCXDTCc6Z7ofdQAAAAAAAAFLAFikCbtP_Dm7iUthjnlnEgAAAAAAAAEq56q5trA0bAF1B-Um6L_rqwAAAAAAAAAGgi_774C-EJhuJfyXXhzVgwAAAAAAAABEgvYbo0YBmE65VwrpTWYalQAAAAAAAABiAvk7vHT4UprOTP5MohWAugAAAAAAAAAa3Opn5CZ2iYMPW1WE6HYQfAAAAAAAAAAE", "types": "CAMQBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "uZJLF8SFtg9OqPXwQZVelg"}} +{"Stacktrace": {"frame": {"ids": "634wiWh6F21tPpXr0Zz3mgAAAAAAEfFi8NlMClggx8jaziUTJXlmWAAAAAAAAIYIZSkKN3zNxr0HYuO2pqe5hQAAAAAAwcBwZSkKN3zNxr0HYuO2pqe5hQAAAAAA5ECvZSkKN3zNxr0HYuO2pqe5hQAAAAAA4_9_ZSkKN3zNxr0HYuO2pqe5hQAAAAAAj7b5ZSkKN3zNxr0HYuO2pqe5hQAAAAAAgwXKZSkKN3zNxr0HYuO2pqe5hQAAAAAAgu3UAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcBtfAnw52Zh-pzF1hN6odGgAAAAAAAAABDJLJRrJNARIRyzxCnxa4WQAAAAAAAAAYmcb8sJ6l3hl2564ZMljLvwAAAAAAAAAINeO57Qkcge62xvmelDstIAAAAAAAAAAoL4TTplFq0_54QvTmQIK7TgAAAAAAAAAScMPxRfIRSUAO9TVG-M8JpAAAAAAAAAB0_iIILZSpQek0elrVmCyWigAAAAAAAAAgE0bQU1G2uDlA0HZGujxA9wAAAAAAAAAEigHK0MNBkupTzE8Gzv6EygAAAAAAAAAJplZYNE8qhJOfCM58L5mfyQAAAAAAAAAdMKXd2GM1KQ-HCXB8XbvuBAAAAAAAAAAQ92n0tgLpue4WLe7eq6vpuwAAAAAAAAAWsxq3K3ppB_xqEWb8y-RbYwAAAAAAAAC8SlqES-nwtjqDlYXOe7K1YAAAAAAAAAAJFTD_DMq9LCr2rMh4XAWQegAAAAAAAAAF5wGUH60A7iQqyWZA8jj7kwAAAAAAAAAdmZJt6uLSgudylMZ0eHd7CAAAAAAAAAAEQkrVHFzrrNPXJryM_udmYQAAAAAAAAAB8__nCHyreLVA7r_BF4RklwAAAAAAAAAIgWzF_k06817SsAPAEeEuVAAAAAAAAAAFP8GYPQV63_J7gmzBFEQnJwAAAAAAAAFgmiWKFzr_hZhlQAnEjidKYAAAAAAAAABITvbloCY34U08znkUfz6GbAAAAAAAAAAU-lMJxP-0bZiF5XB2gMdbVwAAAAAAAAAGYxIrzYSolBsbC9VHbJjZmgAAAAAAAAAH_2Sq1jeupJuJ-QB3TmIqcwAAAAAAAADXCQI8nGGXTZm2Anxc2rVzlAAAAAAAAAAT2V2dRNf-iGCAuWhm4oOfRAAAAAAAAABE5iqVHRdlT1dn5BdxVeiR7QAAAAAAAABLUI6vy2YkxZpNhPhBVD9zrQAAAAAAAAABWIcSMVTq4UI8IrRuG9Ls8gAAAAAAAAAS6bLW5yM_cMbSli8_NsjzNQAAAAAAAAAcKACWjtDKKsY3WzryMLMuQgAAAAAAAACNbOTvPcD8pPP1fG6-CUmg6wAAAAAAAAGvHtE3mFX5tbjDXSQ7-hE7bwAAAAAAAAF5e8ARrqT9JmZcuLd6G8ZTMgAAAAAAAAAGV6Xv8yqxVPgM9JZWMvD_ogAAAAAAAAADS4xo2K-7S9Psi5KVtkSU0AAAAAAAAAAONNX6LKmc60fQ-V_Nw2_9cAAAAAAAAACwVGJu-bYJBqeYrugO5COQdQAAAAAAAAHGAxJNlcXBvrgVOpR8a1WEYgAAAAAAAALRiM0ISMJkW1PbadMrIfDN5gAAAAAAAAAZc2BpwrDKQx3haZVDD56LgAAAAAAAAABsnENxGWN4qoBcDN73jfy1WgAAAAAAAADf1QVyanYcPpNfuP3xZ0ogLQAAAAAAAAAFqAh8Zr_sGDY6rzZXJvta_gAAAAAAAAAfxBi4pQ8utWGRmJVqaVBkxwAAAAAAAAAfOhy0c1RhzpJIxIHtdNOBrgAAAAAAAABerjyp8aib7sZjDzshnnXJQQAAAAAAAABBc8gmQiPjvAnokd-rAm4_OQAAAAAAAABHrXmzLUA4HH3snMulWKOyNgAAAAAAAAAWXxvzEXolyPiyR3WZkqmNewAAAAAAAAAQ8A-e4vMGSQm_8vtVRynoJAAAAAAAAADLNaokQDqfrb6dyCBpANIkNwAAAAAAAAEvqzb1PmzSJpYKqqQC4Es1dAAAAAAAAAAHBjHQJvdwWPCTBjvnSeRZ2wAAAAAAAAAHid_Nq5V40zsa5EMiEnSMswAAAAAAAAA0UJLc0zscqzwPwar7clkaagAAAAAAAAAPtgCBGeongyDKoP0eQjcOvQAAAAAAAAA3s1ge7c8paVPaLi3Du1Dh1QAAAAAAAAAE2Swt2qN1JJo2lNqpZIN9-wAAAAAAAAAAJsaw4aRwbsrSC4mkOwhwQQAAAAAAAINa8NlMClggx8jaziUTJXlmWAAAAAAAATLOlHp5_WAgpLy2alrUVab6HAAAAAAAwACLlHp5_WAgpLy2alrUVab6HAAAAAAAAEIGlHp5_WAgpLy2alrUVab6HAAAAAAALW4JlHp5_WAgpLy2alrUVab6HAAAAAAALW22lHp5_WAgpLy2alrUVab6HAAAAAAALUfwlHp5_WAgpLy2alrUVab6HAAAAAAALTsYlHp5_WAgpLy2alrUVab6HAAAAAAALTpUlHp5_WAgpLy2alrUVab6HAAAAAAAimwClHp5_WAgpLy2alrUVab6HAAAAAAAimtblHp5_WAgpLy2alrUVab6HAAAAAAAmOK4lHp5_WAgpLy2alrUVab6HAAAAAAAlcA7lHp5_WAgpLy2alrUVab6HAAAAAAAlbpNlHp5_WAgpLy2alrUVab6HAAAAAAAUd51lHp5_WAgpLy2alrUVab6HAAAAAAApYft", "types": "CANBBQIDDgQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "bA1f839p2HCzC7fcTWp8Kw"}} +{"Stacktrace": {"frame": {"ids": "Z3CKaS4aU08RhYA19y5ITQAAAAABubEQZ3CKaS4aU08RhYA19y5ITQAAAAAC1On5Z3CKaS4aU08RhYA19y5ITQAAAAAC19YhZ3CKaS4aU08RhYA19y5ITQAAAAAC14rZZ3CKaS4aU08RhYA19y5ITQAAAAACs8h3Z3CKaS4aU08RhYA19y5ITQAAAAACqk0gZ3CKaS4aU08RhYA19y5ITQAAAAACIAqaZ3CKaS4aU08RhYA19y5ITQAAAAACtAiPZ3CKaS4aU08RhYA19y5ITQAAAAACsxfDZ3CKaS4aU08RhYA19y5ITQAAAAACIA1aZ3CKaS4aU08RhYA19y5ITQAAAAACIbK8Z3CKaS4aU08RhYA19y5ITQAAAAACIP7qZ3CKaS4aU08RhYA19y5ITQAAAAABtFAmZ3CKaS4aU08RhYA19y5ITQAAAAABtEB1", "types": "DgM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "Hwc9ojSnNRegxwpOeXsNnw"}} +{"Stacktrace": {"frame": {"ids": "634wiWh6F21tPpXr0Zz3mgAAAAAAEfFi8NlMClggx8jaziUTJXlmWAAAAAAAAIYIZSkKN3zNxr0HYuO2pqe5hQAAAAAAwcBwZSkKN3zNxr0HYuO2pqe5hQAAAAAA5ECvZSkKN3zNxr0HYuO2pqe5hQAAAAAA4_9_ZSkKN3zNxr0HYuO2pqe5hQAAAAAAj7b5ZSkKN3zNxr0HYuO2pqe5hQAAAAAAgwXKZSkKN3zNxr0HYuO2pqe5hQAAAAAAgu3UAAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qZgY89pr05YIxi0DTL7hyTAAAAAAAAAALzZZ6VCjFYAFVAKtY0XlyPwAAAAAAAAAFySPx-89oJ6TfXYn-uir7mQAAAAAAAABcnVB2vvQdnm3M5BpEt6xnFAAAAAAAAAAEXqZ8EmLPraAc01S4qiJfWgAAAAAAAAAZGabeEr60puRs20AAHoWA0AAAAAAAAAAInGPKMkILxfXIjt9GRbMXwAAAAAAAAABYcn7jRZ-ZvDDBUMM5Rer9wAAAAAAAAAAD5IpWfunjiW7HmwJPoVX8WQAAAAAAAAAO11cfhNddX4HypzZBkL8GJAAAAAAAAAAN5Ga434JXyMCORvI3aSPl0wAAAAAAAAAxRyec96xv7uLWO7iiTNqYKgAAAAAAAAAmr4QCFwkajSWVBltX9dOYSwAAAAAAAAAA", "types": "CAMOBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "ehqmjj-2Kv8A1vCgGKyP2Q"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABPeVkW0TQ9c5aTnXLXV2plwAAAAAAAAAnP7mHtlyIKp9O861pj2g7OwAAAAAAAAAdBPzs7KzDmaKPMROQHmdZzAAAAAAAAAAFvM_fQkfMwYX5ebFDxVQdtwAAAAAAAAAGIpKkMAR_dWJeYgHr6Pp56AAAAAAAAAABRQiOAUVYPrJNPqfM54o9YwAAAAAAAAALSCkRMoBKKPg9IolzXt51kAAAAAAAAAAIY8sJdN8JrOKeDGcKRLKzYgAAAAAAAACuyaSy1RHb0tl4hOkUXXLXQQAAAAAAAAAzGUwipo03N4glyS6268EEmgAAAAAAAAAXC7Gr60ZG8XU9_R3ucXNk1wAAAAAAAAErrLErn8W8_gFwO7zvGO-LBwAAAAAAAAAIOUMeXZZ6fO-t9RTzPCiRtQAAAAAAAAABKZn_I-ZtnojJ7fIDZHHneQAAAAAAAADy92JDYcQA4cXvpv7axYYOPwAAAAAAAAAouBDO9BMjtjTmzHTlTfDLzAAAAAAAAAAECGeCGw7geerub3AylJt7SwAAAAAAAAABA0rFfZQgIUFzz7A2AHAg7gAAAAAAAAAE", "types": "CAMXBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "fL6FlEuXKQQ1vbZSuOkNDA"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA76Af8DDAjVJl721y2vxKHGSBegAAAAAAdJED8DDAjVJl721y2vxKHGSBegAAAAAAdA-2", "types": "BwM"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "FWCWmB5KaAfOjnip-rDwUQ"}} +{"Stacktrace": {"frame": {"ids": "qjB_amR59Lv13Nhle_NwbwAAAAAAEfEyvQ7EacG6CR4rzcFaNFEsTQAAAAAAAIYI8DDAjVJl721y2vxKHGSBegAAAAAAyUNH8DDAjVJl721y2vxKHGSBegAAAAAA5HKP8DDAjVJl721y2vxKHGSBegAAAAAA49it8DDAjVJl721y2vxKHGSBegAAAAAAlT598DDAjVJl721y2vxKHGSBegAAAAAAiFDh8DDAjVJl721y2vxKHGSBegAAAAAAiDi0AAAAAAAAV4sAAAAAAAAAHezOBBlhpr8qytcfMbv--KUIhYho462CJAAAAAAAAAAN1LqFeo_BgrVHpNhRzkRnIAAAAAAAAAAF8NHfQtcyI14VYOjZC2v_gwAAAAAAAABcv7DgkUS5-ysky2RXxPMSjAAAAAAAAAAgRCG4Qx5E9EZ0R-BBEbZnmwAAAAAAAAABk8v4vce5CPjZaoKS8wiSZwAAAAAAAAAVV2OKkEZQbgFjMzVwaTkytwAAAAAAAAAQ3tyP2MUj5DlzxqSh6DRnAwAAAAAAAAFkZ737A4mvoWuasLPvz8LMuAAAAAAAAABDCKOgldhpJkxI6GxtBnkMFwAAAAAAAABIDP-R410tjxklQGgyNbSilQAAAAAAAADK", "types": "CAMMBQ"}}, "ecs": {"version": "1.12.0"}} +{"create": {"_index": "profiling-stacktraces", "_id": "ZnXwrIATmk5PmXMsv4nKtg"}} +{"Stacktrace": {"frame": {"ids": "-FiXU0afkg-sRwubj2WiAwAAAAAAAKlJUJmIMEc6K8gJ39NHYaD0WwAAAAAAAhyG-FiXU0afkg-sRwubj2WiAwAAAAAAAKMr-FiXU0afkg-sRwubj2WiAwAAAAAAAmE0-FiXU0afkg-sRwubj2WiAwAAAAAAAlz6-FiXU0afkg-sRwubj2WiAwAAAAAAAla0Fuiq0RN8VMfwSLHsA6DzhAAAAAAAATq8-FiXU0afkg-sRwubj2WiAwAAAAAAAZtb-FiXU0afkg-sRwubj2WiAwAAAAAAAZdL-FiXU0afkg-sRwubj2WiAwAAAAAAAkK3-FiXU0afkg-sRwubj2WiAwAAAAAAAiUs", "types": "CwM"}}, "ecs": {"version": "1.12.0"}} From 9fdc67c845df9cce4acfdef51f51ab2b3b5f7ad4 Mon Sep 17 00:00:00 2001 From: Zing Zai <9292451+2gavy@users.noreply.github.com> Date: Thu, 9 Nov 2023 22:44:55 +0800 Subject: [PATCH 030/513] Update movfn-aggregation.asciidoc (#101959) Updated one-day to one-month interval to match the code --- docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc b/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc index b5f1315531916..44a00b9f5b99e 100644 --- a/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc +++ b/docs/reference/aggregations/pipeline/movfn-aggregation.asciidoc @@ -68,7 +68,7 @@ POST /_search -------------------------------------------------- // TEST[setup:sales] -<1> A `date_histogram` named "my_date_histo" is constructed on the "timestamp" field, with one-day intervals +<1> A `date_histogram` named "my_date_histo" is constructed on the "timestamp" field, with one-month intervals <2> A `sum` metric is used to calculate the sum of a field. This could be any numeric metric (sum, min, max, etc) <3> Finally, we specify a `moving_fn` aggregation which uses "the_sum" metric as its input. From 2d4c60f0bf605ac7575058e7b07063d8976ea8b6 Mon Sep 17 00:00:00 2001 From: Iraklis Psaroudakis Date: Thu, 9 Nov 2023 17:38:24 +0200 Subject: [PATCH 031/513] Wait for term gen should wait for started shard (#101792) Relates ES-7223 --- .../elasticsearch/index/shard/IndexShard.java | 23 +++++++++++++- .../index/shard/IndexShardTests.java | 30 +++++++++++++++++++ 2 files changed, 52 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index f4812f280f917..2491d13784483 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -285,6 +285,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl private final LongSupplier relativeTimeInNanosSupplier; private volatile long startedRelativeTimeInNanos; private volatile long indexingTimeBeforeShardStartedInNanos; + private final SubscribableListener waitForEngineOrClosedShardListeners = new SubscribableListener<>(); // the translog keeps track of the GCP, but unpromotable shards have no translog so we need to track the GCP here instead private volatile long globalCheckPointIfUnpromotable; @@ -1658,6 +1659,7 @@ public void close(String reason, boolean flushEngine) throws IOException { synchronized (mutex) { changeState(IndexShardState.CLOSED, reason); } + checkAndCallWaitForEngineOrClosedShardListeners(); } finally { final Engine engine = this.currentEngineReference.getAndSet(null); try { @@ -2016,6 +2018,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) t onSettingsChanged(); assert assertSequenceNumbersInCommit(); recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); + checkAndCallWaitForEngineOrClosedShardListeners(); } private boolean assertSequenceNumbersInCommit() throws IOException { @@ -4181,10 +4184,28 @@ public void waitForSegmentGeneration(long segmentGeneration, ActionListener listener) { + waitForEngineOrClosedShardListeners.addListener(listener); + } + /** * Registers a listener for an event when the shard advances to the provided primary term and segment generation */ public void waitForPrimaryTermAndGeneration(long primaryTerm, long segmentGeneration, ActionListener listener) { - getEngine().addPrimaryTermAndGenerationListener(primaryTerm, segmentGeneration, listener); + waitForEngineOrClosedShard( + listener.delegateFailureAndWrap( + (l, ignored) -> getEngine().addPrimaryTermAndGenerationListener(primaryTerm, segmentGeneration, l) + ) + ); } + } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index d4560fb6d765e..9765618e05e34 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -3088,6 +3088,36 @@ public void finalizeRecovery(long globalCheckpoint, long trimAboveSeqNo, ActionL closeShards(primary, replica); } + public void testWaitForEngineListener() throws IOException { + Settings settings = indexSettings(IndexVersion.current(), 1, 1).build(); + IndexMetadata metadata = IndexMetadata.builder("test").putMapping(""" + { "properties": { "foo": { "type": "text"}}}""").settings(settings).primaryTerm(0, 1).build(); + IndexShard primary = newShard(new ShardId(metadata.getIndex(), 0), true, "n1", metadata, null); + + AtomicBoolean called = new AtomicBoolean(false); + primary.waitForEngineOrClosedShard(ActionListener.running(() -> called.set(true))); + assertThat("listener should not have been called yet", called.get(), equalTo(false)); + + recoverShardFromStore(primary); + assertThat("listener should have been called", called.get(), equalTo(true)); + + closeShards(primary); + } + + public void testWaitForClosedListener() throws IOException { + Settings settings = indexSettings(IndexVersion.current(), 1, 1).build(); + IndexMetadata metadata = IndexMetadata.builder("test").putMapping(""" + { "properties": { "foo": { "type": "text"}}}""").settings(settings).primaryTerm(0, 1).build(); + IndexShard primary = newShard(new ShardId(metadata.getIndex(), 0), true, "n1", metadata, null); + + AtomicBoolean called = new AtomicBoolean(false); + primary.waitForEngineOrClosedShard(ActionListener.running(() -> called.set(true))); + assertThat("listener should not have been called yet", called.get(), equalTo(false)); + + closeShards(primary); + assertThat("listener should have been called", called.get(), equalTo(true)); + } + public void testRecoverFromLocalShard() throws IOException { Settings settings = indexSettings(IndexVersion.current(), 1, 1).build(); IndexMetadata metadata = IndexMetadata.builder("source") From eb43e3ec787c6670ef1932402aa689a3c044491c Mon Sep 17 00:00:00 2001 From: Michael Peterson Date: Thu, 9 Nov 2023 11:12:19 -0500 Subject: [PATCH 032/513] Log shard errors when search/scroll returns partial results (#101750) Scroll actions that entirely fail already have their exceptions logged by the RestResponse object. With partial results, any shard failures are returned in the XContent response to users. This commit also logs those errors at WARN level, including stack trace. --- .../search/TransportSearchScrollAction.java | 23 ++++++++++++++++--- 1 file changed, 20 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchScrollAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchScrollAction.java index d097b10b7162d..9874bcfb56c6a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchScrollAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchScrollAction.java @@ -8,7 +8,11 @@ package org.elasticsearch.action.search; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.service.ClusterService; @@ -22,7 +26,7 @@ import static org.elasticsearch.action.search.TransportSearchHelper.parseScrollId; public class TransportSearchScrollAction extends HandledTransportAction { - + private static final Logger logger = LogManager.getLogger(TransportSearchScrollAction.class); private final ClusterService clusterService; private final SearchTransportService searchTransportService; @@ -40,6 +44,19 @@ public TransportSearchScrollAction( @Override protected void doExecute(Task task, SearchScrollRequest request, ActionListener listener) { + ActionListener loggingListener = listener.delegateFailureAndWrap((l, searchResponse) -> { + if (searchResponse.getShardFailures() != null && searchResponse.getShardFailures().length > 0) { + ShardOperationFailedException[] groupedFailures = ExceptionsHelper.groupBy(searchResponse.getShardFailures()); + for (ShardOperationFailedException f : groupedFailures) { + Throwable cause = f.getCause() == null ? f : f.getCause(); + if (ExceptionsHelper.status(cause).getStatus() >= 500 + && ExceptionsHelper.isNodeOrShardUnavailableTypeException(cause) == false) { + logger.warn("TransportSearchScrollAction shard failure (partial results response)", f); + } + } + } + l.onResponse(searchResponse); + }); try { ParsedScrollId scrollId = parseScrollId(request.scrollId()); Runnable action = switch (scrollId.getType()) { @@ -50,7 +67,7 @@ protected void doExecute(Task task, SearchScrollRequest request, ActionListener< request, (SearchTask) task, scrollId, - listener + loggingListener ); case QUERY_AND_FETCH_TYPE -> // TODO can we get rid of this? new SearchScrollQueryAndFetchAsyncAction( @@ -60,7 +77,7 @@ protected void doExecute(Task task, SearchScrollRequest request, ActionListener< request, (SearchTask) task, scrollId, - listener + loggingListener ); default -> throw new IllegalArgumentException("Scroll id type [" + scrollId.getType() + "] unrecognized"); }; From 1656f0522b206da42fa555a1beff8af902631f4e Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 9 Nov 2023 11:36:11 -0500 Subject: [PATCH 033/513] Fix incorrect dynamic mapping for non-numeric-value arrays #101965 (#101967) After https://github.com/elastic/elasticsearch/pull/98512 we incorrectly attempt to map an array of any single value type to dense_vector. Instead, we should validate that ALL mappers are numeric and that ALL of them are `float`. closes: https://github.com/elastic/elasticsearch/issues/101965 --- docs/changelog/101967.yaml | 5 +++ .../60_dense_vector_dynamic_mapping.yml | 31 +++++++++++++++++++ .../index/mapper/DocumentParser.java | 6 +++- .../index/mapper/DynamicMappingTests.java | 12 +++++++ 4 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 docs/changelog/101967.yaml diff --git a/docs/changelog/101967.yaml b/docs/changelog/101967.yaml new file mode 100644 index 0000000000000..84f188db1e30b --- /dev/null +++ b/docs/changelog/101967.yaml @@ -0,0 +1,5 @@ +pr: 101967 +summary: "Fix incorrect dynamic mapping for non-numeric-value arrays #101965" +area: Mapping +type: bug +issues: [] diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/60_dense_vector_dynamic_mapping.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/60_dense_vector_dynamic_mapping.yml index 151698482368a..8453bfbe297e4 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/60_dense_vector_dynamic_mapping.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/60_dense_vector_dynamic_mapping.yml @@ -2,7 +2,38 @@ setup: - skip: version: ' - 8.10.99' reason: 'Dynamic mapping of floats to dense_vector was added in 8.11' +--- +"Fields indexed as strings won't be transformed into dense_vector": + - skip: + # TODO adjust after backport + version: ' - 8.11.99' + reason: 'Dynamic mapping of things other than numbers is fixed in 8.12' + - do: + index: + index: strings-are-not-floats + refresh: true + body: + obviously_string: ["foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo", + "foo", "foo", "foo", "foo", "foo", "foo", "foo", "foo"] + - do: + cluster.health: + wait_for_events: languid + - do: + indices.get_mapping: + index: strings-are-not-floats + - match: { strings-are-not-floats.mappings.properties.obviously_string.type: text } --- "Fields with float arrays below the threshold still map as float": diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java index 996c6243064e9..17af6259ca27c 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentParser.java @@ -613,8 +613,12 @@ private static void postProcessDynamicArrayMapping(DocumentParserContext context || context.isCopyToField(fullFieldName) || mappers.size() < MIN_DIMS_FOR_DYNAMIC_FLOAT_MAPPING || mappers.size() > MAX_DIMS_COUNT + // Anything that is NOT a number or anything that IS a number but not mapped to `float` should NOT be mapped to dense_vector || mappers.stream() - .allMatch(m -> m instanceof NumberFieldMapper.Builder nb && nb.type != NumberFieldMapper.NumberType.FLOAT)) { + .anyMatch( + m -> m instanceof NumberFieldMapper.Builder == false + || ((NumberFieldMapper.Builder) m).type != NumberFieldMapper.NumberType.FLOAT + )) { return; } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java index cd04f81f0f355..4a2d74016615e 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DynamicMappingTests.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.time.Instant; +import java.util.stream.Stream; import static org.elasticsearch.index.mapper.vectors.DenseVectorFieldMapper.MAX_DIMS_COUNT; import static org.elasticsearch.index.mapper.vectors.DenseVectorFieldMapper.MIN_DIMS_FOR_DYNAMIC_FLOAT_MAPPING; @@ -991,4 +992,15 @@ public void testDefaultDenseVectorMappingsObject() throws IOException { ObjectMapper parent = (ObjectMapper) update.getRoot().getMapper("parent_object"); assertThat(((FieldMapper) parent.getMapper("mapsToDenseVector")).fieldType().typeName(), equalTo("dense_vector")); } + + public void testStringArraysAreText() throws IOException { + DocumentMapper mapper = createDocumentMapper(topMapping(b -> b.field("numeric_detection", true))); + BytesReference source = BytesReference.bytes( + XContentFactory.jsonBuilder().startObject().field("mapsToString", Stream.generate(() -> "foo").limit(129).toArray()).endObject() + ); + ParsedDocument parsedDocument = mapper.parse(new SourceToParse("id", source, XContentType.JSON)); + Mapping update = parsedDocument.dynamicMappingsUpdate(); + assertNotNull(update); + assertThat(((FieldMapper) update.getRoot().getMapper("mapsToString")).fieldType().typeName(), equalTo("text")); + } } From d6a2db62301dccf3f64fe820a833d366d1607b09 Mon Sep 17 00:00:00 2001 From: David Kyle Date: Thu, 9 Nov 2023 17:13:55 +0000 Subject: [PATCH 034/513] [ML] Fix inference timeout from the Inference Ingest Processor (#101971) When using the Inference ingest processor with the input_output configuration option timeout after 10 seconds. The timeout should be much higher for ingest where large numbers of requests can queue up and take longer to be processed. --- docs/changelog/101971.yaml | 5 ++ .../core/ml/action/InferModelAction.java | 48 +++++++++++++----- .../action/InferModelActionRequestTests.java | 8 +-- .../license/MachineLearningLicensingIT.java | 20 +++++--- .../integration/ModelInferenceActionIT.java | 50 +++++++++++++++---- .../inference/ingest/InferenceProcessor.java | 16 +++++- .../ml/queries/TextExpansionQueryBuilder.java | 4 +- .../TextEmbeddingQueryVectorBuilder.java | 4 +- .../ingest/InferenceProcessorTests.java | 18 +++++-- .../TextExpansionQueryBuilderTests.java | 1 + .../TextEmbeddingQueryVectorBuilderTests.java | 1 + 11 files changed, 133 insertions(+), 42 deletions(-) create mode 100644 docs/changelog/101971.yaml diff --git a/docs/changelog/101971.yaml b/docs/changelog/101971.yaml new file mode 100644 index 0000000000000..23fb5463bae79 --- /dev/null +++ b/docs/changelog/101971.yaml @@ -0,0 +1,5 @@ +pr: 101971 +summary: Fix inference timeout from the Inference Ingest Processor +area: Machine Learning +type: bug +issues: [] diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/InferModelAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/InferModelAction.java index a06d0fe0ce0ce..61e52935f46e9 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/InferModelAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/InferModelAction.java @@ -82,7 +82,7 @@ public static Builder parseRequest(String id, XContentParser parser) { private final List> objectsToInfer; private final InferenceConfigUpdate update; private final boolean previouslyLicensed; - private TimeValue inferenceTimeout; + private final TimeValue inferenceTimeout; // textInput added for uses that accept a query string // and do know which field the model expects to find its // input and so cannot construct a document. @@ -95,18 +95,32 @@ public static Builder parseRequest(String id, XContentParser parser) { * the inference queue for) is set to a high value {@code #DEFAULT_TIMEOUT_FOR_INGEST} * to prefer slow ingest over dropping documents. */ + + /** + * Build a request from a list of documents as maps. + * + * @param id The model Id + * @param docs List of document maps + * @param update Inference config update + * @param previouslyLicensed License has been checked previously + * and can now be skipped + * @param inferenceTimeout The inference timeout (how long the + * request waits in the inference queue for) + * @return the new Request + */ public static Request forIngestDocs( String id, List> docs, InferenceConfigUpdate update, - boolean previouslyLicensed + boolean previouslyLicensed, + TimeValue inferenceTimeout ) { return new Request( ExceptionsHelper.requireNonNull(id, InferModelAction.Request.ID), update, ExceptionsHelper.requireNonNull(Collections.unmodifiableList(docs), DOCS), null, - DEFAULT_TIMEOUT_FOR_INGEST, + inferenceTimeout, previouslyLicensed ); } @@ -114,17 +128,30 @@ public static Request forIngestDocs( /** * Build a request from a list of strings, each string * is one evaluation of the model. - * The inference timeout (how long the request waits in - * the inference queue for) is set to {@code #DEFAULT_TIMEOUT_FOR_API} + * + * @param id The model Id + * @param update Inference config update + * @param textInput Inference input + * @param previouslyLicensed License has been checked previously + * and can now be skipped + * @param inferenceTimeout The inference timeout (how long the + * request waits in the inference queue for) + * @return the new Request */ - public static Request forTextInput(String id, InferenceConfigUpdate update, List textInput) { + public static Request forTextInput( + String id, + InferenceConfigUpdate update, + List textInput, + boolean previouslyLicensed, + TimeValue inferenceTimeout + ) { return new Request( id, update, List.of(), ExceptionsHelper.requireNonNull(textInput, "inference text input"), - DEFAULT_TIMEOUT_FOR_API, - false + inferenceTimeout, + previouslyLicensed ); } @@ -197,11 +224,6 @@ public TimeValue getInferenceTimeout() { return inferenceTimeout; } - public Request setInferenceTimeout(TimeValue inferenceTimeout) { - this.inferenceTimeout = inferenceTimeout; - return this; - } - public boolean isHighPriority() { return highPriority; } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/InferModelActionRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/InferModelActionRequestTests.java index 2f073cb32d09a..69c1b23a5ff85 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/InferModelActionRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/InferModelActionRequestTests.java @@ -54,12 +54,15 @@ protected Request createTestInstance() { randomAlphaOfLength(10), Stream.generate(InferModelActionRequestTests::randomMap).limit(randomInt(10)).collect(Collectors.toList()), randomInferenceConfigUpdate(), - randomBoolean() + randomBoolean(), + TimeValue.timeValueMillis(randomLongBetween(1, 2048)) ) : Request.forTextInput( randomAlphaOfLength(10), randomInferenceConfigUpdate(), - Arrays.asList(generateRandomStringArray(3, 5, false)) + Arrays.asList(generateRandomStringArray(3, 5, false)), + randomBoolean(), + TimeValue.timeValueMillis(randomLongBetween(1, 2048)) ); request.setHighPriority(randomBoolean()); @@ -114,7 +117,6 @@ protected Request mutateInstance(Request instance) { var r = new Request(modelId, update, objectsToInfer, textInput, timeout, previouslyLicensed); r.setHighPriority(highPriority); - r.setInferenceTimeout(timeout); return r; } diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/license/MachineLearningLicensingIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/license/MachineLearningLicensingIT.java index 9933daa4693ce..01a9c166ff0e4 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/license/MachineLearningLicensingIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/license/MachineLearningLicensingIT.java @@ -669,8 +669,9 @@ public void testMachineLearningInferModelRestricted() { modelId, Collections.singletonList(Collections.emptyMap()), RegressionConfigUpdate.EMPTY_PARAMS, - false - ).setInferenceTimeout(TimeValue.timeValueSeconds(5)), + false, + TimeValue.timeValueSeconds(5) + ), inferModelSuccess ); InferModelAction.Response response = inferModelSuccess.actionGet(); @@ -690,8 +691,9 @@ public void testMachineLearningInferModelRestricted() { modelId, Collections.singletonList(Collections.emptyMap()), RegressionConfigUpdate.EMPTY_PARAMS, - false - ).setInferenceTimeout(TimeValue.timeValueSeconds(5)) + false, + TimeValue.timeValueSeconds(5) + ) ).actionGet(); }); assertThat(e.status(), is(RestStatus.FORBIDDEN)); @@ -706,8 +708,9 @@ public void testMachineLearningInferModelRestricted() { modelId, Collections.singletonList(Collections.emptyMap()), RegressionConfigUpdate.EMPTY_PARAMS, - true - ).setInferenceTimeout(TimeValue.timeValueSeconds(5)), + true, + TimeValue.timeValueSeconds(5) + ), inferModelSuccess ); response = inferModelSuccess.actionGet(); @@ -726,8 +729,9 @@ public void testMachineLearningInferModelRestricted() { modelId, Collections.singletonList(Collections.emptyMap()), RegressionConfigUpdate.EMPTY_PARAMS, - false - ).setInferenceTimeout(TimeValue.timeValueSeconds(5)), + false, + TimeValue.timeValueSeconds(5) + ), listener ); assertThat(listener.actionGet().getInferenceResults(), is(not(empty()))); diff --git a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/ModelInferenceActionIT.java b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/ModelInferenceActionIT.java index b9ca3946412bc..e03445912175a 100644 --- a/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/ModelInferenceActionIT.java +++ b/x-pack/plugin/ml/src/internalClusterTest/java/org/elasticsearch/xpack/ml/integration/ModelInferenceActionIT.java @@ -175,7 +175,8 @@ public void testInferModels() throws Exception { modelId1, toInfer, RegressionConfigUpdate.EMPTY_PARAMS, - true + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST ); InferModelAction.Response response = client().execute(InferModelAction.INSTANCE, request).actionGet(); assertThat( @@ -183,7 +184,13 @@ public void testInferModels() throws Exception { contains(1.3, 1.25) ); - request = InferModelAction.Request.forIngestDocs(modelId1, toInfer2, RegressionConfigUpdate.EMPTY_PARAMS, true); + request = InferModelAction.Request.forIngestDocs( + modelId1, + toInfer2, + RegressionConfigUpdate.EMPTY_PARAMS, + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST + ); response = client().execute(InferModelAction.INSTANCE, request).actionGet(); assertThat( response.getInferenceResults().stream().map(i -> ((SingleValueInferenceResults) i).value()).collect(Collectors.toList()), @@ -191,7 +198,13 @@ public void testInferModels() throws Exception { ); // Test classification - request = InferModelAction.Request.forIngestDocs(modelId2, toInfer, ClassificationConfigUpdate.EMPTY_PARAMS, true); + request = InferModelAction.Request.forIngestDocs( + modelId2, + toInfer, + ClassificationConfigUpdate.EMPTY_PARAMS, + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST + ); response = client().execute(InferModelAction.INSTANCE, request).actionGet(); assertThat( response.getInferenceResults() @@ -206,7 +219,8 @@ public void testInferModels() throws Exception { modelId2, toInfer, new ClassificationConfigUpdate(2, null, null, null, null), - true + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST ); response = client().execute(InferModelAction.INSTANCE, request).actionGet(); @@ -234,7 +248,8 @@ public void testInferModels() throws Exception { modelId2, toInfer2, new ClassificationConfigUpdate(1, null, null, null, null), - true + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST ); response = client().execute(InferModelAction.INSTANCE, request).actionGet(); @@ -338,7 +353,8 @@ public void testInferModelMultiClassModel() throws Exception { modelId, toInfer, ClassificationConfigUpdate.EMPTY_PARAMS, - true + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST ); InferModelAction.Response response = client().execute(InferModelAction.INSTANCE, request).actionGet(); assertThat( @@ -349,7 +365,13 @@ public void testInferModelMultiClassModel() throws Exception { contains("option_0", "option_2") ); - request = InferModelAction.Request.forIngestDocs(modelId, toInfer2, ClassificationConfigUpdate.EMPTY_PARAMS, true); + request = InferModelAction.Request.forIngestDocs( + modelId, + toInfer2, + ClassificationConfigUpdate.EMPTY_PARAMS, + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST + ); response = client().execute(InferModelAction.INSTANCE, request).actionGet(); assertThat( response.getInferenceResults() @@ -360,7 +382,13 @@ public void testInferModelMultiClassModel() throws Exception { ); // Get top classes - request = InferModelAction.Request.forIngestDocs(modelId, toInfer, new ClassificationConfigUpdate(3, null, null, null, null), true); + request = InferModelAction.Request.forIngestDocs( + modelId, + toInfer, + new ClassificationConfigUpdate(3, null, null, null, null), + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST + ); response = client().execute(InferModelAction.INSTANCE, request).actionGet(); ClassificationInferenceResults classificationInferenceResults = (ClassificationInferenceResults) response.getInferenceResults() @@ -382,7 +410,8 @@ public void testInferMissingModel() { model, Collections.emptyList(), RegressionConfigUpdate.EMPTY_PARAMS, - true + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST ); try { client().execute(InferModelAction.INSTANCE, request).actionGet(); @@ -428,7 +457,8 @@ public void testInferMissingFields() throws Exception { modelId, toInferMissingField, RegressionConfigUpdate.EMPTY_PARAMS, - true + true, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST ); try { InferenceResults result = client().execute(InferModelAction.INSTANCE, request).actionGet().getInferenceResults().get(0); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessor.java index 905317713263e..5518903dde125 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessor.java @@ -245,7 +245,13 @@ InferModelAction.Request buildRequest(IngestDocument ingestDocument) { } } } - return InferModelAction.Request.forTextInput(modelId, inferenceConfig, requestInputs); + return InferModelAction.Request.forTextInput( + modelId, + inferenceConfig, + requestInputs, + previouslyLicensed, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST + ); } else { Map fields = new HashMap<>(ingestDocument.getSourceAndMetadata()); // Add ingestMetadata as previous processors might have added metadata from which we are predicting (see: foreach processor) @@ -254,7 +260,13 @@ InferModelAction.Request buildRequest(IngestDocument ingestDocument) { } LocalModel.mapFieldsIfNecessary(fields, fieldMap); - return InferModelAction.Request.forIngestDocs(modelId, List.of(fields), inferenceConfig, previouslyLicensed); + return InferModelAction.Request.forIngestDocs( + modelId, + List.of(fields), + inferenceConfig, + previouslyLicensed, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST + ); } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilder.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilder.java index 2d74b1b34888f..40e4f5d9ede78 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilder.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilder.java @@ -126,7 +126,9 @@ protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws InferModelAction.Request inferRequest = InferModelAction.Request.forTextInput( modelId, TextExpansionConfigUpdate.EMPTY_UPDATE, - List.of(modelText) + List.of(modelText), + false, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_API ); inferRequest.setHighPriority(true); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/vectors/TextEmbeddingQueryVectorBuilder.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/vectors/TextEmbeddingQueryVectorBuilder.java index 2dd76c8fab7cc..2e780c9849bd5 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/vectors/TextEmbeddingQueryVectorBuilder.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/vectors/TextEmbeddingQueryVectorBuilder.java @@ -95,7 +95,9 @@ public void buildVector(Client client, ActionListener listener) { InferModelAction.Request inferRequest = InferModelAction.Request.forTextInput( modelId, TextEmbeddingConfigUpdate.EMPTY_INSTANCE, - List.of(modelText) + List.of(modelText), + false, + InferModelAction.Request.DEFAULT_TIMEOUT_FOR_API ); inferRequest.setHighPriority(true); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessorTests.java index a68084aa6eb28..88dcc2ba5d697 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/inference/ingest/InferenceProcessorTests.java @@ -303,14 +303,19 @@ public void testGenerateRequestWithEmptyMapping() { }; IngestDocument document = TestIngestDocument.ofIngestWithNullableVersion(source, new HashMap<>()); - assertThat(processor.buildRequest(document).getObjectsToInfer().get(0), equalTo(source)); + var request = processor.buildRequest(document); + assertThat(request.getObjectsToInfer().get(0), equalTo(source)); + assertEquals(InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST, request.getInferenceTimeout()); Map ingestMetadata = Collections.singletonMap("_value", 3); document = TestIngestDocument.ofIngestWithNullableVersion(source, ingestMetadata); Map expected = new HashMap<>(source); expected.put("_ingest", ingestMetadata); - assertThat(processor.buildRequest(document).getObjectsToInfer().get(0), equalTo(expected)); + + request = processor.buildRequest(document); + assertThat(request.getObjectsToInfer().get(0), equalTo(expected)); + assertEquals(InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST, request.getInferenceTimeout()); } public void testGenerateWithMapping() { @@ -346,14 +351,18 @@ public void testGenerateWithMapping() { expectedMap.put("categorical", "foo"); expectedMap.put("new_categorical", "foo"); expectedMap.put("un_touched", "bar"); - assertThat(processor.buildRequest(document).getObjectsToInfer().get(0), equalTo(expectedMap)); + var request = processor.buildRequest(document); + assertThat(request.getObjectsToInfer().get(0), equalTo(expectedMap)); + assertEquals(InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST, request.getInferenceTimeout()); Map ingestMetadata = Collections.singletonMap("_value", "baz"); document = TestIngestDocument.ofIngestWithNullableVersion(source, ingestMetadata); expectedMap = new HashMap<>(expectedMap); expectedMap.put("metafield", "baz"); expectedMap.put("_ingest", ingestMetadata); - assertThat(processor.buildRequest(document).getObjectsToInfer().get(0), equalTo(expectedMap)); + request = processor.buildRequest(document); + assertThat(request.getObjectsToInfer().get(0), equalTo(expectedMap)); + assertEquals(InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST, request.getInferenceTimeout()); } public void testGenerateWithMappingNestedFields() { @@ -597,6 +606,7 @@ public void testBuildRequestWithInputFields() { assertTrue(request.getObjectsToInfer().isEmpty()); var requestInputs = request.getTextInput(); assertThat(requestInputs, contains("body_text", "title_text")); + assertEquals(InferModelAction.Request.DEFAULT_TIMEOUT_FOR_INGEST, request.getInferenceTimeout()); } public void testBuildRequestWithInputFields_WrongType() { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilderTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilderTests.java index 7326dd0754041..a329a55d8afe9 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilderTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/queries/TextExpansionQueryBuilderTests.java @@ -77,6 +77,7 @@ protected boolean canSimulateMethod(Method method, Object[] args) throws NoSuchM @Override protected Object simulateMethod(Method method, Object[] args) { InferModelAction.Request request = (InferModelAction.Request) args[1]; + assertEquals(InferModelAction.Request.DEFAULT_TIMEOUT_FOR_API, request.getInferenceTimeout()); // Randomisation cannot be used here as {@code #doAssertLuceneQuery} // asserts that 2 rewritten queries are the same diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/vectors/TextEmbeddingQueryVectorBuilderTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/vectors/TextEmbeddingQueryVectorBuilderTests.java index 6fc81dca16176..2c83777487685 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/vectors/TextEmbeddingQueryVectorBuilderTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/vectors/TextEmbeddingQueryVectorBuilderTests.java @@ -38,6 +38,7 @@ protected void doAssertClientRequest(ActionRequest request, TextEmbeddingQueryVe assertThat(inferRequest.getTextInput(), hasSize(1)); assertEquals(builder.getModelText(), inferRequest.getTextInput().get(0)); assertEquals(builder.getModelId(), inferRequest.getId()); + assertEquals(InferModelAction.Request.DEFAULT_TIMEOUT_FOR_API, inferRequest.getInferenceTimeout()); } public ActionResponse createResponse(float[] array, TextEmbeddingQueryVectorBuilder builder) { From 520e1185618c674907035717929eecc2411208e3 Mon Sep 17 00:00:00 2001 From: Mark Vieira Date: Thu, 9 Nov 2023 09:46:18 -0800 Subject: [PATCH 035/513] Don't apply IntelliJ illegal module dependency inspection to test code (#101977) We don't enforce Java modularity when executing tests, so we shouldn't bother highlighting issues in test framework code. This PR adds a new IntelliJ scope that excludes everything under `:test` and configures the illegal module dependency inspection to use this new scope. --- .idea/inspectionProfiles/Project_Default.xml | 7 ++++--- .idea/scopes/Production_minus_fixtures.xml | 3 +++ 2 files changed, 7 insertions(+), 3 deletions(-) create mode 100644 .idea/scopes/Production_minus_fixtures.xml diff --git a/.idea/inspectionProfiles/Project_Default.xml b/.idea/inspectionProfiles/Project_Default.xml index ae2c8d2be1deb..3efd2cce181d4 100644 --- a/.idea/inspectionProfiles/Project_Default.xml +++ b/.idea/inspectionProfiles/Project_Default.xml @@ -2,12 +2,13 @@ diff --git a/.idea/scopes/Production_minus_fixtures.xml b/.idea/scopes/Production_minus_fixtures.xml new file mode 100644 index 0000000000000..07510326481b4 --- /dev/null +++ b/.idea/scopes/Production_minus_fixtures.xml @@ -0,0 +1,3 @@ + + + From ef063491cb4633b8e9e2f443c91223fdc7658536 Mon Sep 17 00:00:00 2001 From: Nikolaj Volgushev Date: Thu, 9 Nov 2023 18:53:15 +0100 Subject: [PATCH 036/513] No wait for search shards in reserved role mapping updates (#101945) Sets `wait_for_active_shards` to `NONE` for indexing role mappings. Previously, this was set to the default value of `1` which in stateless implies waiting to an active search shard. However, we only need to wait for the primary. For indexing, we implicitly wait for the primary shard to be available (both in stateless and stateful) so it's safe to use `NONE` here instead. --- .../security/authc/support/mapper/NativeRoleMappingStore.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java index ba28e2a9952cc..6e359aa7297ab 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStore.java @@ -13,6 +13,7 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.CheckedBiConsumer; @@ -243,6 +244,7 @@ private void innerPutMapping(PutRoleMappingRequest request, ActionListener() { @Override @@ -275,6 +277,7 @@ private void innerDeleteMapping(DeleteRoleMappingRequest request, ActionListener SECURITY_ORIGIN, client.prepareDelete(SECURITY_MAIN_ALIAS, getIdForName(request.getName())) .setRefreshPolicy(request.getRefreshPolicy()) + .setWaitForActiveShards(ActiveShardCount.NONE) .request(), new ActionListener() { From 3ca8b6bb810c18a07e27d04ce6f712488705e598 Mon Sep 17 00:00:00 2001 From: Michael Peterson Date: Thu, 9 Nov 2023 13:55:30 -0500 Subject: [PATCH 037/513] Log significant 500 status errors in TransportSearchAction when partial results are returned (#101930) --- .../action/search/TransportSearchAction.java | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) 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 5030bd875a0f6..a2d01e226b4ed 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -10,11 +10,13 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsAction; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; @@ -278,7 +280,24 @@ public long buildTookInMillis() { @Override protected void doExecute(Task task, SearchRequest searchRequest, ActionListener listener) { - executeRequest((SearchTask) task, searchRequest, listener, AsyncSearchActionProvider::new); + ActionListener loggingListener = listener.delegateFailureAndWrap((l, searchResponse) -> { + if (searchResponse.getShardFailures() != null && searchResponse.getShardFailures().length > 0) { + // Deduplicate failures by exception message and index + ShardOperationFailedException[] groupedFailures = ExceptionsHelper.groupBy(searchResponse.getShardFailures()); + for (ShardOperationFailedException f : groupedFailures) { + boolean causeHas500Status = false; + if (f.getCause() != null) { + causeHas500Status = ExceptionsHelper.status(f.getCause()).getStatus() >= 500; + } + if ((f.status().getStatus() >= 500 || causeHas500Status) + && ExceptionsHelper.isNodeOrShardUnavailableTypeException(f.getCause()) == false) { + logger.warn("TransportSearchAction shard failure (partial results response)", f); + } + } + } + l.onResponse(searchResponse); + }); + executeRequest((SearchTask) task, searchRequest, loggingListener, AsyncSearchActionProvider::new); } void executeRequest( From 017885783971f13db3086a3151c8845ec17d5049 Mon Sep 17 00:00:00 2001 From: Keith Massey Date: Thu, 9 Nov 2023 13:17:03 -0600 Subject: [PATCH 038/513] No longer creating the global template as a v2 template ever (#101886) --- .../rest/yaml/ESClientYamlSuiteTestCase.java | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/test/yaml-rest-runner/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java b/test/yaml-rest-runner/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java index da6786c8341cc..630ee9883ff83 100644 --- a/test/yaml-rest-runner/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java +++ b/test/yaml-rest-runner/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java @@ -484,27 +484,19 @@ public void test() throws IOException { ); final Settings globalTemplateSettings = getGlobalTemplateSettings(testCandidate.getTestSection().getSkipSection().getFeatures()); - if (globalTemplateSettings.isEmpty() == false) { - boolean useComponentTemplate = ESRestTestCase.has(ProductFeature.LEGACY_TEMPLATES) == false; + if (globalTemplateSettings.isEmpty() == false && ESRestTestCase.has(ProductFeature.LEGACY_TEMPLATES)) { final XContentBuilder template = jsonBuilder(); template.startObject(); { template.array("index_patterns", "*"); - if (useComponentTemplate) { - template.field("priority", 4); // relatively low priority, but hopefully uncommon enough not to conflict - template.startObject("template"); - } template.startObject("settings"); globalTemplateSettings.toXContent(template, ToXContent.EMPTY_PARAMS); template.endObject(); - if (useComponentTemplate) { - template.endObject(); - } } template.endObject(); - final Request request = new Request("PUT", useComponentTemplate ? "/_index_template/global" : "/_template/global"); + final Request request = new Request("PUT", "/_template/global"); request.setJsonEntity(Strings.toString(template)); // Because not all case have transitioned to a composable template, it's possible that // this can overlap an installed composable template since this is a global (*) @@ -512,9 +504,7 @@ public void test() throws IOException { // to be permissive in this case. This can be removed once all tests use composable // templates instead of legacy templates RequestOptions.Builder builder = RequestOptions.DEFAULT.toBuilder(); - if (useComponentTemplate == false) { - builder.setWarningsHandler(WarningsHandler.PERMISSIVE); - } + builder.setWarningsHandler(WarningsHandler.PERMISSIVE); request.setOptions(builder.build()); adminClient().performRequest(request); } From 4a9f2d86514c71dfa69f85a92e97648f71aa42a2 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Thu, 9 Nov 2023 15:08:16 -0500 Subject: [PATCH 039/513] Removing skip after backport #101965 (#101986) --- .../test/search.vectors/60_dense_vector_dynamic_mapping.yml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/60_dense_vector_dynamic_mapping.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/60_dense_vector_dynamic_mapping.yml index 8453bfbe297e4..62d752b1efe88 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/60_dense_vector_dynamic_mapping.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search.vectors/60_dense_vector_dynamic_mapping.yml @@ -4,10 +4,6 @@ setup: reason: 'Dynamic mapping of floats to dense_vector was added in 8.11' --- "Fields indexed as strings won't be transformed into dense_vector": - - skip: - # TODO adjust after backport - version: ' - 8.11.99' - reason: 'Dynamic mapping of things other than numbers is fixed in 8.12' - do: index: index: strings-are-not-floats From a5652b5d0d2331831ff608e4f2039c298a6a5845 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Thu, 9 Nov 2023 22:06:36 +0100 Subject: [PATCH 040/513] Remove explicit SearchResponse references from server metrics aggs (part1) (#101765) Remove explicit references from org.elasticsearch.search.aggregations.metrics --- .../aggregations/metrics/ExtendedStatsIT.java | 1320 +++++++++-------- .../aggregations/metrics/GeoBoundsIT.java | 68 +- .../aggregations/metrics/GeoCentroidIT.java | 36 +- .../metrics/HDRPercentileRanksIT.java | 644 ++++---- .../metrics/HDRPercentilesIT.java | 655 ++++---- .../metrics/MedianAbsoluteDeviationIT.java | 545 +++---- .../metrics/ScriptedMetricIT.java | 941 ++++++------ 7 files changed, 2206 insertions(+), 2003 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsIT.java index 64a97bf0f6f16..295486fba2e56 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/ExtendedStatsIT.java @@ -7,7 +7,6 @@ */ package org.elasticsearch.search.aggregations.metrics; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.Script; @@ -39,6 +38,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -91,310 +91,325 @@ private static double varianceSampling(int... vals) { @Override public void testEmptyAggregation() throws Exception { - SearchResponse searchResponse = prepareSearch("empty_bucket_idx").setQuery(matchAllQuery()) - .addAggregation( - histogram("histo").field("value").interval(1L).minDocCount(0).subAggregation(extendedStats("stats").field("value")) - ) - .get(); - - assertThat(searchResponse.getHits().getTotalHits().value, equalTo(2L)); - Histogram histo = searchResponse.getAggregations().get("histo"); - assertThat(histo, notNullValue()); - Histogram.Bucket bucket = histo.getBuckets().get(1); - assertThat(bucket, notNullValue()); - - ExtendedStats stats = bucket.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getSumOfSquares(), equalTo(0.0)); - assertThat(stats.getCount(), equalTo(0L)); - assertThat(stats.getSum(), equalTo(0.0)); - assertThat(stats.getMin(), equalTo(Double.POSITIVE_INFINITY)); - assertThat(stats.getMax(), equalTo(Double.NEGATIVE_INFINITY)); - assertThat(Double.isNaN(stats.getStdDeviation()), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationPopulation()), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationSampling()), is(true)); - assertThat(Double.isNaN(stats.getAvg()), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_POPULATION)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_POPULATION)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_SAMPLING)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_SAMPLING)), is(true)); + assertResponse( + prepareSearch("empty_bucket_idx").setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field("value").interval(1L).minDocCount(0).subAggregation(extendedStats("stats").field("value")) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(2L)); + Histogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + Histogram.Bucket bucket = histo.getBuckets().get(1); + assertThat(bucket, notNullValue()); + + ExtendedStats stats = bucket.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getSumOfSquares(), equalTo(0.0)); + assertThat(stats.getCount(), equalTo(0L)); + assertThat(stats.getSum(), equalTo(0.0)); + assertThat(stats.getMin(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(stats.getMax(), equalTo(Double.NEGATIVE_INFINITY)); + assertThat(Double.isNaN(stats.getStdDeviation()), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationPopulation()), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationSampling()), is(true)); + assertThat(Double.isNaN(stats.getAvg()), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_POPULATION)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_POPULATION)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_SAMPLING)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_SAMPLING)), is(true)); + } + ); } @Override public void testUnmapped() throws Exception { - SearchResponse searchResponse = prepareSearch("idx_unmapped").setQuery(matchAllQuery()) - .addAggregation(extendedStats("stats").field("value")) - .get(); - - assertThat(searchResponse.getHits().getTotalHits().value, equalTo(0L)); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo(Double.NaN)); - assertThat(stats.getMin(), equalTo(Double.POSITIVE_INFINITY)); - assertThat(stats.getMax(), equalTo(Double.NEGATIVE_INFINITY)); - assertThat(stats.getSum(), equalTo(0.0)); - assertThat(stats.getCount(), equalTo(0L)); - assertThat(stats.getSumOfSquares(), equalTo(0.0)); - assertThat(stats.getVariance(), equalTo(Double.NaN)); - assertThat(stats.getVariancePopulation(), equalTo(Double.NaN)); - assertThat(stats.getVarianceSampling(), equalTo(Double.NaN)); - assertThat(stats.getStdDeviation(), equalTo(Double.NaN)); - assertThat(stats.getStdDeviationPopulation(), equalTo(Double.NaN)); - assertThat(stats.getStdDeviationSampling(), equalTo(Double.NaN)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_POPULATION)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_POPULATION)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_SAMPLING)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_SAMPLING)), is(true)); + assertResponse( + prepareSearch("idx_unmapped").setQuery(matchAllQuery()).addAggregation(extendedStats("stats").field("value")), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(0L)); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo(Double.NaN)); + assertThat(stats.getMin(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(stats.getMax(), equalTo(Double.NEGATIVE_INFINITY)); + assertThat(stats.getSum(), equalTo(0.0)); + assertThat(stats.getCount(), equalTo(0L)); + assertThat(stats.getSumOfSquares(), equalTo(0.0)); + assertThat(stats.getVariance(), equalTo(Double.NaN)); + assertThat(stats.getVariancePopulation(), equalTo(Double.NaN)); + assertThat(stats.getVarianceSampling(), equalTo(Double.NaN)); + assertThat(stats.getStdDeviation(), equalTo(Double.NaN)); + assertThat(stats.getStdDeviationPopulation(), equalTo(Double.NaN)); + assertThat(stats.getStdDeviationSampling(), equalTo(Double.NaN)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_POPULATION)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_POPULATION)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_SAMPLING)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_SAMPLING)), is(true)); + } + ); } public void testPartiallyUnmapped() { double sigma = randomDouble() * 5; - ExtendedStats s1 = prepareSearch("idx").addAggregation(extendedStats("stats").field("value").sigma(sigma)) - .get() - .getAggregations() - .get("stats"); - ExtendedStats s2 = prepareSearch("idx", "idx_unmapped").addAggregation(extendedStats("stats").field("value").sigma(sigma)) - .get() - .getAggregations() - .get("stats"); - assertEquals(s1.getAvg(), s2.getAvg(), 1e-10); - assertEquals(s1.getCount(), s2.getCount()); - assertEquals(s1.getMin(), s2.getMin(), 0d); - assertEquals(s1.getMax(), s2.getMax(), 0d); - assertEquals(s1.getStdDeviation(), s2.getStdDeviation(), 1e-10); - assertEquals(s1.getStdDeviationPopulation(), s2.getStdDeviationPopulation(), 1e-10); - assertEquals(s1.getStdDeviationSampling(), s2.getStdDeviationSampling(), 1e-10); - assertEquals(s1.getSumOfSquares(), s2.getSumOfSquares(), 1e-10); - assertEquals(s1.getStdDeviationBound(Bounds.LOWER), s2.getStdDeviationBound(Bounds.LOWER), 1e-10); - assertEquals(s1.getStdDeviationBound(Bounds.UPPER), s2.getStdDeviationBound(Bounds.UPPER), 1e-10); - assertEquals(s1.getStdDeviationBound(Bounds.LOWER_POPULATION), s2.getStdDeviationBound(Bounds.LOWER_POPULATION), 1e-10); - assertEquals(s1.getStdDeviationBound(Bounds.UPPER_POPULATION), s2.getStdDeviationBound(Bounds.UPPER_POPULATION), 1e-10); - assertEquals(s1.getStdDeviationBound(Bounds.LOWER_SAMPLING), s2.getStdDeviationBound(Bounds.LOWER_SAMPLING), 1e-10); - assertEquals(s1.getStdDeviationBound(Bounds.UPPER_SAMPLING), s2.getStdDeviationBound(Bounds.UPPER_SAMPLING), 1e-10); + assertResponse(prepareSearch("idx").addAggregation(extendedStats("stats").field("value").sigma(sigma)), response1 -> { + ExtendedStats s1 = response1.getAggregations().get("stats"); + assertResponse( + prepareSearch("idx", "idx_unmapped").addAggregation(extendedStats("stats").field("value").sigma(sigma)), + response2 -> { + ExtendedStats s2 = response2.getAggregations().get("stats"); + assertEquals(s1.getAvg(), s2.getAvg(), 1e-10); + assertEquals(s1.getCount(), s2.getCount()); + assertEquals(s1.getMin(), s2.getMin(), 0d); + assertEquals(s1.getMax(), s2.getMax(), 0d); + assertEquals(s1.getStdDeviation(), s2.getStdDeviation(), 1e-10); + assertEquals(s1.getStdDeviationPopulation(), s2.getStdDeviationPopulation(), 1e-10); + assertEquals(s1.getStdDeviationSampling(), s2.getStdDeviationSampling(), 1e-10); + assertEquals(s1.getSumOfSquares(), s2.getSumOfSquares(), 1e-10); + assertEquals(s1.getStdDeviationBound(Bounds.LOWER), s2.getStdDeviationBound(Bounds.LOWER), 1e-10); + assertEquals(s1.getStdDeviationBound(Bounds.UPPER), s2.getStdDeviationBound(Bounds.UPPER), 1e-10); + assertEquals(s1.getStdDeviationBound(Bounds.LOWER_POPULATION), s2.getStdDeviationBound(Bounds.LOWER_POPULATION), 1e-10); + assertEquals(s1.getStdDeviationBound(Bounds.UPPER_POPULATION), s2.getStdDeviationBound(Bounds.UPPER_POPULATION), 1e-10); + assertEquals(s1.getStdDeviationBound(Bounds.LOWER_SAMPLING), s2.getStdDeviationBound(Bounds.LOWER_SAMPLING), 1e-10); + assertEquals(s1.getStdDeviationBound(Bounds.UPPER_SAMPLING), s2.getStdDeviationBound(Bounds.UPPER_SAMPLING), 1e-10); + } + ); + }); } @Override public void testSingleValuedField() throws Exception { double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(extendedStats("stats").field("value").sigma(sigma)) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); - assertThat(stats.getMin(), equalTo(1.0)); - assertThat(stats.getMax(), equalTo(10.0)); - assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); - assertThat(stats.getCount(), equalTo(10L)); - assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); - assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - checkUpperLowerBounds(stats, sigma); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(extendedStats("stats").field("value").sigma(sigma)), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); + assertThat(stats.getMin(), equalTo(1.0)); + assertThat(stats.getMax(), equalTo(10.0)); + assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); + assertThat(stats.getCount(), equalTo(10L)); + assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); + assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + checkUpperLowerBounds(stats, sigma); + } + ); } public void testSingleValuedFieldDefaultSigma() throws Exception { // Same as previous test, but uses a default value for sigma - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(extendedStats("stats").field("value")) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); - assertThat(stats.getMin(), equalTo(1.0)); - assertThat(stats.getMax(), equalTo(10.0)); - assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); - assertThat(stats.getCount(), equalTo(10L)); - assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); - assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - checkUpperLowerBounds(stats, 2); + assertResponse(prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(extendedStats("stats").field("value")), response -> { + + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); + assertThat(stats.getMin(), equalTo(1.0)); + assertThat(stats.getMax(), equalTo(10.0)); + assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); + assertThat(stats.getCount(), equalTo(10L)); + assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); + assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + checkUpperLowerBounds(stats, 2); + }); } public void testSingleValuedField_WithFormatter() throws Exception { double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(extendedStats("stats").format("0000.0").field("value").sigma(sigma)) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); - assertThat(stats.getAvgAsString(), equalTo("0005.5")); - assertThat(stats.getMin(), equalTo(1.0)); - assertThat(stats.getMinAsString(), equalTo("0001.0")); - assertThat(stats.getMax(), equalTo(10.0)); - assertThat(stats.getMaxAsString(), equalTo("0010.0")); - assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); - assertThat(stats.getSumAsString(), equalTo("0055.0")); - assertThat(stats.getCount(), equalTo(10L)); - assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); - assertThat(stats.getSumOfSquaresAsString(), equalTo("0385.0")); - assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVarianceAsString(), equalTo("0008.2")); - assertThat(stats.getVariancePopulationAsString(), equalTo("0008.2")); - assertThat(stats.getVarianceSamplingAsString(), equalTo("0009.2")); - assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationAsString(), equalTo("0002.9")); - assertThat(stats.getStdDeviationPopulationAsString(), equalTo("0002.9")); - assertThat(stats.getStdDeviationSamplingAsString(), equalTo("0003.0")); - checkUpperLowerBounds(stats, sigma); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation(extendedStats("stats").format("0000.0").field("value").sigma(sigma)), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); + assertThat(stats.getAvgAsString(), equalTo("0005.5")); + assertThat(stats.getMin(), equalTo(1.0)); + assertThat(stats.getMinAsString(), equalTo("0001.0")); + assertThat(stats.getMax(), equalTo(10.0)); + assertThat(stats.getMaxAsString(), equalTo("0010.0")); + assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); + assertThat(stats.getSumAsString(), equalTo("0055.0")); + assertThat(stats.getCount(), equalTo(10L)); + assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); + assertThat(stats.getSumOfSquaresAsString(), equalTo("0385.0")); + assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVarianceAsString(), equalTo("0008.2")); + assertThat(stats.getVariancePopulationAsString(), equalTo("0008.2")); + assertThat(stats.getVarianceSamplingAsString(), equalTo("0009.2")); + assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationAsString(), equalTo("0002.9")); + assertThat(stats.getStdDeviationPopulationAsString(), equalTo("0002.9")); + assertThat(stats.getStdDeviationSamplingAsString(), equalTo("0003.0")); + checkUpperLowerBounds(stats, sigma); + } + ); } @Override public void testSingleValuedFieldGetProperty() throws Exception { - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(global("global").subAggregation(extendedStats("stats").field("value"))) - .get(); - - assertHitCount(searchResponse, 10); - - Global global = searchResponse.getAggregations().get("global"); - assertThat(global, notNullValue()); - assertThat(global.getName(), equalTo("global")); - assertThat(global.getDocCount(), equalTo(10L)); - assertThat(global.getAggregations(), notNullValue()); - assertThat(global.getAggregations().asMap().size(), equalTo(1)); - - ExtendedStats stats = global.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - ExtendedStats statsFromProperty = (ExtendedStats) ((InternalAggregation) global).getProperty("stats"); - assertThat(statsFromProperty, notNullValue()); - assertThat(statsFromProperty, sameInstance(stats)); - double expectedAvgValue = (double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10; - assertThat(stats.getAvg(), equalTo(expectedAvgValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.avg"), equalTo(expectedAvgValue)); - double expectedMinValue = 1.0; - assertThat(stats.getMin(), equalTo(expectedMinValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.min"), equalTo(expectedMinValue)); - double expectedMaxValue = 10.0; - assertThat(stats.getMax(), equalTo(expectedMaxValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.max"), equalTo(expectedMaxValue)); - double expectedSumValue = 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10; - assertThat(stats.getSum(), equalTo(expectedSumValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.sum"), equalTo(expectedSumValue)); - long expectedCountValue = 10; - assertThat(stats.getCount(), equalTo(expectedCountValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.count"), equalTo((double) expectedCountValue)); - double expectedSumOfSquaresValue = (double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100; - assertThat(stats.getSumOfSquares(), equalTo(expectedSumOfSquaresValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.sum_of_squares"), equalTo(expectedSumOfSquaresValue)); - double expectedVarianceValue = variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - assertThat(stats.getVariance(), equalTo(expectedVarianceValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.variance"), equalTo(expectedVarianceValue)); - double expectedVariancePopulationValue = variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - assertThat(stats.getVariancePopulation(), equalTo(expectedVariancePopulationValue)); - assertThat( - (double) ((InternalAggregation) global).getProperty("stats.variance_population"), - equalTo(expectedVariancePopulationValue) - ); - double expectedVarianceSamplingValue = varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - assertThat(stats.getVarianceSampling(), equalTo(expectedVarianceSamplingValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.variance_sampling"), equalTo(expectedVarianceSamplingValue)); - double expectedStdDevValue = stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - assertThat(stats.getStdDeviation(), equalTo(expectedStdDevValue)); - assertThat((double) ((InternalAggregation) global).getProperty("stats.std_deviation"), equalTo(expectedStdDevValue)); - double expectedStdDevPopulationValue = stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - assertThat(stats.getStdDeviationPopulation(), equalTo(expectedStdDevValue)); - assertThat( - (double) ((InternalAggregation) global).getProperty("stats.std_deviation_population"), - equalTo(expectedStdDevPopulationValue) - ); - double expectedStdDevSamplingValue = stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); - assertThat(stats.getStdDeviationSampling(), equalTo(expectedStdDevSamplingValue)); - assertThat( - (double) ((InternalAggregation) global).getProperty("stats.std_deviation_sampling"), - equalTo(expectedStdDevSamplingValue) + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation(global("global").subAggregation(extendedStats("stats").field("value"))), + response -> { + assertHitCount(response, 10); + + Global global = response.getAggregations().get("global"); + assertThat(global, notNullValue()); + assertThat(global.getName(), equalTo("global")); + assertThat(global.getDocCount(), equalTo(10L)); + assertThat(global.getAggregations(), notNullValue()); + assertThat(global.getAggregations().asMap().size(), equalTo(1)); + + ExtendedStats stats = global.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + ExtendedStats statsFromProperty = (ExtendedStats) ((InternalAggregation) global).getProperty("stats"); + assertThat(statsFromProperty, notNullValue()); + assertThat(statsFromProperty, sameInstance(stats)); + double expectedAvgValue = (double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10; + assertThat(stats.getAvg(), equalTo(expectedAvgValue)); + assertThat((double) ((InternalAggregation) global).getProperty("stats.avg"), equalTo(expectedAvgValue)); + double expectedMinValue = 1.0; + assertThat(stats.getMin(), equalTo(expectedMinValue)); + assertThat((double) ((InternalAggregation) global).getProperty("stats.min"), equalTo(expectedMinValue)); + double expectedMaxValue = 10.0; + assertThat(stats.getMax(), equalTo(expectedMaxValue)); + assertThat((double) ((InternalAggregation) global).getProperty("stats.max"), equalTo(expectedMaxValue)); + double expectedSumValue = 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10; + assertThat(stats.getSum(), equalTo(expectedSumValue)); + assertThat((double) ((InternalAggregation) global).getProperty("stats.sum"), equalTo(expectedSumValue)); + long expectedCountValue = 10; + assertThat(stats.getCount(), equalTo(expectedCountValue)); + assertThat((double) ((InternalAggregation) global).getProperty("stats.count"), equalTo((double) expectedCountValue)); + double expectedSumOfSquaresValue = (double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100; + assertThat(stats.getSumOfSquares(), equalTo(expectedSumOfSquaresValue)); + assertThat((double) ((InternalAggregation) global).getProperty("stats.sum_of_squares"), equalTo(expectedSumOfSquaresValue)); + double expectedVarianceValue = variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + assertThat(stats.getVariance(), equalTo(expectedVarianceValue)); + assertThat((double) ((InternalAggregation) global).getProperty("stats.variance"), equalTo(expectedVarianceValue)); + double expectedVariancePopulationValue = variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + assertThat(stats.getVariancePopulation(), equalTo(expectedVariancePopulationValue)); + assertThat( + (double) ((InternalAggregation) global).getProperty("stats.variance_population"), + equalTo(expectedVariancePopulationValue) + ); + double expectedVarianceSamplingValue = varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + assertThat(stats.getVarianceSampling(), equalTo(expectedVarianceSamplingValue)); + assertThat( + (double) ((InternalAggregation) global).getProperty("stats.variance_sampling"), + equalTo(expectedVarianceSamplingValue) + ); + double expectedStdDevValue = stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + assertThat(stats.getStdDeviation(), equalTo(expectedStdDevValue)); + assertThat((double) ((InternalAggregation) global).getProperty("stats.std_deviation"), equalTo(expectedStdDevValue)); + double expectedStdDevPopulationValue = stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + assertThat(stats.getStdDeviationPopulation(), equalTo(expectedStdDevValue)); + assertThat( + (double) ((InternalAggregation) global).getProperty("stats.std_deviation_population"), + equalTo(expectedStdDevPopulationValue) + ); + double expectedStdDevSamplingValue = stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + assertThat(stats.getStdDeviationSampling(), equalTo(expectedStdDevSamplingValue)); + assertThat( + (double) ((InternalAggregation) global).getProperty("stats.std_deviation_sampling"), + equalTo(expectedStdDevSamplingValue) + ); + } ); } @Override public void testSingleValuedFieldPartiallyUnmapped() throws Exception { double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx", "idx_unmapped").setQuery(matchAllQuery()) - .addAggregation(extendedStats("stats").field("value").sigma(sigma)) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); - assertThat(stats.getMin(), equalTo(1.0)); - assertThat(stats.getMax(), equalTo(10.0)); - assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); - assertThat(stats.getCount(), equalTo(10L)); - assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); - assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - checkUpperLowerBounds(stats, sigma); + assertResponse( + prepareSearch("idx", "idx_unmapped").setQuery(matchAllQuery()) + .addAggregation(extendedStats("stats").field("value").sigma(sigma)), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); + assertThat(stats.getMin(), equalTo(1.0)); + assertThat(stats.getMax(), equalTo(10.0)); + assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); + assertThat(stats.getCount(), equalTo(10L)); + assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); + assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + checkUpperLowerBounds(stats, sigma); + } + ); } @Override public void testSingleValuedFieldWithValueScript() throws Exception { double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - extendedStats("stats").field("value") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())) - .sigma(sigma) - ) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 10)); - assertThat(stats.getMin(), equalTo(2.0)); - assertThat(stats.getMax(), equalTo(11.0)); - assertThat(stats.getSum(), equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11)); - assertThat(stats.getCount(), equalTo(10L)); - assertThat(stats.getSumOfSquares(), equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121)); - assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - checkUpperLowerBounds(stats, sigma); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + extendedStats("stats").field("value") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())) + .sigma(sigma) + ), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 10)); + assertThat(stats.getMin(), equalTo(2.0)); + assertThat(stats.getMax(), equalTo(11.0)); + assertThat(stats.getSum(), equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11)); + assertThat(stats.getCount(), equalTo(10L)); + assertThat(stats.getSumOfSquares(), equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121)); + assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + checkUpperLowerBounds(stats, sigma); + } + ); } @Override @@ -402,118 +417,139 @@ public void testSingleValuedFieldWithValueScriptWithParams() throws Exception { Map params = new HashMap<>(); params.put("inc", 1); double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - extendedStats("stats").field("value") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + inc", params)) - .sigma(sigma) - ) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 10)); - assertThat(stats.getMin(), equalTo(2.0)); - assertThat(stats.getMax(), equalTo(11.0)); - assertThat(stats.getSum(), equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11)); - assertThat(stats.getCount(), equalTo(10L)); - assertThat(stats.getSumOfSquares(), equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121)); - assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - checkUpperLowerBounds(stats, sigma); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + extendedStats("stats").field("value") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + inc", params)) + .sigma(sigma) + ), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 10)); + assertThat(stats.getMin(), equalTo(2.0)); + assertThat(stats.getMax(), equalTo(11.0)); + assertThat(stats.getSum(), equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11)); + assertThat(stats.getCount(), equalTo(10L)); + assertThat(stats.getSumOfSquares(), equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121)); + assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + checkUpperLowerBounds(stats, sigma); + } + ); } @Override public void testMultiValuedField() throws Exception { double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(extendedStats("stats").field("values").sigma(sigma)) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat( - stats.getAvg(), - equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12) / 20) + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(extendedStats("stats").field("values").sigma(sigma)), + response -> { + + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat( + stats.getAvg(), + equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12) / 20) + ); + assertThat(stats.getMin(), equalTo(2.0)); + assertThat(stats.getMax(), equalTo(12.0)); + assertThat( + stats.getSum(), + equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12) + ); + assertThat(stats.getCount(), equalTo(20L)); + assertThat( + stats.getSumOfSquares(), + equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121 + 144) + ); + assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); + assertThat( + stats.getVariancePopulation(), + equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + ); + assertThat( + stats.getVarianceSampling(), + equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + ); + assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); + assertThat( + stats.getStdDeviationPopulation(), + equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + ); + assertThat( + stats.getStdDeviationSampling(), + equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + ); + checkUpperLowerBounds(stats, sigma); + } ); - assertThat(stats.getMin(), equalTo(2.0)); - assertThat(stats.getMax(), equalTo(12.0)); - assertThat(stats.getSum(), equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12)); - assertThat(stats.getCount(), equalTo(20L)); - assertThat( - stats.getSumOfSquares(), - equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121 + 144) - ); - assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); - assertThat( - stats.getVariancePopulation(), - equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) - ); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); - assertThat( - stats.getStdDeviationPopulation(), - equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) - ); - assertThat( - stats.getStdDeviationSampling(), - equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) - ); - checkUpperLowerBounds(stats, sigma); } @Override public void testMultiValuedFieldWithValueScript() throws Exception { double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - extendedStats("stats").field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", Collections.emptyMap())) - .sigma(sigma) - ) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat( - stats.getAvg(), - equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 20) + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + extendedStats("stats").field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", Collections.emptyMap())) + .sigma(sigma) + ), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat( + stats.getAvg(), + equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 20) + ); + assertThat(stats.getMin(), equalTo(1.0)); + assertThat(stats.getMax(), equalTo(11.0)); + assertThat( + stats.getSum(), + equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) + ); + assertThat(stats.getCount(), equalTo(20L)); + assertThat( + stats.getSumOfSquares(), + equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121) + ); + assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat( + stats.getVariancePopulation(), + equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) + ); + assertThat( + stats.getVarianceSampling(), + equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) + ); + assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat( + stats.getStdDeviationPopulation(), + equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) + ); + assertThat( + stats.getStdDeviationSampling(), + equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) + ); + + checkUpperLowerBounds(stats, sigma); + } ); - assertThat(stats.getMin(), equalTo(1.0)); - assertThat(stats.getMax(), equalTo(11.0)); - assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11)); - assertThat(stats.getCount(), equalTo(20L)); - assertThat( - stats.getSumOfSquares(), - equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121) - ); - assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat( - stats.getVariancePopulation(), - equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) - ); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat( - stats.getStdDeviationPopulation(), - equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) - ); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - - checkUpperLowerBounds(stats, sigma); } @Override @@ -521,75 +557,88 @@ public void testMultiValuedFieldWithValueScriptWithParams() throws Exception { Map params = new HashMap<>(); params.put("dec", 1); double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - extendedStats("stats").field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) - .sigma(sigma) - ) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat( - stats.getAvg(), - equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 20) + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + extendedStats("stats").field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) + .sigma(sigma) + ), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat( + stats.getAvg(), + equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 20) + ); + assertThat(stats.getMin(), equalTo(1.0)); + assertThat(stats.getMax(), equalTo(11.0)); + assertThat( + stats.getSum(), + equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) + ); + assertThat(stats.getCount(), equalTo(20L)); + assertThat( + stats.getSumOfSquares(), + equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121) + ); + assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat( + stats.getVariancePopulation(), + equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) + ); + assertThat( + stats.getVarianceSampling(), + equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) + ); + assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat( + stats.getStdDeviationPopulation(), + equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) + ); + assertThat( + stats.getStdDeviationSampling(), + equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) + ); + checkUpperLowerBounds(stats, sigma); + } ); - assertThat(stats.getMin(), equalTo(1.0)); - assertThat(stats.getMax(), equalTo(11.0)); - assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11)); - assertThat(stats.getCount(), equalTo(20L)); - assertThat( - stats.getSumOfSquares(), - equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121) - ); - assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat( - stats.getVariancePopulation(), - equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) - ); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat( - stats.getStdDeviationPopulation(), - equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11)) - ); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - checkUpperLowerBounds(stats, sigma); } @Override public void testScriptSingleValued() throws Exception { double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - extendedStats("stats").script( - new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", Collections.emptyMap()) - ).sigma(sigma) - ) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); - assertThat(stats.getMin(), equalTo(1.0)); - assertThat(stats.getMax(), equalTo(10.0)); - assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); - assertThat(stats.getCount(), equalTo(10L)); - assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); - assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); - checkUpperLowerBounds(stats, sigma); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + extendedStats("stats").script( + new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", Collections.emptyMap()) + ).sigma(sigma) + ), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10) / 10)); + assertThat(stats.getMin(), equalTo(1.0)); + assertThat(stats.getMax(), equalTo(10.0)); + assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10)); + assertThat(stats.getCount(), equalTo(10L)); + assertThat(stats.getSumOfSquares(), equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100)); + assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10))); + checkUpperLowerBounds(stats, sigma); + } + ); } @Override @@ -600,74 +649,83 @@ public void testScriptSingleValuedWithParams() throws Exception { Script script = new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value + inc", params); double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(extendedStats("stats").script(script).sigma(sigma)) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 10)); - assertThat(stats.getMin(), equalTo(2.0)); - assertThat(stats.getMax(), equalTo(11.0)); - assertThat(stats.getSum(), equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11)); - assertThat(stats.getCount(), equalTo(10L)); - assertThat(stats.getSumOfSquares(), equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121)); - assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); - checkUpperLowerBounds(stats, sigma); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(extendedStats("stats").script(script).sigma(sigma)), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getAvg(), equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11) / 10)); + assertThat(stats.getMin(), equalTo(2.0)); + assertThat(stats.getMax(), equalTo(11.0)); + assertThat(stats.getSum(), equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11)); + assertThat(stats.getCount(), equalTo(10L)); + assertThat(stats.getSumOfSquares(), equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121)); + assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviationPopulation(), equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11))); + checkUpperLowerBounds(stats, sigma); + } + ); } @Override public void testScriptMultiValued() throws Exception { double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - extendedStats("stats").script( - new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['values']", Collections.emptyMap()) - ).sigma(sigma) - ) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat( - stats.getAvg(), - equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12) / 20) - ); - assertThat(stats.getMin(), equalTo(2.0)); - assertThat(stats.getMax(), equalTo(12.0)); - assertThat(stats.getSum(), equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12)); - assertThat(stats.getCount(), equalTo(20L)); - assertThat( - stats.getSumOfSquares(), - equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121 + 144) - ); - assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); - assertThat( - stats.getVariancePopulation(), - equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) - ); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); - assertThat( - stats.getStdDeviationPopulation(), - equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) - ); - assertThat( - stats.getStdDeviationSampling(), - equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + extendedStats("stats").script( + new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['values']", Collections.emptyMap()) + ).sigma(sigma) + ), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat( + stats.getAvg(), + equalTo((double) (2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12) / 20) + ); + assertThat(stats.getMin(), equalTo(2.0)); + assertThat(stats.getMax(), equalTo(12.0)); + assertThat( + stats.getSum(), + equalTo((double) 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 11 + 12) + ); + assertThat(stats.getCount(), equalTo(20L)); + assertThat( + stats.getSumOfSquares(), + equalTo((double) 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 121 + 144) + ); + assertThat(stats.getVariance(), equalTo(variance(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); + assertThat( + stats.getVariancePopulation(), + equalTo(variancePopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + ); + assertThat( + stats.getVarianceSampling(), + equalTo(varianceSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + ); + assertThat(stats.getStdDeviation(), equalTo(stdDev(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12))); + assertThat( + stats.getStdDeviationPopulation(), + equalTo(stdDevPopulation(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + ); + assertThat( + stats.getStdDeviationSampling(), + equalTo(stdDevSampling(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12)) + ); + checkUpperLowerBounds(stats, sigma); + } ); - checkUpperLowerBounds(stats, sigma); } @Override @@ -683,125 +741,147 @@ public void testScriptMultiValuedWithParams() throws Exception { ); double sigma = randomDouble() * randomIntBetween(1, 10); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(extendedStats("stats").script(script).sigma(sigma)) - .get(); - - assertHitCount(searchResponse, 10); - - ExtendedStats stats = searchResponse.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getAvg(), equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9) / 20)); - assertThat(stats.getMin(), equalTo(0.0)); - assertThat(stats.getMax(), equalTo(10.0)); - assertThat(stats.getSum(), equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9)); - assertThat(stats.getCount(), equalTo(20L)); - assertThat( - stats.getSumOfSquares(), - equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 0 + 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81) - ); - assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); - assertThat(stats.getVariancePopulation(), equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); - assertThat(stats.getVarianceSampling(), equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); - assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); - assertThat( - stats.getStdDeviationPopulation(), - equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(extendedStats("stats").script(script).sigma(sigma)), + response -> { + assertHitCount(response, 10); + + ExtendedStats stats = response.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat( + stats.getAvg(), + equalTo((double) (1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9) / 20) + ); + assertThat(stats.getMin(), equalTo(0.0)); + assertThat(stats.getMax(), equalTo(10.0)); + assertThat( + stats.getSum(), + equalTo((double) 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10 + 0 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9) + ); + assertThat(stats.getCount(), equalTo(20L)); + assertThat( + stats.getSumOfSquares(), + equalTo((double) 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81 + 100 + 0 + 1 + 4 + 9 + 16 + 25 + 36 + 49 + 64 + 81) + ); + assertThat(stats.getVariance(), equalTo(variance(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); + assertThat( + stats.getVariancePopulation(), + equalTo(variancePopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) + ); + assertThat( + stats.getVarianceSampling(), + equalTo(varianceSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) + ); + assertThat(stats.getStdDeviation(), equalTo(stdDev(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); + assertThat( + stats.getStdDeviationPopulation(), + equalTo(stdDevPopulation(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) + ); + assertThat( + stats.getStdDeviationSampling(), + equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9)) + ); + checkUpperLowerBounds(stats, sigma); + } ); - assertThat(stats.getStdDeviationSampling(), equalTo(stdDevSampling(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9))); - checkUpperLowerBounds(stats, sigma); } public void testEmptySubAggregation() { - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - terms("value").field("value") - .subAggregation(missing("values").field("values").subAggregation(extendedStats("stats").field("value"))) - ) - .get(); - - assertHitCount(searchResponse, 10); - - Terms terms = searchResponse.getAggregations().get("value"); - assertThat(terms, notNullValue()); - assertThat(terms.getBuckets().size(), equalTo(10)); - - for (Terms.Bucket bucket : terms.getBuckets()) { - assertThat(bucket.getDocCount(), equalTo(1L)); - - Missing missing = bucket.getAggregations().get("values"); - assertThat(missing, notNullValue()); - assertThat(missing.getDocCount(), equalTo(0L)); - - ExtendedStats stats = missing.getAggregations().get("stats"); - assertThat(stats, notNullValue()); - assertThat(stats.getName(), equalTo("stats")); - assertThat(stats.getSumOfSquares(), equalTo(0.0)); - assertThat(stats.getCount(), equalTo(0L)); - assertThat(stats.getSum(), equalTo(0.0)); - assertThat(stats.getMin(), equalTo(Double.POSITIVE_INFINITY)); - assertThat(stats.getMax(), equalTo(Double.NEGATIVE_INFINITY)); - assertThat(Double.isNaN(stats.getStdDeviation()), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationPopulation()), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationSampling()), is(true)); - assertThat(Double.isNaN(stats.getAvg()), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_POPULATION)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_POPULATION)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_SAMPLING)), is(true)); - assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_SAMPLING)), is(true)); - } + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + terms("value").field("value") + .subAggregation(missing("values").field("values").subAggregation(extendedStats("stats").field("value"))) + ), + response -> { + assertHitCount(response, 10); + + Terms terms = response.getAggregations().get("value"); + assertThat(terms, notNullValue()); + assertThat(terms.getBuckets().size(), equalTo(10)); + + for (Terms.Bucket bucket : terms.getBuckets()) { + assertThat(bucket.getDocCount(), equalTo(1L)); + + Missing missing = bucket.getAggregations().get("values"); + assertThat(missing, notNullValue()); + assertThat(missing.getDocCount(), equalTo(0L)); + + ExtendedStats stats = missing.getAggregations().get("stats"); + assertThat(stats, notNullValue()); + assertThat(stats.getName(), equalTo("stats")); + assertThat(stats.getSumOfSquares(), equalTo(0.0)); + assertThat(stats.getCount(), equalTo(0L)); + assertThat(stats.getSum(), equalTo(0.0)); + assertThat(stats.getMin(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(stats.getMax(), equalTo(Double.NEGATIVE_INFINITY)); + assertThat(Double.isNaN(stats.getStdDeviation()), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationPopulation()), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationSampling()), is(true)); + assertThat(Double.isNaN(stats.getAvg()), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_POPULATION)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_POPULATION)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.UPPER_SAMPLING)), is(true)); + assertThat(Double.isNaN(stats.getStdDeviationBound(Bounds.LOWER_SAMPLING)), is(true)); + } + } + ); } @Override public void testOrderByEmptyAggregation() throws Exception { - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - terms("terms").field("value") - .order(BucketOrder.compound(BucketOrder.aggregation("filter>extendedStats.avg", true))) - .subAggregation(filter("filter", termQuery("value", 100)).subAggregation(extendedStats("extendedStats").field("value"))) - ) - .get(); - - assertHitCount(searchResponse, 10); - - Terms terms = searchResponse.getAggregations().get("terms"); - assertThat(terms, notNullValue()); - List buckets = terms.getBuckets(); - assertThat(buckets, notNullValue()); - assertThat(buckets.size(), equalTo(10)); - - for (int i = 0; i < 10; i++) { - Terms.Bucket bucket = buckets.get(i); - assertThat(bucket, notNullValue()); - assertThat(bucket.getKeyAsNumber(), equalTo((long) i + 1)); - assertThat(bucket.getDocCount(), equalTo(1L)); - Filter filter = bucket.getAggregations().get("filter"); - assertThat(filter, notNullValue()); - assertThat(filter.getDocCount(), equalTo(0L)); - ExtendedStats extendedStats = filter.getAggregations().get("extendedStats"); - assertThat(extendedStats, notNullValue()); - assertThat(extendedStats.getMin(), equalTo(Double.POSITIVE_INFINITY)); - assertThat(extendedStats.getMax(), equalTo(Double.NEGATIVE_INFINITY)); - assertThat(extendedStats.getAvg(), equalTo(Double.NaN)); - assertThat(extendedStats.getSum(), equalTo(0.0)); - assertThat(extendedStats.getCount(), equalTo(0L)); - assertThat(extendedStats.getStdDeviation(), equalTo(Double.NaN)); - assertThat(extendedStats.getStdDeviationPopulation(), equalTo(Double.NaN)); - assertThat(extendedStats.getStdDeviationSampling(), equalTo(Double.NaN)); - assertThat(extendedStats.getSumOfSquares(), equalTo(0.0)); - assertThat(extendedStats.getVariance(), equalTo(Double.NaN)); - assertThat(extendedStats.getVariancePopulation(), equalTo(Double.NaN)); - assertThat(extendedStats.getVarianceSampling(), equalTo(Double.NaN)); - assertThat(extendedStats.getStdDeviationBound(Bounds.LOWER), equalTo(Double.NaN)); - assertThat(extendedStats.getStdDeviationBound(Bounds.UPPER), equalTo(Double.NaN)); - assertThat(extendedStats.getStdDeviationBound(Bounds.LOWER_POPULATION), equalTo(Double.NaN)); - assertThat(extendedStats.getStdDeviationBound(Bounds.UPPER_POPULATION), equalTo(Double.NaN)); - assertThat(extendedStats.getStdDeviationBound(Bounds.LOWER_SAMPLING), equalTo(Double.NaN)); - assertThat(extendedStats.getStdDeviationBound(Bounds.UPPER_SAMPLING), equalTo(Double.NaN)); - } + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + terms("terms").field("value") + .order(BucketOrder.compound(BucketOrder.aggregation("filter>extendedStats.avg", true))) + .subAggregation( + filter("filter", termQuery("value", 100)).subAggregation(extendedStats("extendedStats").field("value")) + ) + ), + response -> { + assertHitCount(response, 10); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + List buckets = terms.getBuckets(); + assertThat(buckets, notNullValue()); + assertThat(buckets.size(), equalTo(10)); + + for (int i = 0; i < 10; i++) { + Terms.Bucket bucket = buckets.get(i); + assertThat(bucket, notNullValue()); + assertThat(bucket.getKeyAsNumber(), equalTo((long) i + 1)); + assertThat(bucket.getDocCount(), equalTo(1L)); + Filter filter = bucket.getAggregations().get("filter"); + assertThat(filter, notNullValue()); + assertThat(filter.getDocCount(), equalTo(0L)); + ExtendedStats extendedStats = filter.getAggregations().get("extendedStats"); + assertThat(extendedStats, notNullValue()); + assertThat(extendedStats.getMin(), equalTo(Double.POSITIVE_INFINITY)); + assertThat(extendedStats.getMax(), equalTo(Double.NEGATIVE_INFINITY)); + assertThat(extendedStats.getAvg(), equalTo(Double.NaN)); + assertThat(extendedStats.getSum(), equalTo(0.0)); + assertThat(extendedStats.getCount(), equalTo(0L)); + assertThat(extendedStats.getStdDeviation(), equalTo(Double.NaN)); + assertThat(extendedStats.getStdDeviationPopulation(), equalTo(Double.NaN)); + assertThat(extendedStats.getStdDeviationSampling(), equalTo(Double.NaN)); + assertThat(extendedStats.getSumOfSquares(), equalTo(0.0)); + assertThat(extendedStats.getVariance(), equalTo(Double.NaN)); + assertThat(extendedStats.getVariancePopulation(), equalTo(Double.NaN)); + assertThat(extendedStats.getVarianceSampling(), equalTo(Double.NaN)); + assertThat(extendedStats.getStdDeviationBound(Bounds.LOWER), equalTo(Double.NaN)); + assertThat(extendedStats.getStdDeviationBound(Bounds.UPPER), equalTo(Double.NaN)); + assertThat(extendedStats.getStdDeviationBound(Bounds.LOWER_POPULATION), equalTo(Double.NaN)); + assertThat(extendedStats.getStdDeviationBound(Bounds.UPPER_POPULATION), equalTo(Double.NaN)); + assertThat(extendedStats.getStdDeviationBound(Bounds.LOWER_SAMPLING), equalTo(Double.NaN)); + assertThat(extendedStats.getStdDeviationBound(Bounds.UPPER_SAMPLING), equalTo(Double.NaN)); + } + } + ); } private void checkUpperLowerBounds(ExtendedStats stats, double sigma) { @@ -845,13 +925,13 @@ public void testScriptCaching() throws Exception { ); // Test that a request using a nondeterministic script does not get cached - SearchResponse r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - extendedStats("foo").field("d") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "Math.random()", Collections.emptyMap())) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + extendedStats("foo").field("d") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "Math.random()", Collections.emptyMap())) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -863,13 +943,13 @@ public void testScriptCaching() throws Exception { ); // Test that a request using a deterministic script gets cached - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - extendedStats("foo").field("d") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + extendedStats("foo").field("d") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -881,8 +961,7 @@ public void testScriptCaching() throws Exception { ); // Ensure that non-scripted requests are cached as normal - r = prepareSearch("cache_test_idx").setSize(0).addAggregation(extendedStats("foo").field("d")).get(); - assertNoFailures(r); + assertNoFailures(prepareSearch("cache_test_idx").setSize(0).addAggregation(extendedStats("foo").field("d"))); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -893,5 +972,4 @@ public void testScriptCaching() throws Exception { equalTo(2L) ); } - } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java index 3aebbce43e1e1..f8b633dca1a10 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/GeoBoundsIT.java @@ -8,13 +8,12 @@ package org.elasticsearch.search.aggregations.metrics; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.SpatialPoint; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.geo.RandomGeoGenerator; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailuresAndResponse; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.equalTo; @@ -26,45 +25,42 @@ public class GeoBoundsIT extends SpatialBoundsAggregationTestBase { public void testSingleValuedFieldNearDateLine() { - SearchResponse response = prepareSearch(DATELINE_IDX_NAME).addAggregation( - boundsAgg(aggName(), SINGLE_VALUED_FIELD_NAME).wrapLongitude(false) - ).get(); - - assertNoFailures(response); - - GeoPoint geoValuesTopLeft = new GeoPoint(38, -179); - GeoPoint geoValuesBottomRight = new GeoPoint(-24, 178); - - GeoBounds geoBounds = response.getAggregations().get(aggName()); - assertThat(geoBounds, notNullValue()); - assertThat(geoBounds.getName(), equalTo(aggName())); - GeoPoint topLeft = geoBounds.topLeft(); - GeoPoint bottomRight = geoBounds.bottomRight(); - assertThat(topLeft.getY(), closeTo(geoValuesTopLeft.getY(), GEOHASH_TOLERANCE)); - assertThat(topLeft.getX(), closeTo(geoValuesTopLeft.getX(), GEOHASH_TOLERANCE)); - assertThat(bottomRight.getY(), closeTo(geoValuesBottomRight.getY(), GEOHASH_TOLERANCE)); - assertThat(bottomRight.getX(), closeTo(geoValuesBottomRight.getX(), GEOHASH_TOLERANCE)); + assertNoFailuresAndResponse( + prepareSearch(DATELINE_IDX_NAME).addAggregation(boundsAgg(aggName(), SINGLE_VALUED_FIELD_NAME).wrapLongitude(false)), + response -> { + GeoPoint geoValuesTopLeft = new GeoPoint(38, -179); + GeoPoint geoValuesBottomRight = new GeoPoint(-24, 178); + + GeoBounds geoBounds = response.getAggregations().get(aggName()); + assertThat(geoBounds, notNullValue()); + assertThat(geoBounds.getName(), equalTo(aggName())); + GeoPoint topLeft = geoBounds.topLeft(); + GeoPoint bottomRight = geoBounds.bottomRight(); + assertThat(topLeft.getY(), closeTo(geoValuesTopLeft.getY(), GEOHASH_TOLERANCE)); + assertThat(topLeft.getX(), closeTo(geoValuesTopLeft.getX(), GEOHASH_TOLERANCE)); + assertThat(bottomRight.getY(), closeTo(geoValuesBottomRight.getY(), GEOHASH_TOLERANCE)); + assertThat(bottomRight.getX(), closeTo(geoValuesBottomRight.getX(), GEOHASH_TOLERANCE)); + } + ); } public void testSingleValuedFieldNearDateLineWrapLongitude() { - GeoPoint geoValuesTopLeft = new GeoPoint(38, 170); GeoPoint geoValuesBottomRight = new GeoPoint(-24, -175); - SearchResponse response = prepareSearch(DATELINE_IDX_NAME).addAggregation( - boundsAgg(aggName(), SINGLE_VALUED_FIELD_NAME).wrapLongitude(true) - ).get(); - - assertNoFailures(response); - - GeoBounds geoBounds = response.getAggregations().get(aggName()); - assertThat(geoBounds, notNullValue()); - assertThat(geoBounds.getName(), equalTo(aggName())); - GeoPoint topLeft = geoBounds.topLeft(); - GeoPoint bottomRight = geoBounds.bottomRight(); - assertThat(topLeft.getY(), closeTo(geoValuesTopLeft.getY(), GEOHASH_TOLERANCE)); - assertThat(topLeft.getX(), closeTo(geoValuesTopLeft.getX(), GEOHASH_TOLERANCE)); - assertThat(bottomRight.getY(), closeTo(geoValuesBottomRight.getY(), GEOHASH_TOLERANCE)); - assertThat(bottomRight.getX(), closeTo(geoValuesBottomRight.getX(), GEOHASH_TOLERANCE)); + assertNoFailuresAndResponse( + prepareSearch(DATELINE_IDX_NAME).addAggregation(boundsAgg(aggName(), SINGLE_VALUED_FIELD_NAME).wrapLongitude(true)), + response -> { + GeoBounds geoBounds = response.getAggregations().get(aggName()); + assertThat(geoBounds, notNullValue()); + assertThat(geoBounds.getName(), equalTo(aggName())); + GeoPoint topLeft = geoBounds.topLeft(); + GeoPoint bottomRight = geoBounds.bottomRight(); + assertThat(topLeft.getY(), closeTo(geoValuesTopLeft.getY(), GEOHASH_TOLERANCE)); + assertThat(topLeft.getX(), closeTo(geoValuesTopLeft.getX(), GEOHASH_TOLERANCE)); + assertThat(bottomRight.getY(), closeTo(geoValuesBottomRight.getY(), GEOHASH_TOLERANCE)); + assertThat(bottomRight.getX(), closeTo(geoValuesBottomRight.getX(), GEOHASH_TOLERANCE)); + } + ); } @Override diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java index 4b12cddde691f..a7d32863718e3 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java @@ -8,7 +8,6 @@ package org.elasticsearch.search.aggregations.metrics; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.SpatialPoint; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGrid; @@ -18,7 +17,7 @@ import java.util.List; import static org.elasticsearch.search.aggregations.AggregationBuilders.geohashGrid; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailuresAndResponse; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; @@ -29,21 +28,24 @@ public class GeoCentroidIT extends CentroidAggregationTestBase { public void testSingleValueFieldAsSubAggToGeohashGrid() { - SearchResponse response = prepareSearch(HIGH_CARD_IDX_NAME).addAggregation( - geohashGrid("geoGrid").field(SINGLE_VALUED_FIELD_NAME).subAggregation(centroidAgg(aggName()).field(SINGLE_VALUED_FIELD_NAME)) - ).get(); - assertNoFailures(response); - - GeoGrid grid = response.getAggregations().get("geoGrid"); - assertThat(grid, notNullValue()); - assertThat(grid.getName(), equalTo("geoGrid")); - List buckets = grid.getBuckets(); - for (GeoGrid.Bucket cell : buckets) { - String geohash = cell.getKeyAsString(); - SpatialPoint expectedCentroid = expectedCentroidsForGeoHash.get(geohash); - GeoCentroid centroidAgg = cell.getAggregations().get(aggName()); - assertSameCentroid(centroidAgg.centroid(), expectedCentroid); - } + assertNoFailuresAndResponse( + prepareSearch(HIGH_CARD_IDX_NAME).addAggregation( + geohashGrid("geoGrid").field(SINGLE_VALUED_FIELD_NAME) + .subAggregation(centroidAgg(aggName()).field(SINGLE_VALUED_FIELD_NAME)) + ), + response -> { + GeoGrid grid = response.getAggregations().get("geoGrid"); + assertThat(grid, notNullValue()); + assertThat(grid.getName(), equalTo("geoGrid")); + List buckets = grid.getBuckets(); + for (GeoGrid.Bucket cell : buckets) { + String geohash = cell.getKeyAsString(); + SpatialPoint expectedCentroid = expectedCentroidsForGeoHash.get(geohash); + GeoCentroid centroidAgg = cell.getAggregations().get(aggName()); + assertSameCentroid(centroidAgg.centroid(), expectedCentroid); + } + } + ); } @Override diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksIT.java index 7d5e446d591bb..d4b5be3045cdf 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/HDRPercentileRanksIT.java @@ -8,7 +8,6 @@ package org.elasticsearch.search.aggregations.metrics; import org.apache.logging.log4j.LogManager; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.Script; @@ -40,6 +39,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -99,70 +99,76 @@ private void assertConsistent(double[] pcts, PercentileRanks values, long minVal @Override public void testEmptyAggregation() throws Exception { int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("empty_bucket_idx").setQuery(matchAllQuery()) - .addAggregation( - histogram("histo").field("value") - .interval(1L) - .minDocCount(0) - .subAggregation( - percentileRanks("percentile_ranks", new double[] { 10, 15 }).field("value") - .method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - ) - ) - .get(); - - assertThat(searchResponse.getHits().getTotalHits().value, equalTo(2L)); - Histogram histo = searchResponse.getAggregations().get("histo"); - assertThat(histo, notNullValue()); - Histogram.Bucket bucket = histo.getBuckets().get(1); - assertThat(bucket, notNullValue()); - - PercentileRanks reversePercentiles = bucket.getAggregations().get("percentile_ranks"); - assertThat(reversePercentiles, notNullValue()); - assertThat(reversePercentiles.getName(), equalTo("percentile_ranks")); - assertThat(reversePercentiles.percent(10), equalTo(Double.NaN)); - assertThat(reversePercentiles.percent(15), equalTo(Double.NaN)); + assertResponse( + prepareSearch("empty_bucket_idx").setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field("value") + .interval(1L) + .minDocCount(0) + .subAggregation( + percentileRanks("percentile_ranks", new double[] { 10, 15 }).field("value") + .method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + ) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(2L)); + Histogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + Histogram.Bucket bucket = histo.getBuckets().get(1); + assertThat(bucket, notNullValue()); + + PercentileRanks reversePercentiles = bucket.getAggregations().get("percentile_ranks"); + assertThat(reversePercentiles, notNullValue()); + assertThat(reversePercentiles.getName(), equalTo("percentile_ranks")); + assertThat(reversePercentiles.percent(10), equalTo(Double.NaN)); + assertThat(reversePercentiles.percent(15), equalTo(Double.NaN)); + } + ); } @Override public void testUnmapped() throws Exception { int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx_unmapped").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", new double[] { 0, 10, 15, 100 }).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("value") - ) - .get(); - - assertThat(searchResponse.getHits().getTotalHits().value, equalTo(0L)); - - PercentileRanks reversePercentiles = searchResponse.getAggregations().get("percentile_ranks"); - assertThat(reversePercentiles, notNullValue()); - assertThat(reversePercentiles.getName(), equalTo("percentile_ranks")); - assertThat(reversePercentiles.percent(0), equalTo(Double.NaN)); - assertThat(reversePercentiles.percent(10), equalTo(Double.NaN)); - assertThat(reversePercentiles.percent(15), equalTo(Double.NaN)); - assertThat(reversePercentiles.percent(100), equalTo(Double.NaN)); + assertResponse( + prepareSearch("idx_unmapped").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", new double[] { 0, 10, 15, 100 }).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("value") + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(0L)); + + PercentileRanks reversePercentiles = response.getAggregations().get("percentile_ranks"); + assertThat(reversePercentiles, notNullValue()); + assertThat(reversePercentiles.getName(), equalTo("percentile_ranks")); + assertThat(reversePercentiles.percent(0), equalTo(Double.NaN)); + assertThat(reversePercentiles.percent(10), equalTo(Double.NaN)); + assertThat(reversePercentiles.percent(15), equalTo(Double.NaN)); + assertThat(reversePercentiles.percent(100), equalTo(Double.NaN)); + } + ); } @Override public void testSingleValuedField() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = randomPercents(minValue, maxValue); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("value") - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValue, maxValue, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("value") + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValue, maxValue, sigDigits); + } + ); } public void testNullValuesField() throws Exception { @@ -201,84 +207,91 @@ public void testEmptyValuesField() throws Exception { public void testSingleValuedFieldGetProperty() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = randomPercents(minValue, maxValue); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - global("global").subAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("value") - ) - ) - .get(); - - assertHitCount(searchResponse, 10); - - Global global = searchResponse.getAggregations().get("global"); - assertThat(global, notNullValue()); - assertThat(global.getName(), equalTo("global")); - assertThat(global.getDocCount(), equalTo(10L)); - assertThat(global.getAggregations(), notNullValue()); - assertThat(global.getAggregations().asMap().size(), equalTo(1)); - - PercentileRanks values = global.getAggregations().get("percentile_ranks"); - assertThat(values, notNullValue()); - assertThat(values.getName(), equalTo("percentile_ranks")); - assertThat(((InternalAggregation) global).getProperty("percentile_ranks"), sameInstance(values)); - + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + global("global").subAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("value") + ) + ), + response -> { + assertHitCount(response, 10); + + Global global = response.getAggregations().get("global"); + assertThat(global, notNullValue()); + assertThat(global.getName(), equalTo("global")); + assertThat(global.getDocCount(), equalTo(10L)); + assertThat(global.getAggregations(), notNullValue()); + assertThat(global.getAggregations().asMap().size(), equalTo(1)); + + PercentileRanks values = global.getAggregations().get("percentile_ranks"); + assertThat(values, notNullValue()); + assertThat(values.getName(), equalTo("percentile_ranks")); + assertThat(((InternalAggregation) global).getProperty("percentile_ranks"), sameInstance(values)); + } + ); } public void testSingleValuedFieldOutsideRange() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = new double[] { minValue - 1, maxValue + 1 }; - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("value") - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValue, maxValue, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("value") + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValue, maxValue, sigDigits); + } + ); } @Override public void testSingleValuedFieldPartiallyUnmapped() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = randomPercents(minValue, maxValue); - SearchResponse searchResponse = prepareSearch("idx", "idx_unmapped").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("value") - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValue, maxValue, sigDigits); + assertResponse( + prepareSearch("idx", "idx_unmapped").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("value") + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValue, maxValue, sigDigits); + } + ); } @Override public void testSingleValuedFieldWithValueScript() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = randomPercents(minValue - 1, maxValue - 1); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("value") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValue - 1, maxValue - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("value") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValue - 1, maxValue - 1, sigDigits); + } + ); } @Override @@ -287,74 +300,82 @@ public void testSingleValuedFieldWithValueScriptWithParams() throws Exception { Map params = new HashMap<>(); params.put("dec", 1); final double[] pcts = randomPercents(minValue - 1, maxValue - 1); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("value") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValue - 1, maxValue - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("value") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValue - 1, maxValue - 1, sigDigits); + } + ); } @Override public void testMultiValuedField() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = randomPercents(minValues, maxValues); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("values") - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValues, maxValues, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("values") + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValues, maxValues, sigDigits); + } + ); } @Override public void testMultiValuedFieldWithValueScript() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = randomPercents(minValues - 1, maxValues - 1); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValues - 1, maxValues - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValues - 1, maxValues - 1, sigDigits); + } + ); } public void testMultiValuedFieldWithValueScriptReverse() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = randomPercents(20 - maxValues, 20 - minValues); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "20 - _value", emptyMap())) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, 20 - maxValues, 20 - minValues, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "20 - _value", emptyMap())) + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, 20 - maxValues, 20 - minValues, sigDigits); + } + ); } @Override @@ -363,37 +384,41 @@ public void testMultiValuedFieldWithValueScriptWithParams() throws Exception { Map params = new HashMap<>(); params.put("dec", 1); final double[] pcts = randomPercents(minValues - 1, maxValues - 1); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValues - 1, maxValues - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValues - 1, maxValues - 1, sigDigits); + } + ); } @Override public void testScriptSingleValued() throws Exception { int sigDigits = randomSignificantDigits(); final double[] pcts = randomPercents(minValue, maxValue); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", emptyMap())) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValue, maxValue, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", emptyMap())) + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValue, maxValue, sigDigits); + } + ); } @Override @@ -405,18 +430,20 @@ public void testScriptSingleValuedWithParams() throws Exception { Script script = new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value - dec", params); final double[] pcts = randomPercents(minValue - 1, maxValue - 1); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .script(script) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValue - 1, maxValue - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .script(script) + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValue - 1, maxValue - 1, sigDigits); + } + ); } @Override @@ -426,18 +453,20 @@ public void testScriptMultiValued() throws Exception { Script script = new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['values']", emptyMap()); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .script(script) - ) - .get(); - - assertHitCount(searchResponse, 10); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .script(script) + ), + response -> { + assertHitCount(response, 10); - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValues, maxValues, sigDigits); + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValues, maxValues, sigDigits); + } + ); } @Override @@ -446,87 +475,93 @@ public void testScriptMultiValuedWithParams() throws Exception { Script script = AggregationTestScriptsPlugin.DECREMENT_ALL_VALUES; final double[] pcts = randomPercents(minValues - 1, maxValues - 1); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .script(script) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final PercentileRanks values = searchResponse.getAggregations().get("percentile_ranks"); - assertConsistent(pcts, values, minValues - 1, maxValues - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentileRanks("percentile_ranks", pcts).method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .script(script) + ), + response -> { + assertHitCount(response, 10); + + final PercentileRanks values = response.getAggregations().get("percentile_ranks"); + assertConsistent(pcts, values, minValues - 1, maxValues - 1, sigDigits); + } + ); } public void testOrderBySubAggregation() { int sigDigits = randomSignificantDigits(); boolean asc = randomBoolean(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - histogram("histo").field("value") - .interval(2L) - .subAggregation( - percentileRanks("percentile_ranks", new double[] { 99 }).field("value") - .method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - ) - .order(BucketOrder.aggregation("percentile_ranks", "99", asc)) - ) - .get(); - - assertHitCount(searchResponse, 10); - - Histogram histo = searchResponse.getAggregations().get("histo"); - double previous = asc ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY; - for (Histogram.Bucket bucket : histo.getBuckets()) { - PercentileRanks values = bucket.getAggregations().get("percentile_ranks"); - double p99 = values.percent(99); - if (asc) { - assertThat(p99, greaterThanOrEqualTo(previous)); - } else { - assertThat(p99, lessThanOrEqualTo(previous)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field("value") + .interval(2L) + .subAggregation( + percentileRanks("percentile_ranks", new double[] { 99 }).field("value") + .method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + ) + .order(BucketOrder.aggregation("percentile_ranks", "99", asc)) + ), + response -> { + assertHitCount(response, 10); + + Histogram histo = response.getAggregations().get("histo"); + double previous = asc ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY; + for (Histogram.Bucket bucket : histo.getBuckets()) { + PercentileRanks values = bucket.getAggregations().get("percentile_ranks"); + double p99 = values.percent(99); + if (asc) { + assertThat(p99, greaterThanOrEqualTo(previous)); + } else { + assertThat(p99, lessThanOrEqualTo(previous)); + } + previous = p99; + } } - previous = p99; - } + ); } @Override public void testOrderByEmptyAggregation() throws Exception { - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - terms("terms").field("value") - .order(BucketOrder.compound(BucketOrder.aggregation("filter>ranks.99", true))) - .subAggregation( - filter("filter", termQuery("value", 100)).subAggregation( - percentileRanks("ranks", new double[] { 99 }).method(PercentilesMethod.HDR).field("value") + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + terms("terms").field("value") + .order(BucketOrder.compound(BucketOrder.aggregation("filter>ranks.99", true))) + .subAggregation( + filter("filter", termQuery("value", 100)).subAggregation( + percentileRanks("ranks", new double[] { 99 }).method(PercentilesMethod.HDR).field("value") + ) ) - ) - ) - .get(); - - assertHitCount(searchResponse, 10); - - Terms terms = searchResponse.getAggregations().get("terms"); - assertThat(terms, notNullValue()); - List buckets = terms.getBuckets(); - assertThat(buckets, notNullValue()); - assertThat(buckets.size(), equalTo(10)); - - for (int i = 0; i < 10; i++) { - Terms.Bucket bucket = buckets.get(i); - assertThat(bucket, notNullValue()); - assertThat(bucket.getKeyAsNumber(), equalTo((long) i + 1)); - assertThat(bucket.getDocCount(), equalTo(1L)); - Filter filter = bucket.getAggregations().get("filter"); - assertThat(filter, notNullValue()); - assertThat(filter.getDocCount(), equalTo(0L)); - PercentileRanks ranks = filter.getAggregations().get("ranks"); - assertThat(ranks, notNullValue()); - assertThat(ranks.percent(99), equalTo(Double.NaN)); - - } + ), + response -> { + assertHitCount(response, 10); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + List buckets = terms.getBuckets(); + assertThat(buckets, notNullValue()); + assertThat(buckets.size(), equalTo(10)); + + for (int i = 0; i < 10; i++) { + Terms.Bucket bucket = buckets.get(i); + assertThat(bucket, notNullValue()); + assertThat(bucket.getKeyAsNumber(), equalTo((long) i + 1)); + assertThat(bucket.getDocCount(), equalTo(1L)); + Filter filter = bucket.getAggregations().get("filter"); + assertThat(filter, notNullValue()); + assertThat(filter.getDocCount(), equalTo(0L)); + PercentileRanks ranks = filter.getAggregations().get("ranks"); + assertThat(ranks, notNullValue()); + assertThat(ranks.percent(99), equalTo(Double.NaN)); + + } + } + ); } /** @@ -555,14 +590,14 @@ public void testScriptCaching() throws Exception { ); // Test that a request using a nondeterministic script does not get cached - SearchResponse r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - percentileRanks("foo", new double[] { 50.0 }).method(PercentilesMethod.HDR) - .field("d") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "Math.random()", emptyMap())) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + percentileRanks("foo", new double[] { 50.0 }).method(PercentilesMethod.HDR) + .field("d") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "Math.random()", emptyMap())) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -574,14 +609,14 @@ public void testScriptCaching() throws Exception { ); // Test that a request using a deterministic script gets cached - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - percentileRanks("foo", new double[] { 50.0 }).method(PercentilesMethod.HDR) - .field("d") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + percentileRanks("foo", new double[] { 50.0 }).method(PercentilesMethod.HDR) + .field("d") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -593,10 +628,10 @@ public void testScriptCaching() throws Exception { ); // Ensure that non-scripted requests are cached as normal - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation(percentileRanks("foo", new double[] { 50.0 }).method(PercentilesMethod.HDR).field("d")) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation(percentileRanks("foo", new double[] { 50.0 }).method(PercentilesMethod.HDR).field("d")) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -607,5 +642,4 @@ public void testScriptCaching() throws Exception { equalTo(2L) ); } - } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesIT.java index 3ac50c7b5e104..9eac8d4a06a43 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/HDRPercentilesIT.java @@ -8,7 +8,6 @@ package org.elasticsearch.search.aggregations.metrics; import org.apache.logging.log4j.LogManager; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.plugins.Plugin; @@ -42,6 +41,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -102,143 +102,154 @@ private void assertConsistent(double[] pcts, Percentiles percentiles, long minVa @Override public void testEmptyAggregation() throws Exception { int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("empty_bucket_idx").setQuery(matchAllQuery()) - .addAggregation( - histogram("histo").field("value") - .interval(1L) - .minDocCount(0) - .subAggregation( - percentiles("percentiles").field("value") - .numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .percentiles(10, 15) - ) - ) - .get(); - - assertThat(searchResponse.getHits().getTotalHits().value, equalTo(2L)); - Histogram histo = searchResponse.getAggregations().get("histo"); - assertThat(histo, notNullValue()); - Histogram.Bucket bucket = histo.getBuckets().get(1); - assertThat(bucket, notNullValue()); - - Percentiles percentiles = bucket.getAggregations().get("percentiles"); - assertThat(percentiles, notNullValue()); - assertThat(percentiles.getName(), equalTo("percentiles")); - assertThat(percentiles.percentile(10), equalTo(Double.NaN)); - assertThat(percentiles.percentile(15), equalTo(Double.NaN)); + assertResponse( + prepareSearch("empty_bucket_idx").setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field("value") + .interval(1L) + .minDocCount(0) + .subAggregation( + percentiles("percentiles").field("value") + .numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .percentiles(10, 15) + ) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(2L)); + Histogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + Histogram.Bucket bucket = histo.getBuckets().get(1); + assertThat(bucket, notNullValue()); + + Percentiles percentiles = bucket.getAggregations().get("percentiles"); + assertThat(percentiles, notNullValue()); + assertThat(percentiles.getName(), equalTo("percentiles")); + assertThat(percentiles.percentile(10), equalTo(Double.NaN)); + assertThat(percentiles.percentile(15), equalTo(Double.NaN)); + } + ); } @Override public void testUnmapped() throws Exception { int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx_unmapped").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("value") - .percentiles(0, 10, 15, 100) - ) - .get(); - - assertThat(searchResponse.getHits().getTotalHits().value, equalTo(0L)); - - Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertThat(percentiles, notNullValue()); - assertThat(percentiles.getName(), equalTo("percentiles")); - assertThat(percentiles.percentile(0), equalTo(Double.NaN)); - assertThat(percentiles.percentile(10), equalTo(Double.NaN)); - assertThat(percentiles.percentile(15), equalTo(Double.NaN)); - assertThat(percentiles.percentile(100), equalTo(Double.NaN)); + assertResponse( + prepareSearch("idx_unmapped").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("value") + .percentiles(0, 10, 15, 100) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(0L)); + + Percentiles percentiles = response.getAggregations().get("percentiles"); + assertThat(percentiles, notNullValue()); + assertThat(percentiles.getName(), equalTo("percentiles")); + assertThat(percentiles.percentile(0), equalTo(Double.NaN)); + assertThat(percentiles.percentile(10), equalTo(Double.NaN)); + assertThat(percentiles.percentile(15), equalTo(Double.NaN)); + assertThat(percentiles.percentile(100), equalTo(Double.NaN)); + } + ); } @Override public void testSingleValuedField() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomIntBetween(1, 5); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("value") - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValue, maxValue, sigDigits); - } - - @Override - public void testSingleValuedFieldGetProperty() throws Exception { - final double[] pcts = randomPercentiles(); - int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - global("global").subAggregation( + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) .method(PercentilesMethod.HDR) .field("value") .percentiles(pcts) - ) - ) - .get(); - - assertHitCount(searchResponse, 10); - - Global global = searchResponse.getAggregations().get("global"); - assertThat(global, notNullValue()); - assertThat(global.getName(), equalTo("global")); - assertThat(global.getDocCount(), equalTo(10L)); - assertThat(global.getAggregations(), notNullValue()); - assertThat(global.getAggregations().asMap().size(), equalTo(1)); + ), + response -> { + assertHitCount(response, 10); - Percentiles percentiles = global.getAggregations().get("percentiles"); - assertThat(percentiles, notNullValue()); - assertThat(percentiles.getName(), equalTo("percentiles")); - assertThat(((InternalAggregation) global).getProperty("percentiles"), sameInstance(percentiles)); + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValue, maxValue, sigDigits); + } + ); + } + @Override + public void testSingleValuedFieldGetProperty() throws Exception { + final double[] pcts = randomPercentiles(); + int sigDigits = randomSignificantDigits(); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + global("global").subAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("value") + .percentiles(pcts) + ) + ), + response -> { + assertHitCount(response, 10); + + Global global = response.getAggregations().get("global"); + assertThat(global, notNullValue()); + assertThat(global.getName(), equalTo("global")); + assertThat(global.getDocCount(), equalTo(10L)); + assertThat(global.getAggregations(), notNullValue()); + assertThat(global.getAggregations().asMap().size(), equalTo(1)); + + Percentiles percentiles = global.getAggregations().get("percentiles"); + assertThat(percentiles, notNullValue()); + assertThat(percentiles.getName(), equalTo("percentiles")); + assertThat(((InternalAggregation) global).getProperty("percentiles"), sameInstance(percentiles)); + } + ); } @Override public void testSingleValuedFieldPartiallyUnmapped() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx", "idx_unmapped").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("value") - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValue, maxValue, sigDigits); + assertResponse( + prepareSearch("idx", "idx_unmapped").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("value") + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValue, maxValue, sigDigits); + } + ); } @Override public void testSingleValuedFieldWithValueScript() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("value") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValue - 1, maxValue - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("value") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValue - 1, maxValue - 1, sigDigits); + } + ); } @Override @@ -248,78 +259,86 @@ public void testSingleValuedFieldWithValueScriptWithParams() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("value") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValue - 1, maxValue - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("value") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValue - 1, maxValue - 1, sigDigits); + } + ); } @Override public void testMultiValuedField() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("values") - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValues, maxValues, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("values") + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValues, maxValues, sigDigits); + } + ); } @Override public void testMultiValuedFieldWithValueScript() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValues - 1, maxValues - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValues - 1, maxValues - 1, sigDigits); + } + ); } public void testMultiValuedFieldWithValueScriptReverse() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "20 - _value", emptyMap())) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, 20 - maxValues, 20 - minValues, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "20 - _value", emptyMap())) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, 20 - maxValues, 20 - minValues, sigDigits); + } + ); } @Override @@ -329,39 +348,43 @@ public void testMultiValuedFieldWithValueScriptWithParams() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValues - 1, maxValues - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - dec", params)) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValues - 1, maxValues - 1, sigDigits); + } + ); } @Override public void testScriptSingleValued() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", emptyMap())) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValue, maxValue, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", emptyMap())) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValue, maxValue, sigDigits); + } + ); } @Override @@ -373,19 +396,21 @@ public void testScriptSingleValuedWithParams() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .script(script) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValue - 1, maxValue - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .script(script) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValue - 1, maxValue - 1, sigDigits); + } + ); } @Override @@ -395,19 +420,21 @@ public void testScriptMultiValued() throws Exception { Script script = new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['values']", emptyMap()); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .script(script) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .script(script) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValues, maxValues, sigDigits); + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValues, maxValues, sigDigits); + } + ); } @Override @@ -416,89 +443,96 @@ public void testScriptMultiValuedWithParams() throws Exception { final double[] pcts = randomPercentiles(); int sigDigits = randomSignificantDigits(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) - .method(PercentilesMethod.HDR) - .script(script) - .percentiles(pcts) - ) - .get(); - - assertHitCount(searchResponse, 10); - - final Percentiles percentiles = searchResponse.getAggregations().get("percentiles"); - assertConsistent(pcts, percentiles, minValues - 1, maxValues - 1, sigDigits); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + percentiles("percentiles").numberOfSignificantValueDigits(sigDigits) + .method(PercentilesMethod.HDR) + .script(script) + .percentiles(pcts) + ), + response -> { + assertHitCount(response, 10); + + final Percentiles percentiles = response.getAggregations().get("percentiles"); + assertConsistent(pcts, percentiles, minValues - 1, maxValues - 1, sigDigits); + + } + ); } public void testOrderBySubAggregation() { int sigDigits = randomSignificantDigits(); boolean asc = randomBoolean(); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - histogram("histo").field("value") - .interval(2L) - .subAggregation( - percentiles("percentiles").field("value") - .method(PercentilesMethod.HDR) - .numberOfSignificantValueDigits(sigDigits) - .percentiles(99) - ) - .order(BucketOrder.aggregation("percentiles", "99", asc)) - ) - .get(); - - assertHitCount(searchResponse, 10); - - Histogram histo = searchResponse.getAggregations().get("histo"); - double previous = asc ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY; - for (Histogram.Bucket bucket : histo.getBuckets()) { - Percentiles percentiles = bucket.getAggregations().get("percentiles"); - double p99 = percentiles.percentile(99); - if (asc) { - assertThat(p99, greaterThanOrEqualTo(previous)); - } else { - assertThat(p99, lessThanOrEqualTo(previous)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field("value") + .interval(2L) + .subAggregation( + percentiles("percentiles").field("value") + .method(PercentilesMethod.HDR) + .numberOfSignificantValueDigits(sigDigits) + .percentiles(99) + ) + .order(BucketOrder.aggregation("percentiles", "99", asc)) + ), + response -> { + assertHitCount(response, 10); + + Histogram histo = response.getAggregations().get("histo"); + double previous = asc ? Double.NEGATIVE_INFINITY : Double.POSITIVE_INFINITY; + for (Histogram.Bucket bucket : histo.getBuckets()) { + Percentiles percentiles = bucket.getAggregations().get("percentiles"); + double p99 = percentiles.percentile(99); + if (asc) { + assertThat(p99, greaterThanOrEqualTo(previous)); + } else { + assertThat(p99, lessThanOrEqualTo(previous)); + } + previous = p99; + } } - previous = p99; - } + ); } @Override public void testOrderByEmptyAggregation() throws Exception { - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - terms("terms").field("value") - .order(BucketOrder.compound(BucketOrder.aggregation("filter>percentiles.99", true))) - .subAggregation( - filter("filter", termQuery("value", 100)).subAggregation( - percentiles("percentiles").method(PercentilesMethod.HDR).field("value") + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + terms("terms").field("value") + .order(BucketOrder.compound(BucketOrder.aggregation("filter>percentiles.99", true))) + .subAggregation( + filter("filter", termQuery("value", 100)).subAggregation( + percentiles("percentiles").method(PercentilesMethod.HDR).field("value") + ) ) - ) - ) - .get(); - - assertHitCount(searchResponse, 10); - - Terms terms = searchResponse.getAggregations().get("terms"); - assertThat(terms, notNullValue()); - List buckets = terms.getBuckets(); - assertThat(buckets, notNullValue()); - assertThat(buckets.size(), equalTo(10)); - - for (int i = 0; i < 10; i++) { - Terms.Bucket bucket = buckets.get(i); - assertThat(bucket, notNullValue()); - assertThat(bucket.getKeyAsNumber(), equalTo((long) i + 1)); - assertThat(bucket.getDocCount(), equalTo(1L)); - Filter filter = bucket.getAggregations().get("filter"); - assertThat(filter, notNullValue()); - assertThat(filter.getDocCount(), equalTo(0L)); - Percentiles percentiles = filter.getAggregations().get("percentiles"); - assertThat(percentiles, notNullValue()); - assertThat(percentiles.percentile(99), equalTo(Double.NaN)); - - } + ), + response -> { + assertHitCount(response, 10); + + Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + List buckets = terms.getBuckets(); + assertThat(buckets, notNullValue()); + assertThat(buckets.size(), equalTo(10)); + + for (int i = 0; i < 10; i++) { + Terms.Bucket bucket = buckets.get(i); + assertThat(bucket, notNullValue()); + assertThat(bucket.getKeyAsNumber(), equalTo((long) i + 1)); + assertThat(bucket.getDocCount(), equalTo(1L)); + Filter filter = bucket.getAggregations().get("filter"); + assertThat(filter, notNullValue()); + assertThat(filter.getDocCount(), equalTo(0L)); + Percentiles percentiles = filter.getAggregations().get("percentiles"); + assertThat(percentiles, notNullValue()); + assertThat(percentiles.percentile(99), equalTo(Double.NaN)); + + } + } + ); } /** @@ -527,15 +561,15 @@ public void testScriptCaching() throws Exception { ); // Test that a request using a nondeterministic script does not get cached - SearchResponse r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - percentiles("foo").method(PercentilesMethod.HDR) - .field("d") - .percentiles(50.0) - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "Math.random()", emptyMap())) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + percentiles("foo").method(PercentilesMethod.HDR) + .field("d") + .percentiles(50.0) + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "Math.random()", emptyMap())) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -547,15 +581,15 @@ public void testScriptCaching() throws Exception { ); // Test that a request using a deterministic script gets cached - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - percentiles("foo").method(PercentilesMethod.HDR) - .field("d") - .percentiles(50.0) - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + percentiles("foo").method(PercentilesMethod.HDR) + .field("d") + .percentiles(50.0) + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -567,10 +601,10 @@ public void testScriptCaching() throws Exception { ); // Ensure that non-scripted requests are cached as normal - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation(percentiles("foo").method(PercentilesMethod.HDR).field("d").percentiles(50.0)) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation(percentiles("foo").method(PercentilesMethod.HDR).field("d").percentiles(50.0)) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -581,5 +615,4 @@ public void testScriptCaching() throws Exception { equalTo(2L) ); } - } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationIT.java index dae90424495a3..f494a339a7a71 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/MedianAbsoluteDeviationIT.java @@ -9,7 +9,6 @@ package org.elasticsearch.search.aggregations.metrics; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.Script; @@ -46,6 +45,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -138,21 +138,24 @@ private static MedianAbsoluteDeviationAggregationBuilder randomBuilder() { @Override public void testEmptyAggregation() throws Exception { - final SearchResponse response = prepareSearch("empty_bucket_idx").addAggregation( - histogram("histogram").field("value").interval(1).minDocCount(0).subAggregation(randomBuilder().field("value")) - ).get(); - - assertHitCount(response, 2); - - final Histogram histogram = response.getAggregations().get("histogram"); - assertThat(histogram, notNullValue()); - final Histogram.Bucket bucket = histogram.getBuckets().get(1); - assertThat(bucket, notNullValue()); - - final MedianAbsoluteDeviation mad = bucket.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - assertThat(mad.getMedianAbsoluteDeviation(), is(Double.NaN)); + assertResponse( + prepareSearch("empty_bucket_idx").addAggregation( + histogram("histogram").field("value").interval(1).minDocCount(0).subAggregation(randomBuilder().field("value")) + ), + response -> { + assertHitCount(response, 2); + + final Histogram histogram = response.getAggregations().get("histogram"); + assertThat(histogram, notNullValue()); + final Histogram.Bucket bucket = histogram.getBuckets().get(1); + assertThat(bucket, notNullValue()); + + final MedianAbsoluteDeviation mad = bucket.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + assertThat(mad.getMedianAbsoluteDeviation(), is(Double.NaN)); + } + ); } @Override @@ -162,68 +165,72 @@ public void testUnmapped() throws Exception { @Override public void testSingleValuedField() throws Exception { - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(randomBuilder().field("value")).get(); + assertResponse(prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(randomBuilder().field("value")), response -> { + assertHitCount(response, NUMBER_OF_DOCS); - assertHitCount(response, NUMBER_OF_DOCS); - - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD)); + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD)); + }); } @Override public void testSingleValuedFieldGetProperty() throws Exception { - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(global("global").subAggregation(randomBuilder().field("value"))) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final Global global = response.getAggregations().get("global"); - assertThat(global, notNullValue()); - assertThat(global.getName(), is("global")); - assertThat(global.getDocCount(), is((long) NUMBER_OF_DOCS)); - assertThat(global.getAggregations(), notNullValue()); - assertThat(global.getAggregations().asMap().entrySet(), hasSize(1)); - - final MedianAbsoluteDeviation mad = global.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - assertThat(((InternalAggregation) global).getProperty("mad"), sameInstance(mad)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(global("global").subAggregation(randomBuilder().field("value"))), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final Global global = response.getAggregations().get("global"); + assertThat(global, notNullValue()); + assertThat(global.getName(), is("global")); + assertThat(global.getDocCount(), is((long) NUMBER_OF_DOCS)); + assertThat(global.getAggregations(), notNullValue()); + assertThat(global.getAggregations().asMap().entrySet(), hasSize(1)); + + final MedianAbsoluteDeviation mad = global.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + assertThat(((InternalAggregation) global).getProperty("mad"), sameInstance(mad)); + } + ); } @Override public void testSingleValuedFieldPartiallyUnmapped() throws Exception { - final SearchResponse response = prepareSearch("idx", "idx_unmapped").setQuery(matchAllQuery()) - .addAggregation(randomBuilder().field("value")) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD)); + assertResponse( + prepareSearch("idx", "idx_unmapped").setQuery(matchAllQuery()).addAggregation(randomBuilder().field("value")), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD)); + } + ); } @Override public void testSingleValuedFieldWithValueScript() throws Exception { - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - randomBuilder().field("value") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - - final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample).map(point -> point + 1).toArray()); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + randomBuilder().field("value") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + + final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample).map(point -> point + 1).toArray()); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + } + ); } @Override @@ -231,53 +238,55 @@ public void testSingleValuedFieldWithValueScriptWithParams() throws Exception { final Map params = new HashMap<>(); params.put("inc", 1); - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - randomBuilder().field("value") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + inc", params)) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - - final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample).map(point -> point + 1).toArray()); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + randomBuilder().field("value") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + inc", params)) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + + final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample).map(point -> point + 1).toArray()); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + } + ); } @Override public void testMultiValuedField() throws Exception { - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(randomBuilder().field("values")) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); + assertResponse(prepareSearch("idx").setQuery(matchAllQuery()).addAggregation(randomBuilder().field("values")), response -> { + assertHitCount(response, NUMBER_OF_DOCS); - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(multiValueExactMAD)); + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(multiValueExactMAD)); + }); } @Override public void testMultiValuedFieldWithValueScript() throws Exception { - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - randomBuilder().field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - - final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(multiValueSample).map(point -> point + 1).toArray()); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + randomBuilder().field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + 1", Collections.emptyMap())) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + + final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(multiValueSample).map(point -> point + 1).toArray()); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + } + ); } @Override @@ -285,38 +294,42 @@ public void testMultiValuedFieldWithValueScriptWithParams() throws Exception { final Map params = new HashMap<>(); params.put("inc", 1); - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - randomBuilder().field("values") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + inc", params)) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - - final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(multiValueSample).map(point -> point + 1).toArray()); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + randomBuilder().field("values") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value + inc", params)) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + + final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(multiValueSample).map(point -> point + 1).toArray()); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + } + ); } @Override public void testScriptSingleValued() throws Exception { - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - randomBuilder().script( - new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", Collections.emptyMap()) - ) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + randomBuilder().script( + new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value", Collections.emptyMap()) + ) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(singleValueExactMAD)); + } + ); } @Override @@ -324,38 +337,44 @@ public void testScriptSingleValuedWithParams() throws Exception { final Map params = new HashMap<>(); params.put("inc", 1); - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - randomBuilder().script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value + inc", params)) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + randomBuilder().script( + new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['value'].value + inc", params) + ) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); - final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample).map(point -> point + 1).toArray()); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + final double fromIncrementedSampleMAD = calculateMAD(Arrays.stream(singleValueSample).map(point -> point + 1).toArray()); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + } + ); } @Override public void testScriptMultiValued() throws Exception { - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - randomBuilder().script( - new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['values']", Collections.emptyMap()) - ) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(multiValueExactMAD)); + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + randomBuilder().script( + new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "doc['values']", Collections.emptyMap()) + ) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(multiValueExactMAD)); + } + ); } @Override @@ -363,107 +382,112 @@ public void testScriptMultiValuedWithParams() throws Exception { final Map params = new HashMap<>(); params.put("inc", 1); - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - randomBuilder().script( - new Script( - ScriptType.INLINE, - AggregationTestScriptsPlugin.NAME, - "[ doc['value'].value, doc['value'].value + inc ]", - params + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + randomBuilder().script( + new Script( + ScriptType.INLINE, + AggregationTestScriptsPlugin.NAME, + "[ doc['value'].value, doc['value'].value + inc ]", + params + ) ) - ) - ) - .get(); + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); - assertHitCount(response, NUMBER_OF_DOCS); + final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getName(), is("mad")); - final MedianAbsoluteDeviation mad = response.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getName(), is("mad")); - - final double fromIncrementedSampleMAD = calculateMAD( - Arrays.stream(singleValueSample).flatMap(point -> LongStream.of(point, point + 1)).toArray() + final double fromIncrementedSampleMAD = calculateMAD( + Arrays.stream(singleValueSample).flatMap(point -> LongStream.of(point, point + 1)).toArray() + ); + assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); + } ); - assertThat(mad.getMedianAbsoluteDeviation(), closeToRelative(fromIncrementedSampleMAD)); } public void testAsSubAggregation() throws Exception { final int rangeBoundary = (MAX_SAMPLE_VALUE + MIN_SAMPLE_VALUE) / 2; - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - range("range").field("value") - .addRange(MIN_SAMPLE_VALUE, rangeBoundary) - .addRange(rangeBoundary, MAX_SAMPLE_VALUE) - .subAggregation(randomBuilder().field("value")) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final long[] lowerBucketSample = Arrays.stream(singleValueSample) - .filter(point -> point >= MIN_SAMPLE_VALUE && point < rangeBoundary) - .toArray(); - final long[] upperBucketSample = Arrays.stream(singleValueSample) - .filter(point -> point >= rangeBoundary && point < MAX_SAMPLE_VALUE) - .toArray(); - - final Range range = response.getAggregations().get("range"); - assertThat(range, notNullValue()); - List buckets = range.getBuckets(); - assertThat(buckets, notNullValue()); - assertThat(buckets, hasSize(2)); - - final Range.Bucket lowerBucket = buckets.get(0); - assertThat(lowerBucket, notNullValue()); - - final MedianAbsoluteDeviation lowerBucketMAD = lowerBucket.getAggregations().get("mad"); - assertThat(lowerBucketMAD, notNullValue()); - assertThat(lowerBucketMAD.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(lowerBucketSample))); - - final Range.Bucket upperBucket = buckets.get(1); - assertThat(upperBucket, notNullValue()); - - final MedianAbsoluteDeviation upperBucketMAD = upperBucket.getAggregations().get("mad"); - assertThat(upperBucketMAD, notNullValue()); - assertThat(upperBucketMAD.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(upperBucketSample))); - + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + range("range").field("value") + .addRange(MIN_SAMPLE_VALUE, rangeBoundary) + .addRange(rangeBoundary, MAX_SAMPLE_VALUE) + .subAggregation(randomBuilder().field("value")) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final long[] lowerBucketSample = Arrays.stream(singleValueSample) + .filter(point -> point >= MIN_SAMPLE_VALUE && point < rangeBoundary) + .toArray(); + final long[] upperBucketSample = Arrays.stream(singleValueSample) + .filter(point -> point >= rangeBoundary && point < MAX_SAMPLE_VALUE) + .toArray(); + + final Range range = response.getAggregations().get("range"); + assertThat(range, notNullValue()); + List buckets = range.getBuckets(); + assertThat(buckets, notNullValue()); + assertThat(buckets, hasSize(2)); + + final Range.Bucket lowerBucket = buckets.get(0); + assertThat(lowerBucket, notNullValue()); + + final MedianAbsoluteDeviation lowerBucketMAD = lowerBucket.getAggregations().get("mad"); + assertThat(lowerBucketMAD, notNullValue()); + assertThat(lowerBucketMAD.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(lowerBucketSample))); + + final Range.Bucket upperBucket = buckets.get(1); + assertThat(upperBucket, notNullValue()); + + final MedianAbsoluteDeviation upperBucketMAD = upperBucket.getAggregations().get("mad"); + assertThat(upperBucketMAD, notNullValue()); + assertThat(upperBucketMAD.getMedianAbsoluteDeviation(), closeToRelative(calculateMAD(upperBucketSample))); + } + ); } @Override public void testOrderByEmptyAggregation() throws Exception { final int numberOfBuckets = 10; - final SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - terms("terms").field("value") - .size(numberOfBuckets) - .order(BucketOrder.compound(BucketOrder.aggregation("filter>mad", true))) - .subAggregation( - filter("filter", termQuery("value", MAX_SAMPLE_VALUE + 1)).subAggregation(randomBuilder().field("value")) - ) - ) - .get(); - - assertHitCount(response, NUMBER_OF_DOCS); - - final Terms terms = response.getAggregations().get("terms"); - assertThat(terms, notNullValue()); - List buckets = terms.getBuckets(); - assertThat(buckets, notNullValue()); - assertThat(buckets, hasSize(numberOfBuckets)); - - for (int i = 0; i < numberOfBuckets; i++) { - Terms.Bucket bucket = buckets.get(i); - assertThat(bucket, notNullValue()); - - Filter filter = bucket.getAggregations().get("filter"); - assertThat(filter, notNullValue()); - assertThat(filter.getDocCount(), equalTo(0L)); - - MedianAbsoluteDeviation mad = filter.getAggregations().get("mad"); - assertThat(mad, notNullValue()); - assertThat(mad.getMedianAbsoluteDeviation(), equalTo(Double.NaN)); - } + assertResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + terms("terms").field("value") + .size(numberOfBuckets) + .order(BucketOrder.compound(BucketOrder.aggregation("filter>mad", true))) + .subAggregation( + filter("filter", termQuery("value", MAX_SAMPLE_VALUE + 1)).subAggregation(randomBuilder().field("value")) + ) + ), + response -> { + assertHitCount(response, NUMBER_OF_DOCS); + + final Terms terms = response.getAggregations().get("terms"); + assertThat(terms, notNullValue()); + List buckets = terms.getBuckets(); + assertThat(buckets, notNullValue()); + assertThat(buckets, hasSize(numberOfBuckets)); + + for (int i = 0; i < numberOfBuckets; i++) { + Terms.Bucket bucket = buckets.get(i); + assertThat(bucket, notNullValue()); + + Filter filter = bucket.getAggregations().get("filter"); + assertThat(filter, notNullValue()); + assertThat(filter.getDocCount(), equalTo(0L)); + + MedianAbsoluteDeviation mad = filter.getAggregations().get("mad"); + assertThat(mad, notNullValue()); + assertThat(mad.getMedianAbsoluteDeviation(), equalTo(Double.NaN)); + } + } + ); } /** @@ -493,13 +517,13 @@ public void testScriptCaching() throws Exception { ); // Test that a request using a nondeterministic script does not get cached - SearchResponse r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - randomBuilder().field("d") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "Math.random()", emptyMap())) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + randomBuilder().field("d") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "Math.random()", emptyMap())) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -511,13 +535,13 @@ public void testScriptCaching() throws Exception { ); // Test that a request using a deterministic script gets cached - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - randomBuilder().field("d") - .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + randomBuilder().field("d") + .script(new Script(ScriptType.INLINE, AggregationTestScriptsPlugin.NAME, "_value - 1", emptyMap())) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -529,8 +553,7 @@ public void testScriptCaching() throws Exception { ); // Ensure that non-scripted requests are cached as normal - r = prepareSearch("cache_test_idx").setSize(0).addAggregation(randomBuilder().field("d")).get(); - assertNoFailures(r); + assertNoFailures(prepareSearch("cache_test_idx").setSize(0).addAggregation(randomBuilder().field("d"))); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricIT.java index 2ea09960071f9..a6876f606ffee 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricIT.java @@ -11,7 +11,6 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequestBuilder; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.support.XContentMapValues; @@ -50,6 +49,7 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.scriptedMetric; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailuresAndResponse; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; @@ -359,37 +359,39 @@ public void testMap() { Script combineScript = new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op aggregation", Collections.emptyMap()); Script reduceScript = new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op list aggregation", Collections.emptyMap()); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation(scriptedMetric("scripted").mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript)) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); - int numShardsRun = 0; - for (Object object : aggregationList) { - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Map.class)); - Map map = (Map) object; - assertThat(map.size(), lessThanOrEqualTo(1)); - if (map.size() == 1) { - assertThat(map.get("count"), notNullValue()); - assertThat(map.get("count"), instanceOf(Number.class)); - assertThat(map.get("count"), equalTo(1)); - numShardsRun++; + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation(scriptedMetric("scripted").mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript)), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); + int numShardsRun = 0; + for (Object object : aggregationList) { + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Map.class)); + Map map = (Map) object; + assertThat(map.size(), lessThanOrEqualTo(1)); + if (map.size() == 1) { + assertThat(map.get("count"), notNullValue()); + assertThat(map.get("count"), instanceOf(Number.class)); + assertThat(map.get("count"), equalTo(1)); + numShardsRun++; + } + } + // We don't know how many shards will have documents but we need to make + // sure that at least one shard ran the map script + assertThat(numShardsRun, greaterThan(0)); } - } - // We don't know how many shards will have documents but we need to make - // sure that at least one shard ran the map script - assertThat(numShardsRun, greaterThan(0)); + ); } public void testMapWithParams() { @@ -401,45 +403,47 @@ public void testMapWithParams() { Script combineScript = new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op aggregation", Collections.emptyMap()); Script reduceScript = new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op list aggregation", Collections.emptyMap()); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(aggregationParams) - .mapScript(mapScript) - .combineScript(combineScript) - .reduceScript(reduceScript) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); - int numShardsRun = 0; - for (Object object : aggregationList) { - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Map.class)); - Map map = (Map) object; - for (Map.Entry entry : map.entrySet()) { - assertThat(entry, notNullValue()); - assertThat(entry.getKey(), notNullValue()); - assertThat(entry.getKey(), instanceOf(String.class)); - assertThat(entry.getValue(), notNullValue()); - assertThat(entry.getValue(), instanceOf(Number.class)); - String stringValue = (String) entry.getKey(); - assertThat(stringValue, equalTo("12")); - Number numberValue = (Number) entry.getValue(); - assertThat(numberValue, equalTo(1)); - numShardsRun++; + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(aggregationParams) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); + int numShardsRun = 0; + for (Object object : aggregationList) { + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Map.class)); + Map map = (Map) object; + for (Map.Entry entry : map.entrySet()) { + assertThat(entry, notNullValue()); + assertThat(entry.getKey(), notNullValue()); + assertThat(entry.getKey(), instanceOf(String.class)); + assertThat(entry.getValue(), notNullValue()); + assertThat(entry.getValue(), instanceOf(Number.class)); + String stringValue = (String) entry.getKey(); + assertThat(stringValue, equalTo("12")); + Number numberValue = (Number) entry.getValue(); + assertThat(numberValue, equalTo(1)); + numShardsRun++; + } + } + assertThat(numShardsRun, greaterThan(0)); } - } - assertThat(numShardsRun, greaterThan(0)); + ); } public void testInitMutatesParams() { @@ -449,47 +453,56 @@ public void testInitMutatesParams() { Map params = new HashMap<>(); params.put("vars", varsMap); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params) - .initScript(new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "vars.multiplier = 3", Collections.emptyMap())) - .mapScript( - new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "state.list.add(vars.multiplier)", Collections.emptyMap()) - ) - .combineScript(new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op aggregation", Collections.emptyMap())) - .reduceScript(new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op list aggregation", Collections.emptyMap())) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); - long totalCount = 0; - for (Object object : aggregationList) { - assertThat(object, notNullValue()); - assertThat(object, instanceOf(HashMap.class)); - @SuppressWarnings("unchecked") - Map map = (Map) object; - assertThat(map, hasKey("list")); - assertThat(map.get("list"), instanceOf(List.class)); - List list = (List) map.get("list"); - for (Object o : list) { - assertThat(o, notNullValue()); - assertThat(o, instanceOf(Number.class)); - Number numberValue = (Number) o; - assertThat(numberValue, equalTo(3)); - totalCount += numberValue.longValue(); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params) + .initScript(new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "vars.multiplier = 3", Collections.emptyMap())) + .mapScript( + new Script( + ScriptType.INLINE, + CustomScriptPlugin.NAME, + "state.list.add(vars.multiplier)", + Collections.emptyMap() + ) + ) + .combineScript(new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op aggregation", Collections.emptyMap())) + .reduceScript( + new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op list aggregation", Collections.emptyMap()) + ) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); + long totalCount = 0; + for (Object object : aggregationList) { + assertThat(object, notNullValue()); + assertThat(object, instanceOf(HashMap.class)); + @SuppressWarnings("unchecked") + Map map = (Map) object; + assertThat(map, hasKey("list")); + assertThat(map.get("list"), instanceOf(List.class)); + List list = (List) map.get("list"); + for (Object o : list) { + assertThat(o, notNullValue()); + assertThat(o, instanceOf(Number.class)); + Number numberValue = (Number) o; + assertThat(numberValue, equalTo(3)); + totalCount += numberValue.longValue(); + } + } + assertThat(totalCount, equalTo(numDocs * 3)); } - } - assertThat(totalCount, equalTo(numDocs * 3)); + ); } public void testMapCombineWithParams() { @@ -508,40 +521,42 @@ public void testMapCombineWithParams() { ); Script reduceScript = new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op list aggregation", Collections.emptyMap()); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params).mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); - long totalCount = 0; - for (Object object : aggregationList) { - assertThat(object, notNullValue()); - assertThat(object, instanceOf(List.class)); - List list = (List) object; - for (Object o : list) { - assertThat(o, notNullValue()); - assertThat(o, instanceOf(Number.class)); - Number numberValue = (Number) o; - // A particular shard may not have any documents stored on it so - // we have to assume the lower bound may be 0. The check at the - // bottom of the test method will make sure the count is correct - assertThat(numberValue.longValue(), allOf(greaterThanOrEqualTo(0L), lessThanOrEqualTo(numDocs))); - totalCount += numberValue.longValue(); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params).mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); + long totalCount = 0; + for (Object object : aggregationList) { + assertThat(object, notNullValue()); + assertThat(object, instanceOf(List.class)); + List list = (List) object; + for (Object o : list) { + assertThat(o, notNullValue()); + assertThat(o, instanceOf(Number.class)); + Number numberValue = (Number) o; + // A particular shard may not have any documents stored on it so + // we have to assume the lower bound may be 0. The check at the + // bottom of the test method will make sure the count is correct + assertThat(numberValue.longValue(), allOf(greaterThanOrEqualTo(0L), lessThanOrEqualTo(numDocs))); + totalCount += numberValue.longValue(); + } + } + assertThat(totalCount, equalTo(numDocs)); } - } - assertThat(totalCount, equalTo(numDocs)); + ); } public void testInitMapCombineWithParams() { @@ -566,44 +581,46 @@ public void testInitMapCombineWithParams() { ); Script reduceScript = new Script(ScriptType.INLINE, CustomScriptPlugin.NAME, "no-op list aggregation", Collections.emptyMap()); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params) - .initScript(initScript) - .mapScript(mapScript) - .combineScript(combineScript) - .reduceScript(reduceScript) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); - long totalCount = 0; - for (Object object : aggregationList) { - assertThat(object, notNullValue()); - assertThat(object, instanceOf(List.class)); - List list = (List) object; - for (Object o : list) { - assertThat(o, notNullValue()); - assertThat(o, instanceOf(Number.class)); - Number numberValue = (Number) o; - // A particular shard may not have any documents stored on it so - // we have to assume the lower bound may be 0. The check at the - // bottom of the test method will make sure the count is correct - assertThat(numberValue.longValue(), allOf(greaterThanOrEqualTo(0L), lessThanOrEqualTo(numDocs * 3))); - totalCount += numberValue.longValue(); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params) + .initScript(initScript) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(getNumShards("idx").numPrimaries)); + long totalCount = 0; + for (Object object : aggregationList) { + assertThat(object, notNullValue()); + assertThat(object, instanceOf(List.class)); + List list = (List) object; + for (Object o : list) { + assertThat(o, notNullValue()); + assertThat(o, instanceOf(Number.class)); + Number numberValue = (Number) o; + // A particular shard may not have any documents stored on it so + // we have to assume the lower bound may be 0. The check at the + // bottom of the test method will make sure the count is correct + assertThat(numberValue.longValue(), allOf(greaterThanOrEqualTo(0L), lessThanOrEqualTo(numDocs * 3))); + totalCount += numberValue.longValue(); + } + } + assertThat(totalCount, equalTo(numDocs * 3)); } - } - assertThat(totalCount, equalTo(numDocs * 3)); + ); } public void testInitMapCombineReduceWithParams() { @@ -633,31 +650,33 @@ public void testInitMapCombineReduceWithParams() { Collections.emptyMap() ); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params) - .initScript(initScript) - .mapScript(mapScript) - .combineScript(combineScript) - .reduceScript(reduceScript) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(1)); - Object object = aggregationList.get(0); - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Number.class)); - assertThat(((Number) object).longValue(), equalTo(numDocs * 3)); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params) + .initScript(initScript) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(1)); + Object object = aggregationList.get(0); + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Number.class)); + assertThat(((Number) object).longValue(), equalTo(numDocs * 3)); + } + ); } @SuppressWarnings("rawtypes") @@ -688,42 +707,43 @@ public void testInitMapCombineReduceGetProperty() throws Exception { Collections.emptyMap() ); - SearchResponse searchResponse = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - global("global").subAggregation( - scriptedMetric("scripted").params(params) - .initScript(initScript) - .mapScript(mapScript) - .combineScript(combineScript) - .reduceScript(reduceScript) - ) - ) - .get(); - - assertNoFailures(searchResponse); - assertThat(searchResponse.getHits().getTotalHits().value, equalTo(numDocs)); - - Global global = searchResponse.getAggregations().get("global"); - assertThat(global, notNullValue()); - assertThat(global.getName(), equalTo("global")); - assertThat(global.getDocCount(), equalTo(numDocs)); - assertThat(global.getAggregations(), notNullValue()); - assertThat(global.getAggregations().asMap().size(), equalTo(1)); - - ScriptedMetric scriptedMetricAggregation = global.getAggregations().get("scripted"); - assertThat(scriptedMetricAggregation, notNullValue()); - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(1)); - Object object = aggregationList.get(0); - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Number.class)); - assertThat(((Number) object).longValue(), equalTo(numDocs * 3)); - assertThat(((InternalAggregation) global).getProperty("scripted"), sameInstance(scriptedMetricAggregation)); - assertThat((List) ((InternalAggregation) global).getProperty("scripted.value"), sameInstance(aggregationList)); - assertThat((List) ((InternalAggregation) scriptedMetricAggregation).getProperty("value"), sameInstance(aggregationList)); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + global("global").subAggregation( + scriptedMetric("scripted").params(params) + .initScript(initScript) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Global global = response.getAggregations().get("global"); + assertThat(global, notNullValue()); + assertThat(global.getName(), equalTo("global")); + assertThat(global.getDocCount(), equalTo(numDocs)); + assertThat(global.getAggregations(), notNullValue()); + assertThat(global.getAggregations().asMap().size(), equalTo(1)); + + ScriptedMetric scriptedMetricAggregation = global.getAggregations().get("scripted"); + assertThat(scriptedMetricAggregation, notNullValue()); + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(1)); + Object object = aggregationList.get(0); + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Number.class)); + assertThat(((Number) object).longValue(), equalTo(numDocs * 3)); + assertThat(((InternalAggregation) global).getProperty("scripted"), sameInstance(scriptedMetricAggregation)); + assertThat((List) ((InternalAggregation) global).getProperty("scripted.value"), sameInstance(aggregationList)); + assertThat((List) ((InternalAggregation) scriptedMetricAggregation).getProperty("value"), sameInstance(aggregationList)); + } + ); } public void testMapCombineReduceWithParams() { @@ -752,27 +772,29 @@ public void testMapCombineReduceWithParams() { Collections.emptyMap() ); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params).mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(1)); - Object object = aggregationList.get(0); - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Number.class)); - assertThat(((Number) object).longValue(), equalTo(numDocs)); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params).mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(1)); + Object object = aggregationList.get(0); + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Number.class)); + assertThat(((Number) object).longValue(), equalTo(numDocs)); + } + ); } public void testInitMapReduceWithParams() { @@ -797,31 +819,33 @@ public void testInitMapReduceWithParams() { Collections.emptyMap() ); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params) - .initScript(initScript) - .mapScript(mapScript) - .combineScript(combineScript) - .reduceScript(reduceScript) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(1)); - Object object = aggregationList.get(0); - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Number.class)); - assertThat(((Number) object).longValue(), equalTo(numDocs * 3)); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params) + .initScript(initScript) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(1)); + Object object = aggregationList.get(0); + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Number.class)); + assertThat(((Number) object).longValue(), equalTo(numDocs * 3)); + } + ); } public void testMapReduceWithParams() { @@ -844,27 +868,29 @@ public void testMapReduceWithParams() { Collections.emptyMap() ); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params).mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(1)); - Object object = aggregationList.get(0); - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Number.class)); - assertThat(((Number) object).longValue(), equalTo(numDocs)); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params).mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(1)); + Object object = aggregationList.get(0); + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Number.class)); + assertThat(((Number) object).longValue(), equalTo(numDocs)); + } + ); } public void testInitMapCombineReduceWithParamsAndReduceParams() { @@ -897,31 +923,33 @@ public void testInitMapCombineReduceWithParamsAndReduceParams() { reduceParams ); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params) - .initScript(initScript) - .mapScript(mapScript) - .combineScript(combineScript) - .reduceScript(reduceScript) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(1)); - Object object = aggregationList.get(0); - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Number.class)); - assertThat(((Number) object).longValue(), equalTo(numDocs * 12)); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params) + .initScript(initScript) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(1)); + Object object = aggregationList.get(0); + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Number.class)); + assertThat(((Number) object).longValue(), equalTo(numDocs * 12)); + } + ); } public void testInitMapCombineReduceWithParamsStored() { @@ -931,31 +959,33 @@ public void testInitMapCombineReduceWithParamsStored() { Map params = new HashMap<>(); params.put("vars", varsMap); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .addAggregation( - scriptedMetric("scripted").params(params) - .initScript(new Script(ScriptType.STORED, null, "initScript_stored", Collections.emptyMap())) - .mapScript(new Script(ScriptType.STORED, null, "mapScript_stored", Collections.emptyMap())) - .combineScript(new Script(ScriptType.STORED, null, "combineScript_stored", Collections.emptyMap())) - .reduceScript(new Script(ScriptType.STORED, null, "reduceScript_stored", Collections.emptyMap())) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - - Aggregation aggregation = response.getAggregations().get("scripted"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(1)); - Object object = aggregationList.get(0); - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Number.class)); - assertThat(((Number) object).longValue(), equalTo(numDocs * 3)); + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .addAggregation( + scriptedMetric("scripted").params(params) + .initScript(new Script(ScriptType.STORED, null, "initScript_stored", Collections.emptyMap())) + .mapScript(new Script(ScriptType.STORED, null, "mapScript_stored", Collections.emptyMap())) + .combineScript(new Script(ScriptType.STORED, null, "combineScript_stored", Collections.emptyMap())) + .reduceScript(new Script(ScriptType.STORED, null, "reduceScript_stored", Collections.emptyMap())) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + + Aggregation aggregation = response.getAggregations().get("scripted"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) aggregation; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(1)); + Object object = aggregationList.get(0); + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Number.class)); + assertThat(((Number) object).longValue(), equalTo(numDocs * 3)); + } + ); } public void testInitMapCombineReduceWithParamsAsSubAgg() { @@ -985,49 +1015,51 @@ public void testInitMapCombineReduceWithParamsAsSubAgg() { Collections.emptyMap() ); - SearchResponse response = prepareSearch("idx").setQuery(matchAllQuery()) - .setSize(1000) - .addAggregation( - histogram("histo").field("l_value") - .interval(1) - .subAggregation( - scriptedMetric("scripted").params(params) - .initScript(initScript) - .mapScript(mapScript) - .combineScript(combineScript) - .reduceScript(reduceScript) - ) - ) - .get(); - assertNoFailures(response); - assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); - Aggregation aggregation = response.getAggregations().get("histo"); - assertThat(aggregation, notNullValue()); - assertThat(aggregation, instanceOf(Histogram.class)); - Histogram histoAgg = (Histogram) aggregation; - assertThat(histoAgg.getName(), equalTo("histo")); - List buckets = histoAgg.getBuckets(); - assertThat(buckets, notNullValue()); - for (Bucket b : buckets) { - assertThat(b, notNullValue()); - assertThat(b.getDocCount(), equalTo(1L)); - Aggregations subAggs = b.getAggregations(); - assertThat(subAggs, notNullValue()); - assertThat(subAggs.asList().size(), equalTo(1)); - Aggregation subAgg = subAggs.get("scripted"); - assertThat(subAgg, notNullValue()); - assertThat(subAgg, instanceOf(ScriptedMetric.class)); - ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) subAgg; - assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); - assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); - assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); - List aggregationList = (List) scriptedMetricAggregation.aggregation(); - assertThat(aggregationList.size(), equalTo(1)); - Object object = aggregationList.get(0); - assertThat(object, notNullValue()); - assertThat(object, instanceOf(Number.class)); - assertThat(((Number) object).longValue(), equalTo(3L)); - } + assertNoFailuresAndResponse( + prepareSearch("idx").setQuery(matchAllQuery()) + .setSize(1000) + .addAggregation( + histogram("histo").field("l_value") + .interval(1) + .subAggregation( + scriptedMetric("scripted").params(params) + .initScript(initScript) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(numDocs)); + Aggregation aggregation = response.getAggregations().get("histo"); + assertThat(aggregation, notNullValue()); + assertThat(aggregation, instanceOf(Histogram.class)); + Histogram histoAgg = (Histogram) aggregation; + assertThat(histoAgg.getName(), equalTo("histo")); + List buckets = histoAgg.getBuckets(); + assertThat(buckets, notNullValue()); + for (Bucket b : buckets) { + assertThat(b, notNullValue()); + assertThat(b.getDocCount(), equalTo(1L)); + Aggregations subAggs = b.getAggregations(); + assertThat(subAggs, notNullValue()); + assertThat(subAggs.asList().size(), equalTo(1)); + Aggregation subAgg = subAggs.get("scripted"); + assertThat(subAgg, notNullValue()); + assertThat(subAgg, instanceOf(ScriptedMetric.class)); + ScriptedMetric scriptedMetricAggregation = (ScriptedMetric) subAgg; + assertThat(scriptedMetricAggregation.getName(), equalTo("scripted")); + assertThat(scriptedMetricAggregation.aggregation(), notNullValue()); + assertThat(scriptedMetricAggregation.aggregation(), instanceOf(ArrayList.class)); + List aggregationList = (List) scriptedMetricAggregation.aggregation(); + assertThat(aggregationList.size(), equalTo(1)); + Object object = aggregationList.get(0); + assertThat(object, notNullValue()); + assertThat(object, instanceOf(Number.class)); + assertThat(((Number) object).longValue(), equalTo(3L)); + } + } + ); } public void testEmptyAggregation() throws Exception { @@ -1057,36 +1089,38 @@ public void testEmptyAggregation() throws Exception { Collections.emptyMap() ); - SearchResponse searchResponse = prepareSearch("empty_bucket_idx").setQuery(matchAllQuery()) - .addAggregation( - histogram("histo").field("value") - .interval(1L) - .minDocCount(0) - .subAggregation( - scriptedMetric("scripted").params(params) - .initScript(initScript) - .mapScript(mapScript) - .combineScript(combineScript) - .reduceScript(reduceScript) - ) - ) - .get(); - - assertThat(searchResponse.getHits().getTotalHits().value, equalTo(2L)); - Histogram histo = searchResponse.getAggregations().get("histo"); - assertThat(histo, notNullValue()); - Histogram.Bucket bucket = histo.getBuckets().get(1); - assertThat(bucket, notNullValue()); - - ScriptedMetric scriptedMetric = bucket.getAggregations().get("scripted"); - assertThat(scriptedMetric, notNullValue()); - assertThat(scriptedMetric.getName(), equalTo("scripted")); - assertThat(scriptedMetric.aggregation(), notNullValue()); - assertThat(scriptedMetric.aggregation(), instanceOf(List.class)); - @SuppressWarnings("unchecked") // We'll just get a ClassCastException a couple lines down if we're wrong, its ok. - List aggregationResult = (List) scriptedMetric.aggregation(); - assertThat(aggregationResult.size(), equalTo(1)); - assertThat(aggregationResult.get(0), equalTo(0)); + assertNoFailuresAndResponse( + prepareSearch("empty_bucket_idx").setQuery(matchAllQuery()) + .addAggregation( + histogram("histo").field("value") + .interval(1L) + .minDocCount(0) + .subAggregation( + scriptedMetric("scripted").params(params) + .initScript(initScript) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ) + ), + response -> { + assertThat(response.getHits().getTotalHits().value, equalTo(2L)); + Histogram histo = response.getAggregations().get("histo"); + assertThat(histo, notNullValue()); + Histogram.Bucket bucket = histo.getBuckets().get(1); + assertThat(bucket, notNullValue()); + + ScriptedMetric scriptedMetric = bucket.getAggregations().get("scripted"); + assertThat(scriptedMetric, notNullValue()); + assertThat(scriptedMetric.getName(), equalTo("scripted")); + assertThat(scriptedMetric.aggregation(), notNullValue()); + assertThat(scriptedMetric.aggregation(), instanceOf(List.class)); + @SuppressWarnings("unchecked") // We'll just get a ClassCastException a couple lines down if we're wrong, its ok. + List aggregationResult = (List) scriptedMetric.aggregation(); + assertThat(aggregationResult.size(), equalTo(1)); + assertThat(aggregationResult.get(0), equalTo(0)); + } + ); } /** @@ -1129,12 +1163,15 @@ public void testScriptCaching() throws Exception { ); // Test that a non-deterministic init script causes the result to not be cached - SearchResponse r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation( - scriptedMetric("foo").initScript(ndInitScript).mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript) - ) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation( + scriptedMetric("foo").initScript(ndInitScript) + .mapScript(mapScript) + .combineScript(combineScript) + .reduceScript(reduceScript) + ) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -1146,10 +1183,10 @@ public void testScriptCaching() throws Exception { ); // Test that a non-deterministic map script causes the result to not be cached - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation(scriptedMetric("foo").mapScript(ndMapScript).combineScript(combineScript).reduceScript(reduceScript)) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation(scriptedMetric("foo").mapScript(ndMapScript).combineScript(combineScript).reduceScript(reduceScript)) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -1161,10 +1198,10 @@ public void testScriptCaching() throws Exception { ); // Test that a non-deterministic combine script causes the result to not be cached - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation(scriptedMetric("foo").mapScript(mapScript).combineScript(ndRandom).reduceScript(reduceScript)) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation(scriptedMetric("foo").mapScript(mapScript).combineScript(ndRandom).reduceScript(reduceScript)) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -1176,10 +1213,10 @@ public void testScriptCaching() throws Exception { ); // NOTE: random reduce scripts don't hit the query shard context (they are done on the coordinator) and so can be cached. - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation(scriptedMetric("foo").mapScript(mapScript).combineScript(combineScript).reduceScript(ndRandom)) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation(scriptedMetric("foo").mapScript(mapScript).combineScript(combineScript).reduceScript(ndRandom)) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), @@ -1191,10 +1228,10 @@ public void testScriptCaching() throws Exception { ); // Test that all deterministic scripts cause the request to be cached - r = prepareSearch("cache_test_idx").setSize(0) - .addAggregation(scriptedMetric("foo").mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript)) - .get(); - assertNoFailures(r); + assertNoFailures( + prepareSearch("cache_test_idx").setSize(0) + .addAggregation(scriptedMetric("foo").mapScript(mapScript).combineScript(combineScript).reduceScript(reduceScript)) + ); assertThat( indicesAdmin().prepareStats("cache_test_idx").setRequestCache(true).get().getTotal().getRequestCache().getHitCount(), From b7b5518acc92d55b609d435479564b8d8fcd9af5 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Thu, 9 Nov 2023 22:08:18 +0100 Subject: [PATCH 041/513] Remove explicit SearchResponse references from plugins (#101277) Remove explicit SearchResponse references from plugins. --- .../ICUCollationKeywordFieldMapperIT.java | 101 ++++++++++-------- .../index/mapper/size/SizeMappingIT.java | 59 ++++++---- .../store/smb/AbstractAzureFsTestCase.java | 6 +- 3 files changed, 98 insertions(+), 68 deletions(-) diff --git a/plugins/analysis-icu/src/internalClusterTest/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperIT.java b/plugins/analysis-icu/src/internalClusterTest/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperIT.java index e6f91efad0162..ae6a0cc71789f 100644 --- a/plugins/analysis-icu/src/internalClusterTest/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperIT.java +++ b/plugins/analysis-icu/src/internalClusterTest/java/org/elasticsearch/index/mapper/ICUCollationKeywordFieldMapperIT.java @@ -12,7 +12,6 @@ import com.ibm.icu.util.ULocale; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.plugin.analysis.icu.AnalysisICUPlugin; import org.elasticsearch.plugins.Plugin; @@ -31,6 +30,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertOrderedSearchHits; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; public class ICUCollationKeywordFieldMapperIT extends ESIntegTestCase { @@ -82,10 +82,11 @@ public void testBasicUsage() throws Exception { .sort("id", SortOrder.DESC) // secondary sort should kick in because both will collate to same value ); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "2", "1"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "2", "1"); + }); } public void testMultipleValues() throws Exception { @@ -126,10 +127,11 @@ public void testMultipleValues() throws Exception { .sort("id", SortOrder.DESC) // will be ignored ); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "1", "2"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "1", "2"); + }); // same thing, using different sort mode that will use a for both docs request = new SearchRequest().indices(index) @@ -141,10 +143,11 @@ public void testMultipleValues() throws Exception { .sort("id", SortOrder.DESC) // will NOT be ignored and will determine order ); - response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "2", "1"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "2", "1"); + }); } /* @@ -186,10 +189,11 @@ public void testNormalization() throws Exception { .sort("id", SortOrder.DESC) // secondary sort should kick in because both will collate to same value ); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "2", "1"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "2", "1"); + }); } /* @@ -230,10 +234,11 @@ public void testSecondaryStrength() throws Exception { .sort("id", SortOrder.DESC) // secondary sort should kick in because both will collate to same value ); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "2", "1"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "2", "1"); + }); } /* @@ -275,10 +280,11 @@ public void testIgnorePunctuation() throws Exception { .sort("id", SortOrder.DESC) // secondary sort should kick in because both will collate to same value ); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "2", "1"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "2", "1"); + }); } /* @@ -321,10 +327,11 @@ public void testIgnoreWhitespace() throws Exception { .sort("id", SortOrder.ASC) ); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 3L); - assertOrderedSearchHits(response, "3", "1", "2"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 3L); + assertOrderedSearchHits(response, "3", "1", "2"); + }); } /* @@ -354,10 +361,11 @@ public void testNumerics() throws Exception { SearchRequest request = new SearchRequest().indices(index) .source(new SearchSourceBuilder().fetchSource(false).sort("collate", SortOrder.ASC)); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "2", "1"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "2", "1"); + }); } /* @@ -393,10 +401,11 @@ public void testIgnoreAccentsButNotCase() throws Exception { SearchRequest request = new SearchRequest().indices(index) .source(new SearchSourceBuilder().fetchSource(false).sort("collate", SortOrder.ASC).sort("id", SortOrder.DESC)); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 4L); - assertOrderedSearchHits(response, "3", "1", "4", "2"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 4L); + assertOrderedSearchHits(response, "3", "1", "4", "2"); + }); } /* @@ -429,10 +438,11 @@ public void testUpperCaseFirst() throws Exception { SearchRequest request = new SearchRequest().indices(index) .source(new SearchSourceBuilder().fetchSource(false).sort("collate", SortOrder.ASC)); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "2", "1"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "2", "1"); + }); } /* @@ -482,9 +492,10 @@ public void testCustomRules() throws Exception { .sort("id", SortOrder.DESC) // secondary sort should kick in because both will collate to same value ); - SearchResponse response = client().search(request).actionGet(); - assertNoFailures(response); - assertHitCount(response, 2L); - assertOrderedSearchHits(response, "2", "1"); + assertResponse(client().search(request), response -> { + assertNoFailures(response); + assertHitCount(response, 2L); + assertOrderedSearchHits(response, "2", "1"); + }); } } diff --git a/plugins/mapper-size/src/internalClusterTest/java/org/elasticsearch/index/mapper/size/SizeMappingIT.java b/plugins/mapper-size/src/internalClusterTest/java/org/elasticsearch/index/mapper/size/SizeMappingIT.java index e92c7ca4bdebb..026dabd64eb0b 100644 --- a/plugins/mapper-size/src/internalClusterTest/java/org/elasticsearch/index/mapper/size/SizeMappingIT.java +++ b/plugins/mapper-size/src/internalClusterTest/java/org/elasticsearch/index/mapper/size/SizeMappingIT.java @@ -9,7 +9,6 @@ import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.plugin.mapper.MapperSizePlugin; import org.elasticsearch.plugins.Plugin; @@ -24,6 +23,7 @@ import java.util.Map; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.hasKey; import static org.hamcrest.Matchers.is; @@ -110,43 +110,64 @@ public void testGetWithFields() throws Exception { assertAcked(prepareCreate("test").setMapping("_size", "enabled=true")); final String source = "{\"f\":\"" + randomAlphaOfLengthBetween(1, 100) + "\"}"; indexRandom(true, client().prepareIndex("test").setId("1").setSource(source, XContentType.JSON)); - SearchResponse searchResponse = prepareSearch("test").addFetchField("_size").get(); - assertEquals(source.length(), ((Long) searchResponse.getHits().getHits()[0].getFields().get("_size").getValue()).intValue()); + assertResponse( + prepareSearch("test").addFetchField("_size"), + response -> assertEquals( + source.length(), + ((Long) response.getHits().getHits()[0].getFields().get("_size").getValue()).intValue() + ) + ); // this should not work when requesting fields via wildcard expression - searchResponse = prepareSearch("test").addFetchField("*").get(); - assertNull(searchResponse.getHits().getHits()[0].getFields().get("_size")); + assertResponse( + prepareSearch("test").addFetchField("*"), + response -> assertNull(response.getHits().getHits()[0].getFields().get("_size")) + ); // This should STILL work - searchResponse = prepareSearch("test").addStoredField("*").get(); - assertNotNull(searchResponse.getHits().getHits()[0].getFields().get("_size")); + assertResponse( + prepareSearch("test").addStoredField("*"), + response -> assertNotNull(response.getHits().getHits()[0].getFields().get("_size")) + ); } public void testWildCardWithFieldsWhenDisabled() throws Exception { assertAcked(prepareCreate("test").setMapping("_size", "enabled=false")); final String source = "{\"f\":\"" + randomAlphaOfLengthBetween(1, 100) + "\"}"; indexRandom(true, client().prepareIndex("test").setId("1").setSource(source, XContentType.JSON)); - SearchResponse searchResponse = prepareSearch("test").addFetchField("_size").get(); - assertNull(searchResponse.getHits().getHits()[0].getFields().get("_size")); + assertResponse( + prepareSearch("test").addFetchField("_size"), + response -> assertNull(response.getHits().getHits()[0].getFields().get("_size")) + ); - searchResponse = prepareSearch("test").addFetchField("*").get(); - assertNull(searchResponse.getHits().getHits()[0].getFields().get("_size")); + assertResponse( + prepareSearch("test").addFetchField("*"), + response -> assertNull(response.getHits().getHits()[0].getFields().get("_size")) + ); - searchResponse = prepareSearch("test").addStoredField("*").get(); - assertNull(searchResponse.getHits().getHits()[0].getFields().get("_size")); + assertResponse( + prepareSearch("test").addStoredField("*"), + response -> assertNull(response.getHits().getHits()[0].getFields().get("_size")) + ); } public void testWildCardWithFieldsWhenNotProvided() throws Exception { assertAcked(prepareCreate("test")); final String source = "{\"f\":\"" + randomAlphaOfLengthBetween(1, 100) + "\"}"; indexRandom(true, client().prepareIndex("test").setId("1").setSource(source, XContentType.JSON)); - SearchResponse searchResponse = prepareSearch("test").addFetchField("_size").get(); - assertNull(searchResponse.getHits().getHits()[0].getFields().get("_size")); + assertResponse( + prepareSearch("test").addFetchField("_size"), + response -> assertNull(response.getHits().getHits()[0].getFields().get("_size")) + ); - searchResponse = prepareSearch("test").addFetchField("*").get(); - assertNull(searchResponse.getHits().getHits()[0].getFields().get("_size")); + assertResponse( + prepareSearch("test").addFetchField("*"), + response -> assertNull(response.getHits().getHits()[0].getFields().get("_size")) + ); - searchResponse = prepareSearch("test").addStoredField("*").get(); - assertNull(searchResponse.getHits().getHits()[0].getFields().get("_size")); + assertResponse( + prepareSearch("test").addStoredField("*"), + response -> assertNull(response.getHits().getHits()[0].getFields().get("_size")) + ); } } diff --git a/plugins/store-smb/src/internalClusterTest/java/org/elasticsearch/index/store/smb/AbstractAzureFsTestCase.java b/plugins/store-smb/src/internalClusterTest/java/org/elasticsearch/index/store/smb/AbstractAzureFsTestCase.java index 4d1f6426821c4..4a35779a42166 100644 --- a/plugins/store-smb/src/internalClusterTest/java/org/elasticsearch/index/store/smb/AbstractAzureFsTestCase.java +++ b/plugins/store-smb/src/internalClusterTest/java/org/elasticsearch/index/store/smb/AbstractAzureFsTestCase.java @@ -8,7 +8,6 @@ package org.elasticsearch.index.store.smb; -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.plugin.store.smb.SMBStorePlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -16,7 +15,7 @@ import java.util.Arrays; import java.util.Collection; -import static org.hamcrest.Matchers.is; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; public abstract class AbstractAzureFsTestCase extends ESIntegTestCase { @Override @@ -32,7 +31,6 @@ public void testAzureFs() { indexDoc("test", "" + i, "foo", "bar"); } refresh(); - SearchResponse response = prepareSearch("test").get(); - assertThat(response.getHits().getTotalHits().value, is(nbDocs)); + assertHitCount(prepareSearch("test"), nbDocs); } } From 8cc771eeb8492e232f404a3f9365190c2c68a548 Mon Sep 17 00:00:00 2001 From: Matt Culbreth Date: Thu, 9 Nov 2023 17:03:42 -0500 Subject: [PATCH 042/513] Add message field to HealthPeriodicLogger and S3RequestRetryStats (#101989) --- docs/changelog/101989.yaml | 5 +++++ .../repositories/s3/S3RequestRetryStats.java | 4 +++- .../org/elasticsearch/health/HealthPeriodicLogger.java | 2 ++ .../elasticsearch/health/HealthPeriodicLoggerTests.java | 9 ++++++++- 4 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 docs/changelog/101989.yaml diff --git a/docs/changelog/101989.yaml b/docs/changelog/101989.yaml new file mode 100644 index 0000000000000..d294d194bd4e8 --- /dev/null +++ b/docs/changelog/101989.yaml @@ -0,0 +1,5 @@ +pr: 101989 +summary: Add message field to `HealthPeriodicLogger` and `S3RequestRetryStats` +area: Health +type: enhancement +issues: [] diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RequestRetryStats.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RequestRetryStats.java index ae2441c2e705d..b7c37c6d95fde 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RequestRetryStats.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3RequestRetryStats.java @@ -24,6 +24,7 @@ * This class emit aws s3 metrics as logs until we have a proper apm integration */ public class S3RequestRetryStats { + public static final String MESSAGE_FIELD = "message"; private static final Logger logger = LogManager.getLogger(S3RequestRetryStats.class); @@ -65,7 +66,8 @@ private static long getCounter(TimingInfo info, AWSRequestMetrics.Field field) { public void emitMetrics() { if (logger.isDebugEnabled()) { - var metrics = Maps.newMapWithExpectedSize(3); + var metrics = Maps.newMapWithExpectedSize(4); + metrics.put(MESSAGE_FIELD, "S3 Request Retry Stats"); metrics.put("elasticsearch.metrics.s3.requests", requests.get()); metrics.put("elasticsearch.metrics.s3.exceptions", exceptions.get()); metrics.put("elasticsearch.metrics.s3.throttles", throttles.get()); diff --git a/server/src/main/java/org/elasticsearch/health/HealthPeriodicLogger.java b/server/src/main/java/org/elasticsearch/health/HealthPeriodicLogger.java index 4d50764aa0cc1..ba8a8458b08cc 100644 --- a/server/src/main/java/org/elasticsearch/health/HealthPeriodicLogger.java +++ b/server/src/main/java/org/elasticsearch/health/HealthPeriodicLogger.java @@ -41,6 +41,7 @@ */ public class HealthPeriodicLogger implements ClusterStateListener, Closeable, SchedulerEngine.Listener { public static final String HEALTH_FIELD_PREFIX = "elasticsearch.health"; + public static final String MESSAGE_FIELD = "message"; public static final Setting POLL_INTERVAL_SETTING = Setting.timeSetting( "health.periodic_logger.poll_interval", @@ -193,6 +194,7 @@ static Map convertToLoggedFields(List ind // overall status final HealthStatus status = HealthStatus.merge(indicatorResults.stream().map(HealthIndicatorResult::status)); result.put(String.format(Locale.ROOT, "%s.overall.status", HEALTH_FIELD_PREFIX), status.xContentValue()); + result.put(MESSAGE_FIELD, String.format(Locale.ROOT, "health=%s", status.xContentValue())); // top-level status for each indicator indicatorResults.forEach((indicatorResult) -> { diff --git a/server/src/test/java/org/elasticsearch/health/HealthPeriodicLoggerTests.java b/server/src/test/java/org/elasticsearch/health/HealthPeriodicLoggerTests.java index 7e77b3a4a1d73..ed18c1b0dc3d5 100644 --- a/server/src/test/java/org/elasticsearch/health/HealthPeriodicLoggerTests.java +++ b/server/src/test/java/org/elasticsearch/health/HealthPeriodicLoggerTests.java @@ -100,7 +100,8 @@ public void testConvertToLoggedFields() { Map loggerResults = HealthPeriodicLogger.convertToLoggedFields(results); - assertThat(loggerResults.size(), equalTo(results.size() + 1)); + // verify that the number of fields is the number of indicators + 2 (for overall and for message) + assertThat(loggerResults.size(), equalTo(results.size() + 2)); // test indicator status assertThat(loggerResults.get(makeHealthStatusString("network_latency")), equalTo("green")); @@ -110,6 +111,12 @@ public void testConvertToLoggedFields() { // test calculated overall status assertThat(loggerResults.get(makeHealthStatusString("overall")), equalTo(overallStatus.xContentValue())); + // test calculated message + assertThat( + loggerResults.get(HealthPeriodicLogger.MESSAGE_FIELD), + equalTo(String.format(Locale.ROOT, "health=%s", overallStatus.xContentValue())) + ); + // test empty results { List empty = new ArrayList<>(); From 21ee091454597a6658b05537727ae62c53c20572 Mon Sep 17 00:00:00 2001 From: Mark Vieira Date: Thu, 9 Nov 2023 14:05:24 -0800 Subject: [PATCH 043/513] Mute MlAssignmentPlannerUpgradeIT --- .../org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java index 22ff69bf5ff55..f1a72663aaf82 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/MlAssignmentPlannerUpgradeIT.java @@ -65,6 +65,7 @@ public class MlAssignmentPlannerUpgradeIT extends AbstractUpgradeTestCase { RAW_MODEL_SIZE = Base64.getDecoder().decode(BASE_64_ENCODED_MODEL).length; } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/101926") public void testMlAssignmentPlannerUpgrade() throws Exception { assumeTrue("NLP model deployments added in 8.0", isOriginalClusterVersionAtLeast(Version.V_8_0_0)); assumeFalse("This test deploys multiple models which cannot be accommodated on a single processor", IS_SINGLE_PROCESSOR_TEST); From 343f332c26854accb9bc9e8c1c82e29450f9d61b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 9 Nov 2023 23:33:57 +0100 Subject: [PATCH 044/513] Misc cleanup in o.e.search (#101976) Cleaning up a few more spots here. --- .../BytesRefFieldComparatorSource.java | 2 +- .../LongValuesComparatorSource.java | 2 +- .../search/DefaultSearchContext.java | 5 - .../elasticsearch/search/MultiValueMode.java | 6 +- .../elasticsearch/search/SearchService.java | 6 +- .../VariableWidthHistogramAggregator.java | 2 +- .../aggregations/metrics/TDigestState.java | 4 - .../search/collapse/CollapseBuilder.java | 2 +- .../search/collapse/CollapseContext.java | 12 +- .../search/internal/ContextIndexSearcher.java | 2 +- .../internal/ExitableDirectoryReader.java | 66 ++++----- .../FieldUsageTrackingDirectoryReader.java | 3 +- .../internal/FilteredSearchContext.java | 5 - .../internal/InternalScrollSearchRequest.java | 5 - .../search/internal/SearchContext.java | 2 - .../search/internal/ShardSearchRequest.java | 4 +- .../search/internal/SubSearchContext.java | 5 - .../search/lookup/LeafDocLookup.java | 78 +++++------ .../search/rank/RankSearchContext.java | 5 - .../search/rescore/QueryRescorer.java | 3 - ...oPointScriptFieldDistanceFeatureQuery.java | 1 - .../search/sort/BucketedSort.java | 22 +-- .../search/sort/ScoreSortBuilder.java | 12 +- .../search/sort/ScriptSortBuilder.java | 6 +- .../search/sort/SortBuilders.java | 11 -- .../elasticsearch/search/sort/SortValue.java | 7 +- .../suggest/DirectSpellcheckerSettings.java | 20 +-- .../elasticsearch/search/suggest/Suggest.java | 5 - .../suggest/SuggestionSearchContext.java | 4 +- .../completion/CompletionSuggestion.java | 2 +- .../CompletionSuggestionBuilder.java | 7 - .../suggest/completion/FuzzyOptions.java | 12 +- .../suggest/completion/RegexOptions.java | 4 +- .../TopSuggestGroupDocsCollector.java | 2 +- .../context/CategoryQueryContext.java | 2 +- .../completion/context/ContextBuilder.java | 2 +- .../completion/context/ContextMapping.java | 38 +---- .../completion/context/ContextMappings.java | 16 +-- .../completion/context/GeoContextMapping.java | 13 -- .../suggest/phrase/CandidateGenerator.java | 7 - .../suggest/phrase/CandidateScorer.java | 24 ++-- .../phrase/DirectCandidateGenerator.java | 26 +--- .../DirectCandidateGeneratorBuilder.java | 36 ++--- .../search/suggest/phrase/Laplace.java | 2 +- .../search/suggest/phrase/LaplaceScorer.java | 2 +- .../MultiCandidateGeneratorWrapper.java | 7 +- .../suggest/phrase/PhraseSuggester.java | 4 +- .../suggest/phrase/PhraseSuggestion.java | 7 - .../phrase/PhraseSuggestionBuilder.java | 9 +- .../phrase/PhraseSuggestionContext.java | 2 +- .../search/suggest/phrase/StupidBackoff.java | 2 +- .../search/suggest/phrase/WordScorer.java | 17 +-- .../search/suggest/term/TermSuggestion.java | 14 +- .../suggest/term/TermSuggestionBuilder.java | 2 +- .../search/vectors/KnnSearchBuilder.java | 4 - .../search/vectors/VectorSimilarityQuery.java | 4 - .../search/MultiValueModeTests.java | 2 - .../CategoryContextMappingTests.java | 132 +++++++++--------- .../completion/GeoContextMappingTests.java | 51 ++++--- .../elasticsearch/test/TestSearchContext.java | 5 - 60 files changed, 260 insertions(+), 504 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/BytesRefFieldComparatorSource.java b/server/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/BytesRefFieldComparatorSource.java index addc6f33c9eba..a18ea0f90ec08 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/BytesRefFieldComparatorSource.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/BytesRefFieldComparatorSource.java @@ -112,7 +112,7 @@ protected BinaryDocValues getBinaryDocValues(LeafReaderContext context, String f final BitSet rootDocs = nested.rootDocs(context); final DocIdSetIterator innerDocs = nested.innerDocs(context); final int maxChildren = nested.getNestedSort() != null ? nested.getNestedSort().getMaxChildren() : Integer.MAX_VALUE; - selectedValues = sortMode.select(values, missingBytes, rootDocs, innerDocs, context.reader().maxDoc(), maxChildren); + selectedValues = sortMode.select(values, missingBytes, rootDocs, innerDocs, maxChildren); } return selectedValues; } diff --git a/server/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/LongValuesComparatorSource.java b/server/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/LongValuesComparatorSource.java index 827e1618adde2..e8d4363ca9932 100644 --- a/server/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/LongValuesComparatorSource.java +++ b/server/src/main/java/org/elasticsearch/index/fielddata/fieldcomparator/LongValuesComparatorSource.java @@ -90,7 +90,7 @@ private NumericDocValues getNumericDocValues(LeafReaderContext context, long mis final BitSet rootDocs = nested.rootDocs(context); final DocIdSetIterator innerDocs = nested.innerDocs(context); final int maxChildren = nested.getNestedSort() != null ? nested.getNestedSort().getMaxChildren() : Integer.MAX_VALUE; - return sortMode.select(values, missingValue, rootDocs, innerDocs, context.reader().maxDoc(), maxChildren); + return sortMode.select(values, missingValue, rootDocs, innerDocs, maxChildren); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java index 4c1df376ebf63..a4f641fd6f071 100644 --- a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java @@ -468,11 +468,6 @@ public boolean sourceRequested() { return fetchSourceContext != null && fetchSourceContext.fetchSource(); } - @Override - public boolean hasFetchSourceContext() { - return fetchSourceContext != null; - } - @Override public FetchSourceContext fetchSourceContext() { return this.fetchSourceContext; diff --git a/server/src/main/java/org/elasticsearch/search/MultiValueMode.java b/server/src/main/java/org/elasticsearch/search/MultiValueMode.java index 9137d5c97248d..2b5d9cb17b4f4 100644 --- a/server/src/main/java/org/elasticsearch/search/MultiValueMode.java +++ b/server/src/main/java/org/elasticsearch/search/MultiValueMode.java @@ -543,7 +543,7 @@ public int docID() { } @Override - public long longValue() throws IOException { + public long longValue() { return value; } }; @@ -571,7 +571,6 @@ public NumericDocValues select( final long missingValue, final BitSet parentDocs, final DocIdSetIterator childDocs, - int maxDoc, int maxChildren ) throws IOException { if (parentDocs == null || childDocs == null) { @@ -654,7 +653,7 @@ public boolean advanceExact(int target) throws IOException { } @Override - public double doubleValue() throws IOException { + public double doubleValue() { return this.value; } }; @@ -804,7 +803,6 @@ public BinaryDocValues select( final BytesRef missingValue, final BitSet parentDocs, final DocIdSetIterator childDocs, - int maxDoc, int maxChildren ) throws IOException { if (parentDocs == null || childDocs == null) { diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 44a8f641fae91..6ee02fa9425c0 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -739,7 +739,7 @@ public void executeQueryPhase( SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(searchContext) ) { searchContext.searcher().setAggregatedDfs(readerContext.getAggregatedDfs(null)); - processScroll(request, readerContext, searchContext); + processScroll(request, searchContext); QueryPhase.execute(searchContext); executor.success(); readerContext.setRescoreDocIds(searchContext.rescoreDocIds()); @@ -830,7 +830,7 @@ public void executeFetchPhase( ) { searchContext.assignRescoreDocIds(readerContext.getRescoreDocIds(null)); searchContext.searcher().setAggregatedDfs(readerContext.getAggregatedDfs(null)); - processScroll(request, readerContext, searchContext); + processScroll(request, searchContext); searchContext.addQueryResult(); QueryPhase.execute(searchContext); final long afterQueryTime = executor.success(); @@ -1506,7 +1506,7 @@ private static void shortcutDocIdsToLoad(SearchContext context) { context.docIdsToLoad(docIdsToLoad); } - private static void processScroll(InternalScrollSearchRequest request, ReaderContext reader, SearchContext context) { + private static void processScroll(InternalScrollSearchRequest request, SearchContext context) { // process scroll context.from(context.from() + context.size()); context.scrollContext().scroll = request.scroll(); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java index d89d9b07e57bc..516c9d91a7b65 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java @@ -179,7 +179,7 @@ private class MergeBucketsPhase extends CollectionPhase { * Sorts the indices of values by their underlying value * This will produce a merge map whose application will sort values */ - private class ClusterSorter extends InPlaceMergeSorter { + private static class ClusterSorter extends InPlaceMergeSorter { final DoubleArray values; final long[] indexes; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestState.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestState.java index d80eb8a58040e..23c26794f6bb5 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestState.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TDigestState.java @@ -77,9 +77,6 @@ public static TDigestState create(double compression, TDigestExecutionHint execu return switch (executionHint) { case HIGH_ACCURACY -> createOptimizedForAccuracy(compression); case DEFAULT -> create(compression); - default -> throw new IllegalArgumentException( - "Unexpected TDigestExecutionHint in TDigestState initialization: " + executionHint - ); }; } @@ -99,7 +96,6 @@ protected TDigestState(Type type, double compression) { case AVL_TREE -> TDigest.createAvlTreeDigest(compression); case SORTING -> TDigest.createSortingDigest(); case MERGING -> TDigest.createMergingDigest(compression); - default -> throw new IllegalArgumentException("Unexpected TDigestState type: " + type); }; this.type = type; this.compression = compression; diff --git a/server/src/main/java/org/elasticsearch/search/collapse/CollapseBuilder.java b/server/src/main/java/org/elasticsearch/search/collapse/CollapseBuilder.java index 049e06b0d98c7..f787e30644658 100644 --- a/server/src/main/java/org/elasticsearch/search/collapse/CollapseBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/collapse/CollapseBuilder.java @@ -204,6 +204,6 @@ public CollapseContext build(SearchExecutionContext searchExecutionContext) { ); } - return new CollapseContext(field, fieldType, innerHits); + return new CollapseContext(field, fieldType); } } diff --git a/server/src/main/java/org/elasticsearch/search/collapse/CollapseContext.java b/server/src/main/java/org/elasticsearch/search/collapse/CollapseContext.java index 62d7f7cc74cd4..080caaeed0fde 100644 --- a/server/src/main/java/org/elasticsearch/search/collapse/CollapseContext.java +++ b/server/src/main/java/org/elasticsearch/search/collapse/CollapseContext.java @@ -11,23 +11,18 @@ import org.apache.lucene.search.Sort; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType.CollapseType; -import org.elasticsearch.index.query.InnerHitBuilder; import org.elasticsearch.lucene.grouping.SinglePassGroupingCollector; -import java.util.List; - /** * Context used for field collapsing */ public class CollapseContext { private final String fieldName; private final MappedFieldType fieldType; - private final List innerHits; - public CollapseContext(String fieldName, MappedFieldType fieldType, List innerHits) { + public CollapseContext(String fieldName, MappedFieldType fieldType) { this.fieldName = fieldName; this.fieldType = fieldType; - this.innerHits = innerHits; } /** @@ -42,11 +37,6 @@ public MappedFieldType getFieldType() { return fieldType; } - /** The inner hit options to expand the collapsed results **/ - public List getInnerHit() { - return innerHits; - } - public SinglePassGroupingCollector createTopDocs(Sort sort, int topN, FieldDoc after) { if (fieldType.collapseType() == CollapseType.KEYWORD) { return SinglePassGroupingCollector.createKeyword(fieldName, fieldType, sort, topN, after); diff --git a/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java b/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java index 3c69db98c7588..b7c77e4968854 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ContextIndexSearcher.java @@ -265,7 +265,7 @@ private static LeafSlice[] computeSlices(List leaves, int min List sortedLeaves = new ArrayList<>(leaves); // Sort by maxDoc, descending: final Comparator leafComparator = Comparator.comparingInt(l -> l.reader().maxDoc()); - Collections.sort(sortedLeaves, leafComparator.reversed()); + sortedLeaves.sort(leafComparator.reversed()); // we add the groups on a priority queue, so we can add orphan leafs to the smallest group final Comparator> groupComparator = Comparator.comparingInt( l -> l.stream().mapToInt(lr -> lr.reader().maxDoc()).sum() diff --git a/server/src/main/java/org/elasticsearch/search/internal/ExitableDirectoryReader.java b/server/src/main/java/org/elasticsearch/search/internal/ExitableDirectoryReader.java index 794e429bbc473..ecb7833558a6b 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ExitableDirectoryReader.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ExitableDirectoryReader.java @@ -142,28 +142,7 @@ public void searchNearestVectors(String field, byte[] target, KnnCollector colle in.searchNearestVectors(field, target, collector, acceptDocs); return; } - // when acceptDocs is null due to no doc deleted, we will instantiate a new one that would - // match all docs to allow timeout checking. - final Bits updatedAcceptDocs = acceptDocs == null ? new Bits.MatchAllBits(maxDoc()) : acceptDocs; - Bits timeoutCheckingAcceptDocs = new Bits() { - private static final int MAX_CALLS_BEFORE_QUERY_TIMEOUT_CHECK = 10; - private int calls; - - @Override - public boolean get(int index) { - if (calls++ % MAX_CALLS_BEFORE_QUERY_TIMEOUT_CHECK == 0) { - queryCancellation.checkCancelled(); - } - - return updatedAcceptDocs.get(index); - } - - @Override - public int length() { - return updatedAcceptDocs.length(); - } - }; - in.searchNearestVectors(field, target, collector, timeoutCheckingAcceptDocs); + in.searchNearestVectors(field, target, collector, new TimeOutCheckingBits(acceptDocs)); } @Override @@ -181,29 +160,32 @@ public void searchNearestVectors(String field, float[] target, KnnCollector coll in.searchNearestVectors(field, target, collector, acceptDocs); return; } - // when acceptDocs is null due to no doc deleted, we will instantiate a new one that would - // match all docs to allow timeout checking. - final Bits updatedAcceptDocs = acceptDocs == null ? new Bits.MatchAllBits(maxDoc()) : acceptDocs; - Bits timeoutCheckingAcceptDocs = new Bits() { - private static final int MAX_CALLS_BEFORE_QUERY_TIMEOUT_CHECK = 10; - private int calls; - - @Override - public boolean get(int index) { - if (calls++ % MAX_CALLS_BEFORE_QUERY_TIMEOUT_CHECK == 0) { - queryCancellation.checkCancelled(); - } - - return updatedAcceptDocs.get(index); - } + in.searchNearestVectors(field, target, collector, new TimeOutCheckingBits(acceptDocs)); + } + + private class TimeOutCheckingBits implements Bits { + private static final int MAX_CALLS_BEFORE_QUERY_TIMEOUT_CHECK = 10; + private final Bits updatedAcceptDocs; + private int calls; - @Override - public int length() { - return updatedAcceptDocs.length(); + TimeOutCheckingBits(Bits acceptDocs) { + // when acceptDocs is null due to no doc deleted, we will instantiate a new one that would + // match all docs to allow timeout checking. + this.updatedAcceptDocs = acceptDocs == null ? new Bits.MatchAllBits(maxDoc()) : acceptDocs; + } + + @Override + public boolean get(int index) { + if (calls++ % MAX_CALLS_BEFORE_QUERY_TIMEOUT_CHECK == 0) { + queryCancellation.checkCancelled(); } - }; + return updatedAcceptDocs.get(index); + } - in.searchNearestVectors(field, target, collector, acceptDocs); + @Override + public int length() { + return updatedAcceptDocs.length(); + } } } diff --git a/server/src/main/java/org/elasticsearch/search/internal/FieldUsageTrackingDirectoryReader.java b/server/src/main/java/org/elasticsearch/search/internal/FieldUsageTrackingDirectoryReader.java index 5dc0374b73fc6..07fa169642dbf 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/FieldUsageTrackingDirectoryReader.java +++ b/server/src/main/java/org/elasticsearch/search/internal/FieldUsageTrackingDirectoryReader.java @@ -251,8 +251,7 @@ public void searchNearestVectors(String field, float[] target, KnnCollector coll @Override public String toString() { - final StringBuilder sb = new StringBuilder("FieldUsageTrackingLeafReader(reader="); - return sb.append(in).append(')').toString(); + return "FieldUsageTrackingLeafReader(reader=" + in + ')'; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index 67a265127026d..8bd91c9b9cfe7 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -174,11 +174,6 @@ public boolean sourceRequested() { return in.sourceRequested(); } - @Override - public boolean hasFetchSourceContext() { - return in.hasFetchSourceContext(); - } - @Override public FetchSourceContext fetchSourceContext() { return in.fetchSourceContext(); diff --git a/server/src/main/java/org/elasticsearch/search/internal/InternalScrollSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/InternalScrollSearchRequest.java index b8886c3e79a8c..911b647067e63 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/InternalScrollSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/InternalScrollSearchRequest.java @@ -54,11 +54,6 @@ public Scroll scroll() { return scroll; } - public InternalScrollSearchRequest scroll(Scroll scroll) { - this.scroll = scroll; - return this; - } - @Override public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { return new SearchShardTask(id, type, action, getDescription(), parentTaskId, headers); diff --git a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java index 274dc233ff5c7..ef67d3d19e42f 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -182,8 +182,6 @@ public final void assignRescoreDocIds(RescoreDocIds rescoreDocIds) { */ public abstract boolean sourceRequested(); - public abstract boolean hasFetchSourceContext(); - public abstract FetchSourceContext fetchSourceContext(); public abstract SearchContext fetchSourceContext(FetchSourceContext fetchSourceContext); diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index fbfcfdf9500ed..fe9cfdc87695e 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -240,7 +240,7 @@ public ShardSearchRequest( this.originalIndices = originalIndices; this.readerId = readerId; this.keepAlive = keepAlive; - assert keepAlive == null || readerId != null : "readerId: " + readerId + " keepAlive: " + keepAlive; + assert keepAlive == null || readerId != null : "readerId: null keepAlive: " + keepAlive; this.channelVersion = TransportVersion.current(); this.waitForCheckpoint = waitForCheckpoint; this.waitForCheckpointsTimeout = waitForCheckpointsTimeout; @@ -334,7 +334,7 @@ public ShardSearchRequest(StreamInput in) throws IOException { readerId = null; keepAlive = null; } - assert keepAlive == null || readerId != null : "readerId: " + readerId + " keepAlive: " + keepAlive; + assert keepAlive == null || readerId != null : "readerId: null keepAlive: " + keepAlive; channelVersion = TransportVersion.min(TransportVersion.readVersion(in), in.getTransportVersion()); if (in.getTransportVersion().onOrAfter(TransportVersions.V_7_16_0)) { waitForCheckpoint = in.readLong(); diff --git a/server/src/main/java/org/elasticsearch/search/internal/SubSearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/SubSearchContext.java index f1fd984aec5ba..8b4824e42cbf4 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SubSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SubSearchContext.java @@ -107,11 +107,6 @@ public boolean sourceRequested() { return fetchSourceContext != null && fetchSourceContext.fetchSource(); } - @Override - public boolean hasFetchSourceContext() { - return fetchSourceContext != null; - } - @Override public FetchSourceContext fetchSourceContext() { return fetchSourceContext; diff --git a/server/src/main/java/org/elasticsearch/search/lookup/LeafDocLookup.java b/server/src/main/java/org/elasticsearch/search/lookup/LeafDocLookup.java index bd6971dceb7be..988ea24d0fcc2 100644 --- a/server/src/main/java/org/elasticsearch/search/lookup/LeafDocLookup.java +++ b/server/src/main/java/org/elasticsearch/search/lookup/LeafDocLookup.java @@ -102,30 +102,27 @@ private FieldFactoryWrapper getFactoryForField(String fieldName) { // Load the field data on behalf of the script. Otherwise, it would require // additional permissions to deal with pagedbytes/ramusagestimator/etc. - return AccessController.doPrivileged(new PrivilegedAction() { - @Override - public FieldFactoryWrapper run() { - FieldFactoryWrapper fieldFactory = null; - IndexFieldData indexFieldData = fieldDataLookup.apply(fieldType, SCRIPT); + return AccessController.doPrivileged((PrivilegedAction) () -> { + IndexFieldData indexFieldData = fieldDataLookup.apply(fieldType, SCRIPT); - FieldFactoryWrapper docFactory = null; + FieldFactoryWrapper docFactory = null; - if (docFactoryCache.isEmpty() == false) { - docFactory = docFactoryCache.get(fieldName); - } + if (docFactoryCache.isEmpty() == false) { + docFactory = docFactoryCache.get(fieldName); + } - // if this field has already been accessed via the doc-access API and the field-access API - // uses doc values then we share to avoid double-loading - if (docFactory != null && indexFieldData instanceof SourceValueFetcherIndexFieldData == false) { - fieldFactory = docFactory; - } else { - fieldFactory = new FieldFactoryWrapper(indexFieldData.load(reader).getScriptFieldFactory(fieldName)); - } + // if this field has already been accessed via the doc-access API and the field-access API + // uses doc values then we share to avoid double-loading + FieldFactoryWrapper fieldFactory; + if (docFactory != null && indexFieldData instanceof SourceValueFetcherIndexFieldData == false) { + fieldFactory = docFactory; + } else { + fieldFactory = new FieldFactoryWrapper(indexFieldData.load(reader).getScriptFieldFactory(fieldName)); + } - fieldFactoryCache.put(fieldName, fieldFactory); + fieldFactoryCache.put(fieldName, fieldFactory); - return fieldFactory; - } + return fieldFactory; }); } @@ -150,35 +147,32 @@ private FieldFactoryWrapper getFactoryForDoc(String fieldName) { // Load the field data on behalf of the script. Otherwise, it would require // additional permissions to deal with pagedbytes/ramusagestimator/etc. - return AccessController.doPrivileged(new PrivilegedAction() { - @Override - public FieldFactoryWrapper run() { - FieldFactoryWrapper docFactory = null; - FieldFactoryWrapper fieldFactory = null; - - if (fieldFactoryCache.isEmpty() == false) { - fieldFactory = fieldFactoryCache.get(fieldName); - } + return AccessController.doPrivileged((PrivilegedAction) () -> { + FieldFactoryWrapper docFactory = null; + FieldFactoryWrapper fieldFactory = null; - if (fieldFactory != null) { - IndexFieldData fieldIndexFieldData = fieldDataLookup.apply(fieldType, SCRIPT); + if (fieldFactoryCache.isEmpty() == false) { + fieldFactory = fieldFactoryCache.get(fieldName); + } - // if this field has already been accessed via the field-access API and the field-access API - // uses doc values then we share to avoid double-loading - if (fieldIndexFieldData instanceof SourceValueFetcherIndexFieldData == false) { - docFactory = fieldFactory; - } - } + if (fieldFactory != null) { + IndexFieldData fieldIndexFieldData = fieldDataLookup.apply(fieldType, SCRIPT); - if (docFactory == null) { - IndexFieldData indexFieldData = fieldDataLookup.apply(fieldType, SEARCH); - docFactory = new FieldFactoryWrapper(indexFieldData.load(reader).getScriptFieldFactory(fieldName)); + // if this field has already been accessed via the field-access API and the field-access API + // uses doc values then we share to avoid double-loading + if (fieldIndexFieldData instanceof SourceValueFetcherIndexFieldData == false) { + docFactory = fieldFactory; } + } - docFactoryCache.put(fieldName, docFactory); - - return docFactory; + if (docFactory == null) { + IndexFieldData indexFieldData = fieldDataLookup.apply(fieldType, SEARCH); + docFactory = new FieldFactoryWrapper(indexFieldData.load(reader).getScriptFieldFactory(fieldName)); } + + docFactoryCache.put(fieldName, docFactory); + + return docFactory; }); } diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java index 4b075523c5286..84f04283d64e8 100644 --- a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java @@ -320,11 +320,6 @@ public boolean sourceRequested() { throw new UnsupportedOperationException(); } - @Override - public boolean hasFetchSourceContext() { - throw new UnsupportedOperationException(); - } - @Override public FetchSourceContext fetchSourceContext() { throw new UnsupportedOperationException(); diff --git a/server/src/main/java/org/elasticsearch/search/rescore/QueryRescorer.java b/server/src/main/java/org/elasticsearch/search/rescore/QueryRescorer.java index 389e3a56cf152..c873717fe55e7 100644 --- a/server/src/main/java/org/elasticsearch/search/rescore/QueryRescorer.java +++ b/server/src/main/java/org/elasticsearch/search/rescore/QueryRescorer.java @@ -197,9 +197,6 @@ public void setScoreMode(QueryRescoreMode scoreMode) { this.scoreMode = scoreMode; } - public void setScoreMode(String scoreMode) { - setScoreMode(QueryRescoreMode.fromString(scoreMode)); - } } } diff --git a/server/src/main/java/org/elasticsearch/search/runtime/GeoPointScriptFieldDistanceFeatureQuery.java b/server/src/main/java/org/elasticsearch/search/runtime/GeoPointScriptFieldDistanceFeatureQuery.java index a7977c18d338c..de081fd386d54 100644 --- a/server/src/main/java/org/elasticsearch/search/runtime/GeoPointScriptFieldDistanceFeatureQuery.java +++ b/server/src/main/java/org/elasticsearch/search/runtime/GeoPointScriptFieldDistanceFeatureQuery.java @@ -114,7 +114,6 @@ private class DistanceScorer extends Scorer { private final TwoPhaseIterator twoPhase; private final DocIdSetIterator disi; private final float weight; - private double maxDistance = GeoUtils.EARTH_MEAN_RADIUS_METERS * Math.PI; protected DistanceScorer(Weight weight, AbstractLongFieldScript script, int maxDoc, float boost) { super(weight); diff --git a/server/src/main/java/org/elasticsearch/search/sort/BucketedSort.java b/server/src/main/java/org/elasticsearch/search/sort/BucketedSort.java index ae2f7fc4ecbbb..b1b30856324b4 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/BucketedSort.java +++ b/server/src/main/java/org/elasticsearch/search/sort/BucketedSort.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.List; -import java.util.Locale; import static java.util.Collections.emptyList; @@ -110,7 +109,7 @@ interface Loader { public void swap(long lhs, long rhs) {} @Override - public Loader loader(LeafReaderContext ctx) throws IOException { + public Loader loader(LeafReaderContext ctx) { return (index, doc) -> {}; } }; @@ -254,24 +253,6 @@ public boolean inHeapMode(long bucket) { */ protected abstract void swap(long lhs, long rhs); - /** - * Return a fairly human readable representation of the array backing the sort. - *

- * This is intentionally not a {@link #toString()} implementation because it'll - * be quite slow. - *

- */ - protected final String debugFormat() { - StringBuilder b = new StringBuilder(); - for (long index = 0; index < values().size(); index++) { - if (index % bucketSize == 0) { - b.append('\n').append(String.format(Locale.ROOT, "%20d", index / bucketSize)).append(": "); - } - b.append(String.format(Locale.ROOT, "%20s", getValue(index))).append(' '); - } - return b.toString(); - } - /** * Initialize the gather offsets after setting up values. Subclasses * should call this once, after setting up their {@link #values()}. @@ -415,7 +396,6 @@ public final void collect(int doc, long bucket) throws IOException { } else { setNextGatherOffset(rootIndex, next - 1); } - return; } /** diff --git a/server/src/main/java/org/elasticsearch/search/sort/ScoreSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/ScoreSortBuilder.java index 5d11563b5d8ed..0c9b56b1855d7 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/ScoreSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/ScoreSortBuilder.java @@ -100,8 +100,12 @@ public SortFieldAndFormat build(SearchExecutionContext context) { } @Override - public BucketedSort buildBucketedSort(SearchExecutionContext context, BigArrays bigArrays, int bucketSize, BucketedSort.ExtraData extra) - throws IOException { + public BucketedSort buildBucketedSort( + SearchExecutionContext context, + BigArrays bigArrays, + int bucketSize, + BucketedSort.ExtraData extra + ) { return new BucketedSort.ForFloats(bigArrays, order, DocValueFormat.RAW, bucketSize, extra) { @Override public boolean needsScores() { @@ -109,7 +113,7 @@ public boolean needsScores() { } @Override - public Leaf forLeaf(LeafReaderContext ctx) throws IOException { + public Leaf forLeaf(LeafReaderContext ctx) { return new BucketedSort.ForFloats.Leaf(ctx) { private Scorable scorer; private float score; @@ -165,7 +169,7 @@ public TransportVersion getMinimalSupportedVersion() { } @Override - public ScoreSortBuilder rewrite(QueryRewriteContext ctx) throws IOException { + public ScoreSortBuilder rewrite(QueryRewriteContext ctx) { return this; } diff --git a/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java b/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java index 4ac7348a6c4a4..a0745d0f9c64a 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/sort/ScriptSortBuilder.java @@ -295,7 +295,7 @@ protected SortedBinaryDocValues getValues(LeafReaderContext context) throws IOEx final BytesRefBuilder spare = new BytesRefBuilder(); @Override - public boolean advanceExact(int doc) throws IOException { + public boolean advanceExact(int doc) { leafScript.setDocument(doc); return true; } @@ -343,7 +343,7 @@ protected SortedNumericDoubleValues getValues(LeafReaderContext context) throws leafScript = numberSortScript.newInstance(new DocValuesDocReader(searchLookup, context)); final NumericDoubleValues values = new NumericDoubleValues() { @Override - public boolean advanceExact(int doc) throws IOException { + public boolean advanceExact(int doc) { leafScript.setDocument(doc); return true; } @@ -374,7 +374,7 @@ protected SortedBinaryDocValues getValues(LeafReaderContext context) throws IOEx final BinaryDocValues values = new AbstractBinaryDocValues() { @Override - public boolean advanceExact(int doc) throws IOException { + public boolean advanceExact(int doc) { leafScript.setDocument(doc); return true; } diff --git a/server/src/main/java/org/elasticsearch/search/sort/SortBuilders.java b/server/src/main/java/org/elasticsearch/search/sort/SortBuilders.java index 34363a614a7e4..c0bcbdc98e35f 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/SortBuilders.java +++ b/server/src/main/java/org/elasticsearch/search/sort/SortBuilders.java @@ -8,7 +8,6 @@ package org.elasticsearch.search.sort; -import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.script.Script; import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.sort.ScriptSortBuilder.ScriptSortType; @@ -65,16 +64,6 @@ public static GeoDistanceSortBuilder geoDistanceSort(String fieldName, double la return new GeoDistanceSortBuilder(fieldName, lat, lon); } - /** - * Constructs a new distance based sort on a geo point like field. - * - * @param fieldName The geo point like field name. - * @param points The points to create the range distance facets from. - */ - public static GeoDistanceSortBuilder geoDistanceSort(String fieldName, GeoPoint... points) { - return new GeoDistanceSortBuilder(fieldName, points); - } - /** * Constructs a new distance based sort on a geo point like field. * diff --git a/server/src/main/java/org/elasticsearch/search/sort/SortValue.java b/server/src/main/java/org/elasticsearch/search/sort/SortValue.java index 067439931a85b..ab7dcd6615f79 100644 --- a/server/src/main/java/org/elasticsearch/search/sort/SortValue.java +++ b/server/src/main/java/org/elasticsearch/search/sort/SortValue.java @@ -369,7 +369,6 @@ private static class EmptySortValue extends SortValue { public static final String NAME = "empty"; private static final String EMPTY_STRING = ""; - private int sortValue = 0; private EmptySortValue() {} @@ -381,7 +380,7 @@ public String getWriteableName() { } @Override - public void writeTo(StreamOutput out) throws IOException {} + public void writeTo(StreamOutput out) {} @Override public Object getKey() { @@ -394,7 +393,7 @@ public String format(DocValueFormat format) { } @Override - protected XContentBuilder rawToXContent(XContentBuilder builder) throws IOException { + protected XContentBuilder rawToXContent(XContentBuilder builder) { return builder; } @@ -420,7 +419,7 @@ public String toString() { @Override public int typeComparisonKey() { - return sortValue; + return 0; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/suggest/DirectSpellcheckerSettings.java b/server/src/main/java/org/elasticsearch/search/suggest/DirectSpellcheckerSettings.java index 80beb5d2ec7ca..0956a9f94677c 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/DirectSpellcheckerSettings.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/DirectSpellcheckerSettings.java @@ -20,17 +20,17 @@ public class DirectSpellcheckerSettings { // NB: If this changes, make sure to change the default in TermBuilderSuggester - public static SuggestMode DEFAULT_SUGGEST_MODE = SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX; - public static float DEFAULT_ACCURACY = 0.5f; - public static SortBy DEFAULT_SORT = SortBy.SCORE; + public static final SuggestMode DEFAULT_SUGGEST_MODE = SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX; + public static final float DEFAULT_ACCURACY = 0.5f; + public static final SortBy DEFAULT_SORT = SortBy.SCORE; // NB: If this changes, make sure to change the default in TermBuilderSuggester - public static StringDistance DEFAULT_STRING_DISTANCE = DirectSpellChecker.INTERNAL_LEVENSHTEIN; - public static int DEFAULT_MAX_EDITS = LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE; - public static int DEFAULT_MAX_INSPECTIONS = 5; - public static float DEFAULT_MAX_TERM_FREQ = 0.01f; - public static int DEFAULT_PREFIX_LENGTH = 1; - public static int DEFAULT_MIN_WORD_LENGTH = 4; - public static float DEFAULT_MIN_DOC_FREQ = 0f; + public static final StringDistance DEFAULT_STRING_DISTANCE = DirectSpellChecker.INTERNAL_LEVENSHTEIN; + public static final int DEFAULT_MAX_EDITS = LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE; + public static final int DEFAULT_MAX_INSPECTIONS = 5; + public static final float DEFAULT_MAX_TERM_FREQ = 0.01f; + public static final int DEFAULT_PREFIX_LENGTH = 1; + public static final int DEFAULT_MIN_WORD_LENGTH = 4; + public static final float DEFAULT_MIN_DOC_FREQ = 0f; private SuggestMode suggestMode = DEFAULT_SUGGEST_MODE; private float accuracy = DEFAULT_ACCURACY; diff --git a/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java b/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java index f126091c785d8..f3371caf4c1a7 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java @@ -206,7 +206,6 @@ public int hashCode() { @SuppressWarnings("rawtypes") public abstract static class Suggestion implements Iterable, NamedWriteable, ToXContentFragment { - public static final int TYPE = 0; protected final String name; protected final int size; protected final List entries = new ArrayList<>(5); @@ -635,10 +634,6 @@ public boolean collateMatch() { return (collateMatch != null) ? collateMatch : true; } - protected void setScore(float score) { - this.score = score; - } - @Override public void writeTo(StreamOutput out) throws IOException { out.writeText(text); diff --git a/server/src/main/java/org/elasticsearch/search/suggest/SuggestionSearchContext.java b/server/src/main/java/org/elasticsearch/search/suggest/SuggestionSearchContext.java index 674f936890283..37cc7bb59c253 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/SuggestionSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/SuggestionSearchContext.java @@ -35,8 +35,8 @@ public abstract static class SuggestionContext { private Analyzer analyzer; private int size = 5; private int shardSize = -1; - private SearchExecutionContext searchExecutionContext; - private Suggester suggester; + private final SearchExecutionContext searchExecutionContext; + private final Suggester suggester; protected SuggestionContext(Suggester suggester, SearchExecutionContext searchExecutionContext) { this.suggester = suggester; diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java index c71673962ca2d..e088948b18e03 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java @@ -56,7 +56,7 @@ */ public final class CompletionSuggestion extends Suggest.Suggestion { - private boolean skipDuplicates; + private final boolean skipDuplicates; /** * Creates a completion suggestion given its name, size and whether it should skip duplicates diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java index 86e18b3e5a406..7a3bc3c67ba6d 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestionBuilder.java @@ -211,13 +211,6 @@ private CompletionSuggestionBuilder contexts(XContentBuilder contextBuilder) { return this; } - /** - * Returns whether duplicate suggestions should be filtered out. - */ - public boolean skipDuplicates() { - return skipDuplicates; - } - /** * Should duplicates be filtered or not. Defaults to {@code false}. */ diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/FuzzyOptions.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/FuzzyOptions.java index f241b6f89633e..7d7d5516c50ae 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/FuzzyOptions.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/FuzzyOptions.java @@ -62,12 +62,12 @@ public static Builder builder() { return new Builder(); } - private int editDistance; - private boolean transpositions; - private int fuzzyMinLength; - private int fuzzyPrefixLength; - private boolean unicodeAware; - private int maxDeterminizedStates; + private final int editDistance; + private final boolean transpositions; + private final int fuzzyMinLength; + private final int fuzzyPrefixLength; + private final boolean unicodeAware; + private final int maxDeterminizedStates; private FuzzyOptions( int editDistance, diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/RegexOptions.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/RegexOptions.java index 0759b413dd664..fdfa1303b2d77 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/RegexOptions.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/RegexOptions.java @@ -61,8 +61,8 @@ static RegexOptions parse(XContentParser parser) throws IOException { return PARSER.parse(parser, null).build(); } - private int flagsValue; - private int maxDeterminizedStates; + private final int flagsValue; + private final int maxDeterminizedStates; private RegexOptions(int flagsValue, int maxDeterminizedStates) { this.flagsValue = flagsValue; diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/TopSuggestGroupDocsCollector.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/TopSuggestGroupDocsCollector.java index bac3b7491a661..31959df6b023e 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/TopSuggestGroupDocsCollector.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/TopSuggestGroupDocsCollector.java @@ -25,7 +25,7 @@ * the best one per document (sorted by weight) is kept. **/ class TopSuggestGroupDocsCollector extends TopSuggestDocsCollector { - private Map> docContexts = new HashMap<>(); + private final Map> docContexts = new HashMap<>(); /** * Sole constructor diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/CategoryQueryContext.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/CategoryQueryContext.java index ce0c58463bad2..65c464cac256d 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/CategoryQueryContext.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/CategoryQueryContext.java @@ -72,7 +72,7 @@ public boolean equals(Object o) { if (isPrefix != that.isPrefix) return false; if (boost != that.boost) return false; - return category != null ? category.equals(that.category) : that.category == null; + return Objects.equals(category, that.category); } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextBuilder.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextBuilder.java index b180e6fd13335..2a83bf289bdef 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextBuilder.java @@ -13,7 +13,7 @@ */ public abstract class ContextBuilder> { - protected String name; + protected final String name; /** * @param name of the context mapper to build diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMapping.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMapping.java index c48a1ccb12e6f..d2edd460b926d 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMapping.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMapping.java @@ -165,41 +165,5 @@ public String toString() { } } - public static class InternalQueryContext { - public final String context; - public final int boost; - public final boolean isPrefix; - - public InternalQueryContext(String context, int boost, boolean isPrefix) { - this.context = context; - this.boost = boost; - this.isPrefix = isPrefix; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - InternalQueryContext that = (InternalQueryContext) o; - - if (boost != that.boost) return false; - if (isPrefix != that.isPrefix) return false; - return context != null ? context.equals(that.context) : that.context == null; - - } - - @Override - public int hashCode() { - int result = context != null ? context.hashCode() : 0; - result = 31 * result + boost; - result = 31 * result + (isPrefix ? 1 : 0); - return result; - } - - @Override - public String toString() { - return "QueryContext{" + "context='" + context + '\'' + ", boost=" + boost + ", isPrefix=" + isPrefix + '}'; - } - } + public record InternalQueryContext(String context, int boost, boolean isPrefix) {} } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMappings.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMappings.java index 9a975fe930979..f7709d7aac911 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMappings.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/ContextMappings.java @@ -142,7 +142,7 @@ protected Iterable contexts() { if (typedContexts.isEmpty()) { throw new IllegalArgumentException("Contexts are mandatory in context enabled completion field [" + name + "]"); } - return new ArrayList(typedContexts); + return new ArrayList<>(typedContexts); } } @@ -166,8 +166,8 @@ public ContextQuery toContextQuery(CompletionQuery query, Map internalQueryContext = queryContexts.get(mapping.name()); if (internalQueryContext != null) { for (ContextMapping.InternalQueryContext context : internalQueryContext) { - scratch.append(context.context); - typedContextQuery.addContext(scratch.toCharsRef(), context.boost, context.isPrefix == false); + scratch.append(context.context()); + typedContextQuery.addContext(scratch.toCharsRef(), context.boost(), context.isPrefix() == false); scratch.setLength(1); hasContext = true; } @@ -193,12 +193,8 @@ public Map> getNamedContexts(List contexts) { int typeId = typedContext.charAt(0); assert typeId < contextMappings.size() : "Returned context has invalid type"; ContextMapping mapping = contextMappings.get(typeId); - Set contextEntries = contextMap.get(mapping.name()); - if (contextEntries == null) { - contextEntries = new HashSet<>(); - contextMap.put(mapping.name(), contextEntries); - } - contextEntries.add(typedContext.subSequence(1, typedContext.length()).toString()); + contextMap.computeIfAbsent(mapping.name(), k -> new HashSet<>()) + .add(typedContext.subSequence(1, typedContext.length()).toString()); } return contextMap; } @@ -273,7 +269,7 @@ public int hashCode() { @Override public boolean equals(Object obj) { - if (obj == null || (obj instanceof ContextMappings) == false) { + if ((obj instanceof ContextMappings) == false) { return false; } ContextMappings other = ((ContextMappings) obj); diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java index 933d2198a2dae..2cd7a751264bd 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/context/GeoContextMapping.java @@ -345,19 +345,6 @@ public Builder precision(String precision) { return precision(DistanceUnit.parse(precision, DistanceUnit.METERS, DistanceUnit.METERS)); } - /** - * Set the precision use o make suggestions - * - * @param precision - * precision value - * @param unit - * {@link DistanceUnit} to use - * @return this - */ - public Builder precision(double precision, DistanceUnit unit) { - return precision(unit.toMeters(precision)); - } - /** * Set the precision use o make suggestions * diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/CandidateGenerator.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/CandidateGenerator.java index 61dfb0f075d34..fc29d1ed7a567 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/CandidateGenerator.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/CandidateGenerator.java @@ -17,15 +17,8 @@ //TODO public for tests public abstract class CandidateGenerator { - public abstract boolean isKnownWord(BytesRef term) throws IOException; - public abstract TermStats termStats(BytesRef term) throws IOException; - public CandidateSet drawCandidates(BytesRef term) throws IOException { - CandidateSet set = new CandidateSet(Candidate.EMPTY, createCandidate(term, true)); - return drawCandidates(set); - } - public Candidate createCandidate(BytesRef term, boolean userInput) throws IOException { return createCandidate(term, termStats(term), 1.0, userInput); } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/CandidateScorer.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/CandidateScorer.java index e379674d02eab..fdc05d12a2389 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/CandidateScorer.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/CandidateScorer.java @@ -28,13 +28,13 @@ public Correction[] findBestCandiates(CandidateSet[] sets, float errorFraction, if (sets.length == 0) { return Correction.EMPTY; } - PriorityQueue corrections = new PriorityQueue(maxNumCorrections) { + PriorityQueue corrections = new PriorityQueue<>(maxNumCorrections) { @Override protected boolean lessThan(Correction a, Correction b) { return a.compareTo(b) < 0; } }; - int numMissspellings = 1; + final int numMissspellings; if (errorFraction >= 1.0) { numMissspellings = (int) errorFraction; } else { @@ -62,11 +62,11 @@ public void findCandidates( CandidateSet current = candidates[ord]; if (ord == candidates.length - 1) { path[ord] = current.originalTerm; - updateTop(candidates, path, corrections, cutoffScore, pathScore + scorer.score(path, candidates, ord, gramSize)); + updateTop(candidates, path, corrections, cutoffScore, pathScore + scorer.score(path, ord, gramSize)); if (numMissspellingsLeft > 0) { for (int i = 0; i < current.candidates.length; i++) { path[ord] = current.candidates[i]; - updateTop(candidates, path, corrections, cutoffScore, pathScore + scorer.score(path, candidates, ord, gramSize)); + updateTop(candidates, path, corrections, cutoffScore, pathScore + scorer.score(path, ord, gramSize)); } } } else { @@ -79,7 +79,7 @@ public void findCandidates( numMissspellingsLeft, corrections, cutoffScore, - pathScore + scorer.score(path, candidates, ord, gramSize) + pathScore + scorer.score(path, ord, gramSize) ); for (int i = 0; i < current.candidates.length; i++) { path[ord] = current.candidates[i]; @@ -90,20 +90,12 @@ public void findCandidates( numMissspellingsLeft - 1, corrections, cutoffScore, - pathScore + scorer.score(path, candidates, ord, gramSize) + pathScore + scorer.score(path, ord, gramSize) ); } } else { path[ord] = current.originalTerm; - findCandidates( - candidates, - path, - ord + 1, - 0, - corrections, - cutoffScore, - pathScore + scorer.score(path, candidates, ord, gramSize) - ); + findCandidates(candidates, path, ord + 1, 0, corrections, cutoffScore, pathScore + scorer.score(path, ord, gramSize)); } } @@ -135,7 +127,7 @@ private void updateTop( public double score(Candidate[] path, CandidateSet[] candidates) throws IOException { double score = 0.0d; for (int i = 0; i < candidates.length; i++) { - score += scorer.score(path, candidates, i, gramSize); + score += scorer.score(path, i, gramSize); } return Math.exp(score); } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java index 98143e0acf413..b95971d13c11d 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGenerator.java @@ -97,14 +97,6 @@ public DirectCandidateGenerator( termsEnum = terms.iterator(); } - /* (non-Javadoc) - * @see org.elasticsearch.search.suggest.phrase.CandidateGenerator#isKnownWord(org.apache.lucene.util.BytesRef) - */ - @Override - public boolean isKnownWord(BytesRef term) throws IOException { - return termStats(term).docFreq > 0; - } - /* (non-Javadoc) * @see org.elasticsearch.search.suggest.phrase.CandidateGenerator#frequency(org.apache.lucene.util.BytesRef) */ @@ -128,10 +120,6 @@ public TermStats internalTermStats(BytesRef term) throws IOException { return new TermStats(0, 0); } - public String getField() { - return field; - } - @Override public CandidateSet drawCandidates(CandidateSet set) throws IOException { Candidate original = set.originalTerm; @@ -181,15 +169,14 @@ protected BytesRef preFilter(final BytesRef term, final CharsRefBuilder spare, f if (preFilter == null) { return term; } - final BytesRefBuilder result = byteSpare; analyze(preFilter, term, field, new TokenConsumer() { @Override - public void nextToken() throws IOException { - this.fillBytesRef(result); + public void nextToken() { + this.fillBytesRef(byteSpare); } }, spare); - return result.get(); + return byteSpare.get(); } protected void postFilter( @@ -344,11 +331,10 @@ public boolean equals(Object obj) { if (getClass() != obj.getClass()) return false; Candidate other = (Candidate) obj; if (term == null) { - if (other.term != null) return false; + return other.term == null; } else { - if (term.equals(other.term) == false) return false; + return term.equals(other.term) != false; } - return true; } /** Lower scores sort first; if scores are equal, then later (zzz) terms sort first */ @@ -364,7 +350,7 @@ public int compareTo(Candidate other) { } @Override - public Candidate createCandidate(BytesRef term, TermStats termStats, double channelScore, boolean userInput) throws IOException { + public Candidate createCandidate(BytesRef term, TermStats termStats, double channelScore, boolean userInput) { return new Candidate(term, termStats, channelScore, score(termStats, channelScore, sumTotalTermFreq), userInput); } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java index b3cb3444d2206..a153d4de54dcb 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java @@ -437,32 +437,24 @@ public PhraseSuggestionContext.DirectCandidateGenerator build(IndexAnalyzers ind private static SuggestMode resolveSuggestMode(String suggestMode) { suggestMode = suggestMode.toLowerCase(Locale.US); - if ("missing".equals(suggestMode)) { - return SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX; - } else if ("popular".equals(suggestMode)) { - return SuggestMode.SUGGEST_MORE_POPULAR; - } else if ("always".equals(suggestMode)) { - return SuggestMode.SUGGEST_ALWAYS; - } else { - throw new IllegalArgumentException("Illegal suggest mode " + suggestMode); - } + return switch (suggestMode) { + case "missing" -> SuggestMode.SUGGEST_WHEN_NOT_IN_INDEX; + case "popular" -> SuggestMode.SUGGEST_MORE_POPULAR; + case "always" -> SuggestMode.SUGGEST_ALWAYS; + default -> throw new IllegalArgumentException("Illegal suggest mode " + suggestMode); + }; } static StringDistance resolveDistance(String distanceVal) { distanceVal = distanceVal.toLowerCase(Locale.ROOT); - if ("internal".equals(distanceVal)) { - return DirectSpellChecker.INTERNAL_LEVENSHTEIN; - } else if ("damerau_levenshtein".equals(distanceVal)) { - return new LuceneLevenshteinDistance(); - } else if ("levenshtein".equals(distanceVal)) { - return new LevenshteinDistance(); - } else if ("jaro_winkler".equals(distanceVal)) { - return new JaroWinklerDistance(); - } else if ("ngram".equals(distanceVal)) { - return new NGramDistance(); - } else { - throw new IllegalArgumentException("Illegal distance option " + distanceVal); - } + return switch (distanceVal) { + case "internal" -> DirectSpellChecker.INTERNAL_LEVENSHTEIN; + case "damerau_levenshtein" -> new LuceneLevenshteinDistance(); + case "levenshtein" -> new LevenshteinDistance(); + case "jaro_winkler" -> new JaroWinklerDistance(); + case "ngram" -> new NGramDistance(); + default -> throw new IllegalArgumentException("Illegal distance option " + distanceVal); + }; } private static void transferIfNotNull(T value, Consumer consumer) { diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/Laplace.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/Laplace.java index fe85dd70b7337..a14bddd03cdec 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/Laplace.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/Laplace.java @@ -42,7 +42,7 @@ public final class Laplace extends SmoothingModel { */ public static final double DEFAULT_LAPLACE_ALPHA = 0.5; - private double alpha = DEFAULT_LAPLACE_ALPHA; + private final double alpha; /** * Creates a Laplace smoothing model. diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/LaplaceScorer.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/LaplaceScorer.java index ff752a8e62985..dce063d6e655b 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/LaplaceScorer.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/LaplaceScorer.java @@ -15,7 +15,7 @@ import java.io.IOException; final class LaplaceScorer extends WordScorer { - private double alpha; + private final double alpha; LaplaceScorer(IndexReader reader, Terms terms, String field, double realWordLikelihood, BytesRef separator, double alpha) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/MultiCandidateGeneratorWrapper.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/MultiCandidateGeneratorWrapper.java index 80ebd9e45acf8..7e804c173da9c 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/MultiCandidateGeneratorWrapper.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/MultiCandidateGeneratorWrapper.java @@ -19,18 +19,13 @@ public final class MultiCandidateGeneratorWrapper extends CandidateGenerator { private final CandidateGenerator[] candidateGenerator; - private int numCandidates; + private final int numCandidates; public MultiCandidateGeneratorWrapper(int numCandidates, CandidateGenerator... candidateGenerators) { this.candidateGenerator = candidateGenerators; this.numCandidates = numCandidates; } - @Override - public boolean isKnownWord(BytesRef term) throws IOException { - return candidateGenerator[0].isKnownWord(term); - } - @Override public TermStats termStats(BytesRef term) throws IOException { return candidateGenerator[0].termStats(term); diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java index 52c15eb214da9..4400852ebbd5a 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggester.java @@ -159,7 +159,7 @@ public Suggestion> innerExecute( return response; } - private static TokenStream tokenStream(Analyzer analyzer, BytesRef query, CharsRefBuilder spare, String field) throws IOException { + private static TokenStream tokenStream(Analyzer analyzer, BytesRef query, CharsRefBuilder spare, String field) { spare.copyUTF8Bytes(query); return analyzer.tokenStream(field, new CharArrayReader(spare.chars(), 0, spare.length())); } @@ -174,7 +174,7 @@ protected Suggestion> emptySuggestion( String name, PhraseSuggestionContext suggestion, CharsRefBuilder spare - ) throws IOException { + ) { PhraseSuggestion phraseSuggestion = new PhraseSuggestion(name, suggestion.getSize()); spare.copyUTF8Bytes(suggestion.getText()); phraseSuggestion.addTerm(new PhraseSuggestion.Entry(new Text(spare.toString()), 0, spare.length())); diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java index 2cb04b73b7f5f..1c881a9887583 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java @@ -73,13 +73,6 @@ public Entry(StreamInput in) throws IOException { cutoffScore = in.readDouble(); } - /** - * @return cutoff score for suggestions. input term score * confidence for phrase suggest, 0 otherwise - */ - public double getCutoffScore() { - return cutoffScore; - } - @Override protected void merge(Suggestion.Entry