From df01f8124dd26cb545fd1f797fd48bf492a306e3 Mon Sep 17 00:00:00 2001 From: Alexander Spies Date: Mon, 16 Dec 2024 12:35:54 +0100 Subject: [PATCH 01/44] Disable test on release builds (#118752) Fix https://github.com/elastic/elasticsearch/issues/118707 I plan to manually backport this to un-mute the test on 8.x. --- .../xpack/esql/optimizer/PhysicalPlanOptimizerTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index dc3ae0a3388cb..d0c7a1cd61010 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -2331,6 +2331,8 @@ public void testVerifierOnMissingReferences() { } public void testVerifierOnMissingReferencesWithBinaryPlans() throws Exception { + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + // Do not assert serialization: // This will have a LookupJoinExec, which is not serializable because it doesn't leave the coordinator. var plan = physicalPlan(""" From cf7edbbc0f08a0584e5690c7acaa33993078d441 Mon Sep 17 00:00:00 2001 From: Alexander Spies Date: Mon, 16 Dec 2024 12:40:35 +0100 Subject: [PATCH 02/44] Properly skip datasets with lookup indices (#118753) Forward-port of https://github.com/elastic/elasticsearch/pull/118682 to main This mostly just minimizes the diff; LOOKUP JOIN is enabled on main, so we don't need to skip the datasets here. --- .../xpack/esql/CsvTestsDataLoader.java | 43 +++++++++++++------ 1 file changed, 30 insertions(+), 13 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java index 3b656ded94dd7..abfe90f80e372 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvTestsDataLoader.java @@ -41,7 +41,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; import static org.elasticsearch.common.logging.LoggerMessageFormat.format; import static org.elasticsearch.xpack.esql.CsvTestUtils.COMMA_ESCAPING_REGEX; @@ -260,11 +259,22 @@ public static void main(String[] args) throws IOException { public static Set availableDatasetsForEs(RestClient client, boolean supportsIndexModeLookup) throws IOException { boolean inferenceEnabled = clusterHasInferenceEndpoint(client); - return CSV_DATASET_MAP.values() - .stream() - .filter(d -> d.requiresInferenceEndpoint == false || inferenceEnabled) - .filter(d -> supportsIndexModeLookup || d.indexName.endsWith("_lookup") == false) // TODO: use actual index settings - .collect(Collectors.toCollection(HashSet::new)); + Set testDataSets = new HashSet<>(); + + for (TestsDataset dataset : CSV_DATASET_MAP.values()) { + if ((inferenceEnabled || dataset.requiresInferenceEndpoint == false) + && (supportsIndexModeLookup || isLookupDataset(dataset) == false)) { + testDataSets.add(dataset); + } + } + + return testDataSets; + } + + public static boolean isLookupDataset(TestsDataset dataset) throws IOException { + Settings settings = dataset.readSettingsFile(); + String mode = settings.get("index.mode"); + return (mode != null && mode.equalsIgnoreCase("lookup")); } public static void loadDataSetIntoEs(RestClient client, boolean supportsIndexModeLookup) throws IOException { @@ -354,13 +364,8 @@ private static void load(RestClient client, TestsDataset dataset, Logger logger, if (data == null) { throw new IllegalArgumentException("Cannot find resource " + dataName); } - Settings indexSettings = Settings.EMPTY; - final String settingName = dataset.settingFileName != null ? "/" + dataset.settingFileName : null; - if (settingName != null) { - indexSettings = Settings.builder() - .loadFromStream(settingName, CsvTestsDataLoader.class.getResourceAsStream(settingName), false) - .build(); - } + + Settings indexSettings = dataset.readSettingsFile(); indexCreator.createIndex(client, dataset.indexName, readMappingFile(mapping, dataset.typeMapping), indexSettings); loadCsvData(client, dataset.indexName, data, dataset.allowSubFields, logger); } @@ -669,6 +674,18 @@ public TestsDataset withTypeMapping(Map typeMapping) { public TestsDataset withInferenceEndpoint(boolean needsInference) { return new TestsDataset(indexName, mappingFileName, dataFileName, settingFileName, allowSubFields, typeMapping, needsInference); } + + private Settings readSettingsFile() throws IOException { + Settings indexSettings = Settings.EMPTY; + final String settingName = settingFileName != null ? "/" + settingFileName : null; + if (settingName != null) { + indexSettings = Settings.builder() + .loadFromStream(settingName, CsvTestsDataLoader.class.getResourceAsStream(settingName), false) + .build(); + } + + return indexSettings; + } } public record EnrichConfig(String policyName, String policyFileName) {} From b461baf1965e7da69c92bca8c97f50ef62bb8c53 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 16 Dec 2024 12:46:36 +0100 Subject: [PATCH 03/44] Skip translog creation and Lucene commits when recovering searchable snapshot shards (#118606) In order to leverage Lucene N-2 version support for searchable snapshots, we'd like to avoid executing Lucene commits during searchable snapshots shards recovery. This is because Lucene commits require to open an IndexWriter, something that Lucene does not support for N-2 versions. Today when searchable snapshot shards are recovering they create a translog on disk as well as a Lucene commit: - the translog is created as an empty translog with a new UUID and an initial global checkpoint value that is the same as the LOCAL_CHECKPOINT_KEY stored in the last Lucene commit data from the snapshot. - a Lucene commit is executed to associate the translog with the Lucene index by storing new translog UUID in the Lucene commit data. - later during recovery, the replication tracker is initialized with a global checkpoint value equals to the LOCAL_CHECKPOINT_KEY stored in the Lucene commit. We can skip the creation of the translog because searchable snapshot shard do not need one, and it's only use to store the local checkpoint locally to be read later during recovery. If we don't have a translog then we don't need to associate it with the Lucene index, so we can skip the commit too. This change introduce an hasTranslog method that is used to know when it is safe to NOT create a translog, in which case the global checkpoint is read from the last Lucene commit during primary shard recovery from snapshot, peer-recovery and recovery from existing store. In case an existing translog exist on disk, it will be cleaned up. They are also few discoveries around some assertions introduced with snapshot based recoveries, as well as a cached estimation of the size of directories that was refreshed due to Lucene commit but now requires to be "marked as stale". --- .../index/engine/NoOpEngine.java | 2 +- .../index/engine/ReadOnlyEngine.java | 3 +- .../elasticsearch/index/shard/IndexShard.java | 43 ++++++++++++--- .../index/shard/StoreRecovery.java | 53 ++++++++++++------- .../index/store/ByteSizeCachingDirectory.java | 27 ++++++++-- .../org/elasticsearch/index/store/Store.java | 2 + .../index/translog/Translog.java | 4 ++ .../index/translog/TranslogConfig.java | 9 ++++ .../recovery/PeerRecoveryTargetService.java | 15 ++---- .../indices/recovery/RecoveryTarget.java | 39 ++++++++++---- .../xpack/lucene/bwc/OldLuceneVersions.java | 8 +++ .../xpack/lucene/bwc/OldSegmentInfos.java | 4 ++ .../BaseSearchableSnapshotsIntegTestCase.java | 19 ++++--- .../FrozenSearchableSnapshotsIntegTests.java | 10 +--- .../SearchableSnapshotIndexEventListener.java | 7 +++ 15 files changed, 175 insertions(+), 70 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java index 8dee39f7050cb..49e0ae0587085 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/NoOpEngine.java @@ -48,7 +48,7 @@ public final class NoOpEngine extends ReadOnlyEngine { public NoOpEngine(EngineConfig config) { this( config, - config.isPromotableToPrimary() ? null : new TranslogStats(0, 0, 0, 0, 0), + config.isPromotableToPrimary() && config.getTranslogConfig().hasTranslog() ? null : new TranslogStats(0, 0, 0, 0, 0), config.isPromotableToPrimary() ? null : new SeqNoStats( diff --git a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index c1d11223fa55e..1d032c1f400ef 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -98,7 +98,7 @@ public class ReadOnlyEngine extends Engine { public ReadOnlyEngine( EngineConfig config, SeqNoStats seqNoStats, - TranslogStats translogStats, + @Nullable TranslogStats translogStats, boolean obtainLock, Function readerWrapperFunction, boolean requireCompleteHistory, @@ -251,6 +251,7 @@ private static SeqNoStats buildSeqNoStats(EngineConfig config, SegmentInfos info } private static TranslogStats translogStats(final EngineConfig config, final SegmentInfos infos) throws IOException { + assert config.getTranslogConfig().hasTranslog(); final String translogUuid = infos.getUserData().get(Translog.TRANSLOG_UUID_KEY); if (translogUuid == null) { throw new IllegalStateException("commit doesn't contain translog unique id"); 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 a76feff84e61b..966764d2797c9 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1487,6 +1487,27 @@ public void flush(FlushRequest request, ActionListener listener) { }); } + /** + * @return true the shard has a translog. + */ + public boolean hasTranslog() { + return translogConfig.hasTranslog(); + } + + /** + * Reads the global checkpoint from the translog checkpoint file if the shard has a translog. Otherwise, reads the local checkpoint from + * the provided commit user data. + * + * @return the global checkpoint to use for recovery + * @throws IOException + */ + public long readGlobalCheckpointForRecovery(Map commitUserData) throws IOException { + if (hasTranslog()) { + return Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), commitUserData.get(Translog.TRANSLOG_UUID_KEY)); + } + return Long.parseLong(commitUserData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + } + /** * checks and removes translog files that no longer need to be retained. See * {@link org.elasticsearch.index.translog.TranslogDeletionPolicy} for details @@ -1859,8 +1880,7 @@ public void recoverLocallyUpToGlobalCheckpoint(ActionListener recoveryStar } assert routingEntry().recoverySource().getType() == RecoverySource.Type.PEER : "not a peer recovery [" + routingEntry() + "]"; try { - final var translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); - final var globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); + final var globalCheckpoint = readGlobalCheckpointForRecovery(store.readLastCommittedSegmentsInfo().getUserData()); final var safeCommit = store.findSafeIndexCommit(globalCheckpoint); ActionListener.run(recoveryStartingSeqNoListener.delegateResponse((l, e) -> { logger.debug(() -> format("failed to recover shard locally up to global checkpoint %s", globalCheckpoint), e); @@ -2084,8 +2104,7 @@ private void loadGlobalCheckpointToReplicationTracker() throws IOException { // we have to set it before we open an engine and recover from the translog because // acquiring a snapshot from the translog causes a sync which causes the global checkpoint to be pulled in, // and an engine can be forced to close in ctor which also causes the global checkpoint to be pulled in. - final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); - final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); + final long globalCheckpoint = readGlobalCheckpointForRecovery(store.readLastCommittedSegmentsInfo().getUserData()); replicationTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "read from translog checkpoint"); } else { replicationTracker.updateGlobalCheckpointOnReplica(globalCheckPointIfUnpromotable, "from CleanFilesRequest"); @@ -2162,7 +2181,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) t // time elapses after the engine is created above (pulling the config settings) until we set the engine reference, during // which settings changes could possibly have happened, so here we forcefully push any config changes to the new engine. onSettingsChanged(); - assert assertSequenceNumbersInCommit(); + assert assertLastestCommitUserData(); recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); checkAndCallWaitForEngineOrClosedShardListeners(); } @@ -2183,9 +2202,13 @@ private Engine createEngine(EngineConfig config) { } } - private boolean assertSequenceNumbersInCommit() throws IOException { + /** + * Asserts that the latest Lucene commit contains expected information about sequence numbers or ES version. + */ + private boolean assertLastestCommitUserData() throws IOException { final SegmentInfos segmentCommitInfos = SegmentInfos.readLatestCommit(store.directory()); final Map userData = segmentCommitInfos.getUserData(); + // Ensure sequence numbers are present in commit data assert userData.containsKey(SequenceNumbers.LOCAL_CHECKPOINT_KEY) : "commit point doesn't contains a local checkpoint"; assert userData.containsKey(SequenceNumbers.MAX_SEQ_NO) : "commit point doesn't contains a maximum sequence number"; assert userData.containsKey(Engine.HISTORY_UUID_KEY) : "commit point doesn't contains a history uuid"; @@ -2195,10 +2218,16 @@ private boolean assertSequenceNumbersInCommit() throws IOException { + "] is different than engine [" + getHistoryUUID() + "]"; + assert userData.containsKey(Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID) : "opening index which was created post 5.5.0 but " + Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID + " is not found in commit"; + + // From 7.16.0, the ES version is included in the Lucene commit user data as well as in the snapshot metadata in the repository. + // This is used during primary failover to detect if the latest snapshot can be used to recover the new primary, because the failed + // primary may have created new segments in a more recent Lucene version, that may have been later snapshotted, meaning that the + // snapshotted files cannot be recovered on a node with a less recent Lucene version. Note that for versions <= 7.15 this assertion + // relies in the previous minor having a different lucene version. final org.apache.lucene.util.Version commitLuceneVersion = segmentCommitInfos.getCommitLuceneVersion(); - // This relies in the previous minor having another lucene version assert commitLuceneVersion.onOrAfter(RecoverySettings.SEQ_NO_SNAPSHOT_RECOVERIES_SUPPORTED_VERSION.luceneVersion()) == false || userData.containsKey(Engine.ES_VERSION) && Engine.readIndexVersion(userData.get(Engine.ES_VERSION)).onOrBefore(IndexVersion.current()) diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index 42f62cf86545b..06f9b3e6c8943 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -315,7 +315,7 @@ void recoverFromRepository(final IndexShard indexShard, Repository repository, A RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType(); assert recoveryType == RecoverySource.Type.SNAPSHOT : "expected snapshot recovery type: " + recoveryType; SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource(); - restore(indexShard, repository, recoverySource, recoveryListener(indexShard, listener).map(ignored -> true)); + recoverFromRepository(indexShard, repository, recoverySource, recoveryListener(indexShard, listener).map(ignored -> true)); } else { listener.onResponse(false); } @@ -459,7 +459,7 @@ private void internalRecoverFromStore(IndexShard indexShard, ActionListener outerListener ) { + assert indexShard.shardRouting.primary() : "only primary shards can recover from snapshot"; logger.debug("restoring from {} ...", indexShard.recoveryState().getRecoverySource()); record ShardAndIndexIds(IndexId indexId, ShardId shardId) {} @@ -538,13 +539,13 @@ record ShardAndIndexIds(IndexId indexId, ShardId shardId) {} .newForked(indexShard::preRecovery) .andThen(shardAndIndexIdsListener -> { - final RecoveryState.Translog translogState = indexShard.recoveryState().getTranslog(); if (restoreSource == null) { throw new IndexShardRestoreFailedException(shardId, "empty restore source"); } if (logger.isTraceEnabled()) { logger.trace("[{}] restoring shard [{}]", restoreSource.snapshot(), shardId); } + final RecoveryState.Translog translogState = indexShard.recoveryState().getTranslog(); translogState.totalOperations(0); translogState.totalOperationsOnStart(0); indexShard.prepareForIndexRecovery(); @@ -588,9 +589,7 @@ record ShardAndIndexIds(IndexId indexId, ShardId shardId) {} .andThen(l -> { indexShard.getIndexEventListener().afterFilesRestoredFromRepository(indexShard); - final Store store = indexShard.store(); - bootstrap(indexShard, store); - assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; + bootstrap(indexShard); writeEmptyRetentionLeasesFile(indexShard); indexShard.openEngineAndRecoverFromTranslog(l); }) @@ -610,19 +609,37 @@ record ShardAndIndexIds(IndexId indexId, ShardId shardId) {} })); } + /** + * @deprecated use {@link #bootstrap(IndexShard)} instead + */ + @Deprecated(forRemoval = true) public static void bootstrap(final IndexShard indexShard, final Store store) throws IOException { - if (indexShard.indexSettings.getIndexMetadata().isSearchableSnapshot() == false) { - // not bootstrapping new history for searchable snapshots (which are read-only) allows sequence-number based peer recoveries + assert indexShard.store() == store; + bootstrap(indexShard); + } + + private static void bootstrap(final IndexShard indexShard) throws IOException { + assert indexShard.routingEntry().primary(); + final var store = indexShard.store(); + store.incRef(); + try { + final var translogLocation = indexShard.shardPath().resolveTranslog(); + if (indexShard.hasTranslog() == false) { + Translog.deleteAll(translogLocation); + return; + } store.bootstrapNewHistory(); + final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); + final long localCheckpoint = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + final String translogUUID = Translog.createEmptyTranslog( + translogLocation, + localCheckpoint, + indexShard.shardId(), + indexShard.getPendingPrimaryTerm() + ); + store.associateIndexWithNewTranslog(translogUUID); + } finally { + store.decRef(); } - final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); - final long localCheckpoint = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); - final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), - localCheckpoint, - indexShard.shardId(), - indexShard.getPendingPrimaryTerm() - ); - store.associateIndexWithNewTranslog(translogUUID); } } diff --git a/server/src/main/java/org/elasticsearch/index/store/ByteSizeCachingDirectory.java b/server/src/main/java/org/elasticsearch/index/store/ByteSizeCachingDirectory.java index 166f0eadc62b4..033859bd62128 100644 --- a/server/src/main/java/org/elasticsearch/index/store/ByteSizeCachingDirectory.java +++ b/server/src/main/java/org/elasticsearch/index/store/ByteSizeCachingDirectory.java @@ -10,6 +10,7 @@ package org.elasticsearch.index.store; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexOutput; import org.elasticsearch.common.lucene.store.FilterIndexOutput; @@ -19,7 +20,7 @@ import java.io.IOException; import java.io.UncheckedIOException; -final class ByteSizeCachingDirectory extends ByteSizeDirectory { +public final class ByteSizeCachingDirectory extends ByteSizeDirectory { private static class SizeAndModCount { final long size; @@ -174,9 +175,29 @@ public void deleteFile(String name) throws IOException { try { super.deleteFile(name); } finally { - synchronized (this) { - modCount++; + markEstimatedSizeAsStale(); + } + } + + /** + * Mark the cached size as stale so that it is guaranteed to be refreshed the next time. + */ + public void markEstimatedSizeAsStale() { + synchronized (this) { + modCount++; + } + } + + public static ByteSizeCachingDirectory unwrapDirectory(Directory dir) { + while (dir != null) { + if (dir instanceof ByteSizeCachingDirectory) { + return (ByteSizeCachingDirectory) dir; + } else if (dir instanceof FilterDirectory) { + dir = ((FilterDirectory) dir).getDelegate(); + } else { + dir = null; } } + return null; } } diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index e6b499c07f189..322064f09cf77 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -1449,6 +1449,7 @@ public void bootstrapNewHistory() throws IOException { * @see SequenceNumbers#MAX_SEQ_NO */ public void bootstrapNewHistory(long localCheckpoint, long maxSeqNo) throws IOException { + assert indexSettings.getIndexMetadata().isSearchableSnapshot() == false; metadataLock.writeLock().lock(); try (IndexWriter writer = newTemporaryAppendingIndexWriter(directory, null)) { final Map map = new HashMap<>(); @@ -1572,6 +1573,7 @@ private IndexWriter newTemporaryEmptyIndexWriter(final Directory dir, final Vers } private IndexWriterConfig newTemporaryIndexWriterConfig() { + assert indexSettings.getIndexMetadata().isSearchableSnapshot() == false; // this config is only used for temporary IndexWriter instances, used to initialize the index or update the commit data, // so we don't want any merges to happen var iwc = indexWriterConfigWithNoMerging(null).setSoftDeletesField(Lucene.SOFT_DELETES_FIELD).setCommitOnClose(false); diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 2d3e2d8c20256..75f38ed5f7342 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -220,6 +220,10 @@ public Translog( } } + public static void deleteAll(Path translogLocation) throws IOException { + IOUtils.rm(translogLocation); + } + /** recover all translog files found on disk */ private ArrayList recoverFromFiles(Checkpoint checkpoint) throws IOException { boolean success = false; diff --git a/server/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java b/server/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java index 66018092089ce..280e319335b12 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java +++ b/server/src/main/java/org/elasticsearch/index/translog/TranslogConfig.java @@ -143,4 +143,13 @@ public OperationListener getOperationListener() { public boolean fsync() { return fsync; } + + /** + * @return {@code true} if the configuration allows the Translog files to exist, {@code false} otherwise. In the case there is no + * translog, the shard is not writeable. + */ + public boolean hasTranslog() { + // Expect no translog files to exist for searchable snapshots + return false == indexSettings.getIndexMetadata().isSearchableSnapshot(); + } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index c8d31d2060caf..d069717a66ad0 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -49,9 +49,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardLongFieldRange; import org.elasticsearch.index.shard.ShardNotFoundException; -import org.elasticsearch.index.shard.StoreRecovery; import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogCorruptedException; import org.elasticsearch.indices.recovery.RecoveriesCollection.RecoveryRef; import org.elasticsearch.tasks.Task; @@ -385,15 +383,8 @@ record StartRecoveryRequestToSend(StartRecoveryRequest startRecoveryRequest, Str logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); indexShard.prepareForIndexRecovery(); if (indexShard.indexSettings().getIndexMetadata().isSearchableSnapshot()) { - // for searchable snapshots, peer recovery is treated similarly to recovery from snapshot + // for archives indices mounted as searchable snapshots, we need to call this indexShard.getIndexEventListener().afterFilesRestoredFromRepository(indexShard); - final Store store = indexShard.store(); - store.incRef(); - try { - StoreRecovery.bootstrap(indexShard, store); - } finally { - store.decRef(); - } } indexShard.recoverLocallyUpToGlobalCheckpoint(ActionListener.assertOnce(l)); }) @@ -488,8 +479,8 @@ public static StartRecoveryRequest getStartRecoveryRequest( // Make sure that the current translog is consistent with the Lucene index; otherwise, we have to throw away the Lucene // index. try { - final String expectedTranslogUUID = metadataSnapshot.commitUserData().get(Translog.TRANSLOG_UUID_KEY); - final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation(), expectedTranslogUUID); + final long globalCheckpoint = recoveryTarget.indexShard() + .readGlobalCheckpointForRecovery(metadataSnapshot.commitUserData()); assert globalCheckpoint + 1 >= startingSeqNo : "invalid startingSeqNo " + startingSeqNo + " >= " + globalCheckpoint; } catch (IOException | TranslogCorruptedException e) { logGlobalCheckpointWarning(logger, startingSeqNo, e); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index ea485a411143e..362a62c838e3b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -45,7 +45,6 @@ import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; -import java.nio.file.Path; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -516,13 +515,7 @@ public void cleanFiles( try { if (indexShard.routingEntry().isPromotableToPrimary()) { store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetadata); - final String translogUUID = Translog.createEmptyTranslog( - indexShard.shardPath().resolveTranslog(), - globalCheckpoint, - shardId, - indexShard.getPendingPrimaryTerm() - ); - store.associateIndexWithNewTranslog(translogUUID); + bootstrap(indexShard, globalCheckpoint); } else { indexShard.setGlobalCheckpointIfUnpromotable(globalCheckpoint); } @@ -634,7 +627,33 @@ public String getTempNameForFile(String origFile) { return multiFileWriter.getTempNameForFile(origFile); } - Path translogLocation() { - return indexShard().shardPath().resolveTranslog(); + private static void bootstrap(final IndexShard indexShard, long globalCheckpoint) throws IOException { + assert indexShard.routingEntry().isPromotableToPrimary(); + final var store = indexShard.store(); + store.incRef(); + try { + final var translogLocation = indexShard.shardPath().resolveTranslog(); + if (indexShard.hasTranslog() == false) { + if (Assertions.ENABLED) { + if (indexShard.indexSettings().getIndexMetadata().isSearchableSnapshot()) { + long localCheckpoint = Long.parseLong( + store.readLastCommittedSegmentsInfo().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY) + ); + assert localCheckpoint == globalCheckpoint : localCheckpoint + " != " + globalCheckpoint; + } + } + Translog.deleteAll(translogLocation); + return; + } + final String translogUUID = Translog.createEmptyTranslog( + indexShard.shardPath().resolveTranslog(), + globalCheckpoint, + indexShard.shardId(), + indexShard.getPendingPrimaryTerm() + ); + store.associateIndexWithNewTranslog(translogUUID); + } finally { + store.decRef(); + } } } diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java index 42fe09691d249..e36ae4994c872 100644 --- a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java +++ b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldLuceneVersions.java @@ -27,6 +27,7 @@ import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.IndexVersions; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.ReadOnlyEngine; @@ -34,6 +35,7 @@ import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.TranslogStats; +import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.license.License; import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.license.LicensedFeature; @@ -201,6 +203,12 @@ private static SegmentInfos convertToNewerLuceneVersion(OldSegmentInfos oldSegme if (map.containsKey(Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID) == false) { map.put(Engine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, "-1"); } + if (map.containsKey(Engine.ES_VERSION) == false) { + assert oldSegmentInfos.getLuceneVersion() + .onOrAfter(RecoverySettings.SEQ_NO_SNAPSHOT_RECOVERIES_SUPPORTED_VERSION.luceneVersion()) == false + : oldSegmentInfos.getLuceneVersion() + " should contain the ES_VERSION"; + map.put(Engine.ES_VERSION, IndexVersions.MINIMUM_COMPATIBLE.toString()); + } segmentInfos.setUserData(map, false); for (SegmentCommitInfo infoPerCommit : oldSegmentInfos.asList()) { final SegmentInfo newInfo = BWCCodec.wrap(infoPerCommit.info); diff --git a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldSegmentInfos.java b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldSegmentInfos.java index 18adebb145f98..b2af41653da61 100644 --- a/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldSegmentInfos.java +++ b/x-pack/plugin/old-lucene-versions/src/main/java/org/elasticsearch/xpack/lucene/bwc/OldSegmentInfos.java @@ -564,6 +564,10 @@ public long getLastGeneration() { return lastGeneration; } + public Version getLuceneVersion() { + return luceneVersion; + } + /** * Prints the given message to the infoStream. Note, this method does not check for null * infoStream. It assumes this check has been performed by the caller, which is recommended to diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java index 6115bec91ad62..a3ced0bf1b607 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java @@ -74,7 +74,6 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; @ESIntegTestCase.ClusterScope(supportsDedicatedMasters = false, numClientNodes = 0) @@ -241,7 +240,7 @@ protected void checkSoftDeletesNotEagerlyLoaded(String restoredIndexName) { } } - protected void assertShardFolders(String indexName, boolean snapshotDirectory) throws IOException { + protected void assertShardFolders(String indexName, boolean isSearchableSnapshot) throws IOException { final Index restoredIndex = resolveIndex(indexName); final String customDataPath = resolveCustomDataPath(indexName); final ShardId shardId = new ShardId(restoredIndex, 0); @@ -261,16 +260,16 @@ protected void assertShardFolders(String indexName, boolean snapshotDirectory) t translogExists ); assertThat( - snapshotDirectory ? "Index file should not exist" : "Index file should exist", + isSearchableSnapshot ? "Index file should not exist" : "Index file should exist", indexExists, - not(snapshotDirectory) + not(isSearchableSnapshot) ); - assertThat("Translog should exist", translogExists, is(true)); - try (Stream dir = Files.list(shardPath.resolveTranslog())) { - final long translogFiles = dir.filter(path -> path.getFileName().toString().contains("translog")).count(); - if (snapshotDirectory) { - assertThat("There should be 2 translog files for a snapshot directory", translogFiles, equalTo(2L)); - } else { + if (isSearchableSnapshot) { + assertThat("Translog should not exist", translogExists, equalTo(false)); + } else { + assertThat("Translog should exist", translogExists, equalTo(true)); + try (Stream dir = Files.list(shardPath.resolveTranslog())) { + final long translogFiles = dir.filter(path -> path.getFileName().toString().contains("translog")).count(); assertThat( "There should be 2+ translog files for a non-snapshot directory", translogFiles, diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java index 67d9d7a82acf3..2797202e5f24e 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/FrozenSearchableSnapshotsIntegTests.java @@ -59,7 +59,6 @@ import java.time.ZoneId; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Map; @@ -169,12 +168,9 @@ public void testCreateAndRestorePartialSearchableSnapshot() throws Exception { logger.info("--> restoring partial index [{}] with cache enabled", restoredIndexName); Settings.Builder indexSettingsBuilder = Settings.builder().put(SearchableSnapshots.SNAPSHOT_CACHE_ENABLED_SETTING.getKey(), true); - final List nonCachedExtensions; if (randomBoolean()) { - nonCachedExtensions = randomSubsetOf(Arrays.asList("fdt", "fdx", "nvd", "dvd", "tip", "cfs", "dim")); + var nonCachedExtensions = randomSubsetOf(Arrays.asList("fdt", "fdx", "nvd", "dvd", "tip", "cfs", "dim")); indexSettingsBuilder.putList(SearchableSnapshots.SNAPSHOT_CACHE_EXCLUDED_FILE_TYPES_SETTING.getKey(), nonCachedExtensions); - } else { - nonCachedExtensions = Collections.emptyList(); } if (randomBoolean()) { indexSettingsBuilder.put( @@ -264,8 +260,6 @@ public void testCreateAndRestorePartialSearchableSnapshot() throws Exception { final long originalSize = snapshotShards.get(shardRouting.getId()).getStats().getTotalSize(); totalExpectedSize += originalSize; - // an extra segments_N file is created for bootstrapping new history and associating translog. We can extract the size of this - // extra file but we have to unwrap the in-memory directory first. final Directory unwrappedDir = FilterDirectory.unwrap( internalCluster().getInstance(IndicesService.class, getDiscoveryNodes().resolveNode(shardRouting.currentNodeId()).getName()) .indexServiceSafe(shardRouting.index()) @@ -277,7 +271,7 @@ public void testCreateAndRestorePartialSearchableSnapshot() throws Exception { assertThat(shardRouting.toString(), unwrappedDir, instanceOf(ByteBuffersDirectory.class)); final ByteBuffersDirectory inMemoryDir = (ByteBuffersDirectory) unwrappedDir; - assertThat(inMemoryDir.listAll(), arrayWithSize(1)); + assertThat(inMemoryDir.listAll(), arrayWithSize(0)); assertThat(shardRouting.toString(), store.totalDataSetSizeInBytes(), equalTo(originalSize)); } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotIndexEventListener.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotIndexEventListener.java index cf0306e3e6ef2..4caf932a99807 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotIndexEventListener.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/SearchableSnapshotIndexEventListener.java @@ -21,6 +21,7 @@ import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.ByteSizeCachingDirectory; import org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.searchablesnapshots.SearchableSnapshots; @@ -61,6 +62,12 @@ public SearchableSnapshotIndexEventListener( public void beforeIndexShardRecovery(IndexShard indexShard, IndexSettings indexSettings, ActionListener listener) { assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.GENERIC); ensureSnapshotIsLoaded(indexShard); + var sizeCachingDirectory = ByteSizeCachingDirectory.unwrapDirectory(indexShard.store().directory()); + if (sizeCachingDirectory != null) { + // Marks the cached estimation of the directory size as stale in ByteSizeCachingDirectory since we just loaded the snapshot + // files list into the searchable snapshot directory. + sizeCachingDirectory.markEstimatedSizeAsStale(); + } listener.onResponse(null); } From caf8afcb2d412051e6c19ae9ff223d1a396f8aaf Mon Sep 17 00:00:00 2001 From: Ievgen Degtiarenko Date: Mon, 16 Dec 2024 13:36:18 +0100 Subject: [PATCH 04/44] Render all datatypes in pipetables error output (#118758) Some types (such as ip or time) require custom formatting logic and were not correctly rendered in actual/expected comparison table. This change: * properly formats such data types * add data types to the column headers * removes outer pipes --- .../elasticsearch/xpack/esql/CsvAssert.java | 152 +++++++++++------- 1 file changed, 98 insertions(+), 54 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java index 8a4d44a690571..692c385cef216 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/CsvAssert.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.compute.data.Page; import org.elasticsearch.logging.Logger; @@ -197,7 +198,13 @@ public static void assertData( for (int row = 0; row < expectedValues.size(); row++) { try { if (row >= actualValues.size()) { - dataFailure("Expected more data but no more entries found after [" + row + "]", dataFailures, expected, actualValues); + dataFailure( + "Expected more data but no more entries found after [" + row + "]", + dataFailures, + expected, + actualValues, + valueTransformer + ); } if (logger != null) { @@ -208,45 +215,17 @@ public static void assertData( var actualRow = actualValues.get(row); for (int column = 0; column < expectedRow.size(); column++) { - var expectedValue = expectedRow.get(column); - var actualValue = actualRow.get(column); var expectedType = expected.columnTypes().get(column); + var expectedValue = convertExpectedValue(expectedType, expectedRow.get(column)); + var actualValue = actualRow.get(column); - if (expectedValue != null) { - // convert the long from CSV back to its STRING form - if (expectedType == Type.DATETIME) { - expectedValue = rebuildExpected(expectedValue, Long.class, x -> UTC_DATE_TIME_FORMATTER.formatMillis((long) x)); - } else if (expectedType == Type.DATE_NANOS) { - expectedValue = rebuildExpected( - expectedValue, - Long.class, - x -> DateFormatter.forPattern("strict_date_optional_time_nanos").formatNanos((long) x) - ); - } else if (expectedType == Type.GEO_POINT) { - expectedValue = rebuildExpected(expectedValue, BytesRef.class, x -> GEO.wkbToWkt((BytesRef) x)); - } else if (expectedType == Type.CARTESIAN_POINT) { - expectedValue = rebuildExpected(expectedValue, BytesRef.class, x -> CARTESIAN.wkbToWkt((BytesRef) x)); - } else if (expectedType == Type.GEO_SHAPE) { - expectedValue = rebuildExpected(expectedValue, BytesRef.class, x -> GEO.wkbToWkt((BytesRef) x)); - } else if (expectedType == Type.CARTESIAN_SHAPE) { - expectedValue = rebuildExpected(expectedValue, BytesRef.class, x -> CARTESIAN.wkbToWkt((BytesRef) x)); - } else if (expectedType == Type.IP) { - // convert BytesRef-packed IP to String, allowing subsequent comparison with what's expected - expectedValue = rebuildExpected(expectedValue, BytesRef.class, x -> DocValueFormat.IP.format((BytesRef) x)); - } else if (expectedType == Type.VERSION) { - // convert BytesRef-packed Version to String - expectedValue = rebuildExpected(expectedValue, BytesRef.class, x -> new Version((BytesRef) x).toString()); - } else if (expectedType == UNSIGNED_LONG) { - expectedValue = rebuildExpected(expectedValue, Long.class, x -> unsignedLongAsNumber((long) x)); - } - } var transformedExpected = valueTransformer.apply(expectedType, expectedValue); var transformedActual = valueTransformer.apply(expectedType, actualValue); if (Objects.equals(transformedExpected, transformedActual) == false) { dataFailures.add(new DataFailure(row, column, transformedExpected, transformedActual)); } if (dataFailures.size() > 10) { - dataFailure("", dataFailures, expected, actualValues); + dataFailure("", dataFailures, expected, actualValues, valueTransformer); } } @@ -255,7 +234,8 @@ public static void assertData( "Plan has extra columns, returned [" + actualRow.size() + "], expected [" + expectedRow.size() + "]", dataFailures, expected, - actualValues + actualValues, + valueTransformer ); } } catch (AssertionError ae) { @@ -267,10 +247,16 @@ public static void assertData( } } if (dataFailures.isEmpty() == false) { - dataFailure("", dataFailures, expected, actualValues); + dataFailure("", dataFailures, expected, actualValues, valueTransformer); } if (expectedValues.size() < actualValues.size()) { - dataFailure("Elasticsearch still has data after [" + expectedValues.size() + "] entries", dataFailures, expected, actualValues); + dataFailure( + "Elasticsearch still has data after [" + expectedValues.size() + "] entries", + dataFailures, + expected, + actualValues, + valueTransformer + ); } } @@ -278,42 +264,72 @@ private static void dataFailure( String description, List dataFailures, ExpectedResults expectedValues, - List> actualValues + List> actualValues, + BiFunction valueTransformer ) { - var expected = pipeTable("Expected:", expectedValues.columnNames(), expectedValues.values(), 25); - var actual = pipeTable("Actual:", expectedValues.columnNames(), actualValues, 25); + var expected = pipeTable( + "Expected:", + expectedValues.columnNames(), + expectedValues.columnTypes(), + expectedValues.values(), + (type, value) -> valueTransformer.apply(type, convertExpectedValue(type, value)) + ); + var actual = pipeTable("Actual:", expectedValues.columnNames(), expectedValues.columnTypes(), actualValues, valueTransformer); fail(description + System.lineSeparator() + describeFailures(dataFailures) + actual + expected); } - private static String pipeTable(String description, List headers, List> values, int maxRows) { + private static final int MAX_ROWS = 25; + + private static String pipeTable( + String description, + List headers, + List types, + List> values, + BiFunction valueTransformer + ) { + int rows = Math.min(MAX_ROWS, values.size()); int[] width = new int[headers.size()]; - for (int i = 0; i < width.length; i++) { - width[i] = headers.get(i).length(); - for (List row : values) { - width[i] = Math.max(width[i], String.valueOf(row.get(i)).length()); + String[][] printableValues = new String[rows][headers.size()]; + for (int c = 0; c < headers.size(); c++) { + width[c] = header(headers.get(c), types.get(c)).length(); + } + for (int r = 0; r < rows; r++) { + for (int c = 0; c < headers.size(); c++) { + printableValues[r][c] = String.valueOf(valueTransformer.apply(types.get(c), values.get(r).get(c))); + width[c] = Math.max(width[c], printableValues[r][c].length()); } } var result = new StringBuilder().append(System.lineSeparator()).append(description).append(System.lineSeparator()); - for (int c = 0; c < width.length; c++) { - appendValue(result, headers.get(c), width[c]); + // headers + appendPaddedValue(result, header(headers.get(0), types.get(0)), width[0]); + for (int c = 1; c < width.length; c++) { + result.append(" | "); + appendPaddedValue(result, header(headers.get(c), types.get(c)), width[c]); } - result.append('|').append(System.lineSeparator()); - for (int r = 0; r < Math.min(maxRows, values.size()); r++) { - for (int c = 0; c < width.length; c++) { - appendValue(result, values.get(r).get(c), width[c]); + result.append(System.lineSeparator()); + // values + for (int r = 0; r < printableValues.length; r++) { + appendPaddedValue(result, printableValues[r][0], width[0]); + for (int c = 1; c < printableValues[r].length; c++) { + result.append(" | "); + appendPaddedValue(result, printableValues[r][c], width[c]); } - result.append('|').append(System.lineSeparator()); + result.append(System.lineSeparator()); } - if (values.size() > maxRows) { + if (values.size() > rows) { result.append("...").append(System.lineSeparator()); } return result.toString(); } - private static void appendValue(StringBuilder result, Object value, int width) { - result.append('|').append(value); - for (int i = 0; i < width - String.valueOf(value).length(); i++) { + private static String header(String name, Type type) { + return name + ':' + Strings.toLowercaseAscii(type.name()); + } + + private static void appendPaddedValue(StringBuilder result, String value, int width) { + result.append(value); + for (int i = 0; i < width - (value != null ? value.length() : 4); i++) { result.append(' '); } } @@ -369,6 +385,34 @@ private static Comparator> resultRowComparator(List types) { }; } + private static Object convertExpectedValue(Type expectedType, Object expectedValue) { + if (expectedValue == null) { + return null; + } + + // convert the long from CSV back to its STRING form + return switch (expectedType) { + case Type.DATETIME -> rebuildExpected(expectedValue, Long.class, x -> UTC_DATE_TIME_FORMATTER.formatMillis((long) x)); + case Type.DATE_NANOS -> rebuildExpected( + expectedValue, + Long.class, + x -> DateFormatter.forPattern("strict_date_optional_time_nanos").formatNanos((long) x) + ); + case Type.GEO_POINT, Type.GEO_SHAPE -> rebuildExpected(expectedValue, BytesRef.class, x -> GEO.wkbToWkt((BytesRef) x)); + case Type.CARTESIAN_POINT, Type.CARTESIAN_SHAPE -> rebuildExpected( + expectedValue, + BytesRef.class, + x -> CARTESIAN.wkbToWkt((BytesRef) x) + ); + case Type.IP -> // convert BytesRef-packed IP to String, allowing subsequent comparison with what's expected + rebuildExpected(expectedValue, BytesRef.class, x -> DocValueFormat.IP.format((BytesRef) x)); + case Type.VERSION -> // convert BytesRef-packed Version to String + rebuildExpected(expectedValue, BytesRef.class, x -> new Version((BytesRef) x).toString()); + case UNSIGNED_LONG -> rebuildExpected(expectedValue, Long.class, x -> unsignedLongAsNumber((long) x)); + default -> expectedValue; + }; + } + private static Object rebuildExpected(Object expectedValue, Class clazz, Function mapper) { if (List.class.isAssignableFrom(expectedValue.getClass())) { assertThat(((List) expectedValue).get(0), instanceOf(clazz)); From 709a87e8013503097d55787a8950da34158b6cb1 Mon Sep 17 00:00:00 2001 From: Joan Fontanals Date: Mon, 16 Dec 2024 14:05:30 +0100 Subject: [PATCH 05/44] Handle `null` text values in RankedDocsResults.asMap() (#118597) --- .../core/inference/results/RankedDocsResults.java | 6 +++++- .../inference/results/RankedDocsResultsTests.java | 12 ++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/RankedDocsResults.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/RankedDocsResults.java index 9c764babe33fc..a5f72bd51c6c6 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/RankedDocsResults.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/RankedDocsResults.java @@ -139,7 +139,11 @@ public void writeTo(StreamOutput out) throws IOException { } public Map asMap() { - return Map.of(NAME, Map.of(INDEX, index, RELEVANCE_SCORE, relevanceScore, TEXT, text)); + if (text != null) { + return Map.of(NAME, Map.of(INDEX, index, RELEVANCE_SCORE, relevanceScore, TEXT, text)); + } else { + return Map.of(NAME, Map.of(INDEX, index, RELEVANCE_SCORE, relevanceScore)); + } } @Override diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/results/RankedDocsResultsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/results/RankedDocsResultsTests.java index 46f10928cad08..ff6f6848f4b69 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/results/RankedDocsResultsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/results/RankedDocsResultsTests.java @@ -12,10 +12,12 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xpack.core.ml.AbstractChunkedBWCSerializationTestCase; +import org.hamcrest.Matchers; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; public class RankedDocsResultsTests extends AbstractChunkedBWCSerializationTestCase { @@ -37,6 +39,16 @@ public static RankedDocsResults.RankedDoc createRandomDoc() { return new RankedDocsResults.RankedDoc(randomIntBetween(0, 100), randomFloat(), randomBoolean() ? null : randomAlphaOfLength(10)); } + public void test_asMap() { + var index = randomIntBetween(0, 100); + var score = randomFloat(); + var mapNullText = new RankedDocsResults.RankedDoc(index, score, null).asMap(); + assertThat(mapNullText, Matchers.is(Map.of("ranked_doc", Map.of("index", index, "relevance_score", score)))); + + var mapWithText = new RankedDocsResults.RankedDoc(index, score, "Sample text").asMap(); + assertThat(mapWithText, Matchers.is(Map.of("ranked_doc", Map.of("index", index, "relevance_score", score, "text", "Sample text")))); + } + @Override protected RankedDocsResults mutateInstance(RankedDocsResults instance) throws IOException { List copy = new ArrayList<>(List.copyOf(instance.getRankedDocs())); From 0efdc4741bfb8610528c9db0b2d72030b6a0684b Mon Sep 17 00:00:00 2001 From: Bogdan Pintea Date: Mon, 16 Dec 2024 14:12:39 +0100 Subject: [PATCH 06/44] Disable one more test failing on pre-8.13 BWC (#118760) Prevent `stats.ByStringAndLongWithAlias` from running on pre-8.13 BWC. Related #118655. --- .../esql/qa/testFixtures/src/main/resources/dissect.csv-spec | 4 ++-- .../esql/qa/testFixtures/src/main/resources/grok.csv-spec | 4 ++-- .../esql/qa/testFixtures/src/main/resources/stats.csv-spec | 5 +++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/dissect.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/dissect.csv-spec index cde5427bf37d6..2b3b0bee93471 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/dissect.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/dissect.csv-spec @@ -223,7 +223,7 @@ null | null | null ; -// the query is incorrectly physically plan (fails the verification) in pre-8.13.0 versions +// the query is incorrectly physically planned (fails the verification) in pre-8.13.0 versions overwriteName#[skip:-8.12.99] from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name) | dissect full_name "%{emp_no} %{b}" | keep full_name, emp_no, b | limit 3; @@ -245,7 +245,7 @@ emp_no:integer | first_name:keyword | rest:keyword ; -// the query is incorrectly physically plan (fails the verification) in pre-8.13.0 versions +// the query is incorrectly physically planned (fails the verification) in pre-8.13.0 versions overwriteNameWhere#[skip:-8.12.99] from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name) | dissect full_name "%{emp_no} %{b}" | where emp_no == "Bezalel" | keep full_name, emp_no, b | limit 3; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/grok.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/grok.csv-spec index eece1bdfbffa4..6dc9148ffc0e8 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/grok.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/grok.csv-spec @@ -199,7 +199,7 @@ null | null | null ; -// the query is incorrectly physically plan (fails the verification) in pre-8.13.0 versions +// the query is incorrectly physically planned (fails the verification) in pre-8.13.0 versions overwriteName#[skip:-8.12.99] from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name) | grok full_name "%{WORD:emp_no} %{WORD:b}" | keep full_name, emp_no, b | limit 3; @@ -210,7 +210,7 @@ Parto Bamford | Parto | Bamford ; -// the query is incorrectly physically plan (fails the verification) in pre-8.13.0 versions +// the query is incorrectly physically planned (fails the verification) in pre-8.13.0 versions overwriteNameWhere#[skip:-8.12.99] from employees | sort emp_no asc | eval full_name = concat(first_name, " ", last_name) | grok full_name "%{WORD:emp_no} %{WORD:b}" | where emp_no == "Bezalel" | keep full_name, emp_no, b | limit 3; diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec index 100c0d716d65c..80586ce9bcb09 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/stats.csv-spec @@ -564,7 +564,7 @@ c:long | gender:keyword | trunk_worked_seconds:long 0 | null | 200000000 ; -// the query is incorrectly physically plan (fails the verification) in pre-8.13.0 versions +// the query is incorrectly physically planned (fails the verification) in pre-8.13.0 versions byStringAndLongWithAlias#[skip:-8.12.99] FROM employees | EVAL trunk_worked_seconds = avg_worked_seconds / 100000000 * 100000000 @@ -720,7 +720,8 @@ c:long | d:date | gender:keyword | languages:integer 2 | 1987-01-01T00:00:00.000Z | M | 1 ; -byDateAndKeywordAndIntWithAlias +// the query is incorrectly physically planned (fails the verification) in pre-8.13.0 versions +byDateAndKeywordAndIntWithAlias#[skip:-8.12.99] from employees | eval d = date_trunc(1 year, hire_date) | rename gender as g, languages as l, emp_no as e | keep d, g, l, e | stats c = count(e) by d, g, l | sort c desc, d, l desc, g desc | limit 10; c:long | d:date | g:keyword | l:integer From 2b8c494b3b8dbca07b90dfed8d6751f15631c814 Mon Sep 17 00:00:00 2001 From: Luigi Dell'Aquila Date: Mon, 16 Dec 2024 14:47:03 +0100 Subject: [PATCH 07/44] ES|QL: Fix RLIKE folding with (unsupported) case insensitive pattern (#118454) --- docs/changelog/118454.yaml | 5 +++ .../expression/predicate/regex/RLike.java | 12 +------ .../predicate/regex/RegexMatch.java | 7 +--- .../predicate/regex/WildcardLike.java | 13 +------ .../xpack/esql/core/util/TestUtils.java | 6 ---- .../src/main/resources/eval.csv-spec | 36 +++++++++++++++++++ .../xpack/esql/action/EsqlCapabilities.java | 7 +++- .../function/scalar/string/RLike.java | 7 +++- .../function/scalar/string/WildcardLike.java | 5 +++ .../rules/logical/ConstantFoldingTests.java | 4 +-- .../PushDownAndCombineFiltersTests.java | 2 +- .../rules/logical/ReplaceRegexMatchTests.java | 4 +-- 12 files changed, 66 insertions(+), 42 deletions(-) create mode 100644 docs/changelog/118454.yaml diff --git a/docs/changelog/118454.yaml b/docs/changelog/118454.yaml new file mode 100644 index 0000000000000..9a19ede64d705 --- /dev/null +++ b/docs/changelog/118454.yaml @@ -0,0 +1,5 @@ +pr: 118454 +summary: Fix RLIKE folding with (unsupported) case insensitive pattern +area: ES|QL +type: bug +issues: [] diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/RLike.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/RLike.java index 5f095a654fc89..b4bccf162d9e4 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/RLike.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/RLike.java @@ -8,12 +8,11 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.xpack.esql.core.expression.Expression; -import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; import java.io.IOException; -public class RLike extends RegexMatch { +public abstract class RLike extends RegexMatch { public RLike(Source source, Expression value, RLikePattern pattern) { super(source, value, pattern, false); @@ -33,13 +32,4 @@ public String getWriteableName() { throw new UnsupportedOperationException(); } - @Override - protected NodeInfo info() { - return NodeInfo.create(this, RLike::new, field(), pattern(), caseInsensitive()); - } - - @Override - protected RLike replaceChild(Expression newChild) { - return new RLike(source(), newChild, pattern(), caseInsensitive()); - } } diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/RegexMatch.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/RegexMatch.java index 32e8b04573d2d..0f9116ade5a31 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/RegexMatch.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/RegexMatch.java @@ -7,7 +7,6 @@ package org.elasticsearch.xpack.esql.core.expression.predicate.regex; -import org.apache.lucene.util.BytesRef; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.Nullability; import org.elasticsearch.xpack.esql.core.expression.function.scalar.UnaryScalarFunction; @@ -64,11 +63,7 @@ public boolean foldable() { @Override public Boolean fold() { - Object val = field().fold(); - if (val instanceof BytesRef br) { - val = br.utf8ToString(); - } - return RegexOperation.match(val, pattern().asJavaRegex()); + throw new UnsupportedOperationException(); } @Override diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/WildcardLike.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/WildcardLike.java index bf54744667217..05027707326bd 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/WildcardLike.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/regex/WildcardLike.java @@ -8,12 +8,11 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.xpack.esql.core.expression.Expression; -import org.elasticsearch.xpack.esql.core.tree.NodeInfo; import org.elasticsearch.xpack.esql.core.tree.Source; import java.io.IOException; -public class WildcardLike extends RegexMatch { +public abstract class WildcardLike extends RegexMatch { public WildcardLike(Source source, Expression left, WildcardPattern pattern) { this(source, left, pattern, false); @@ -33,14 +32,4 @@ public String getWriteableName() { throw new UnsupportedOperationException(); } - @Override - protected NodeInfo info() { - return NodeInfo.create(this, WildcardLike::new, field(), pattern(), caseInsensitive()); - } - - @Override - protected WildcardLike replaceChild(Expression newLeft) { - return new WildcardLike(source(), newLeft, pattern(), caseInsensitive()); - } - } diff --git a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/util/TestUtils.java b/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/util/TestUtils.java index 9f8e23cb15a97..b37ca0431ec2d 100644 --- a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/util/TestUtils.java +++ b/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/util/TestUtils.java @@ -11,8 +11,6 @@ import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; import org.elasticsearch.xpack.esql.core.expression.Literal; import org.elasticsearch.xpack.esql.core.expression.predicate.Range; -import org.elasticsearch.xpack.esql.core.expression.predicate.regex.RLike; -import org.elasticsearch.xpack.esql.core.expression.predicate.regex.RLikePattern; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.core.type.EsField; @@ -46,10 +44,6 @@ public static Range rangeOf(Expression value, Expression lower, boolean includeL return new Range(EMPTY, value, lower, includeLower, upper, includeUpper, randomZone()); } - public static RLike rlike(Expression left, String exp) { - return new RLike(EMPTY, left, new RLikePattern(exp)); - } - public static FieldAttribute fieldAttribute() { return fieldAttribute(randomAlphaOfLength(10), randomFrom(DataType.types())); } diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/eval.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/eval.csv-spec index 592b06107c8b5..72660c11d8b73 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/eval.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/eval.csv-spec @@ -601,3 +601,39 @@ Mokhtar |Bernatsky |38992 |BM Parto |Bamford |61805 |BP Premal |Baek |52833 |BP ; + + +caseInsensitiveRegex +from employees | where first_name RLIKE "(?i)geor.*" | keep first_name +; + +first_name:keyword +; + + +caseInsensitiveRegex2 +from employees | where first_name RLIKE "(?i)Geor.*" | keep first_name +; + +first_name:keyword +; + + +caseInsensitiveRegexFold +required_capability: fixed_regex_fold +row foo = "Bar" | where foo rlike "(?i)ba.*" +; + +foo:keyword +; + + +caseInsensitiveRegexFold2 +required_capability: fixed_regex_fold +row foo = "Bar" | where foo rlike "(?i)Ba.*" +; + +foo:keyword +; + + diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 649ec1eba9785..e9a0f89e4f448 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -577,7 +577,12 @@ public enum Cap { /** * Additional types for match function and operator */ - MATCH_ADDITIONAL_TYPES; + MATCH_ADDITIONAL_TYPES, + + /** + * Fix for regex folding with case-insensitive pattern https://github.com/elastic/elasticsearch/issues/118371 + */ + FIXED_REGEX_FOLD; private final boolean enabled; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RLike.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RLike.java index cd42711177510..996c90a8e40bc 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RLike.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/RLike.java @@ -79,7 +79,7 @@ public String getWriteableName() { } @Override - protected NodeInfo info() { + protected NodeInfo info() { return NodeInfo.create(this, RLike::new, field(), pattern(), caseInsensitive()); } @@ -93,6 +93,11 @@ protected TypeResolution resolveType() { return isString(field(), sourceText(), DEFAULT); } + @Override + public Boolean fold() { + return (Boolean) EvaluatorMapper.super.fold(); + } + @Override public EvalOperator.ExpressionEvaluator.Factory toEvaluator(ToEvaluator toEvaluator) { return AutomataMatch.toEvaluator(source(), toEvaluator.apply(field()), pattern().createAutomaton()); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/WildcardLike.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/WildcardLike.java index c1b4f20f41795..d2edb0f92e8f2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/WildcardLike.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/WildcardLike.java @@ -99,6 +99,11 @@ protected TypeResolution resolveType() { return isString(field(), sourceText(), DEFAULT); } + @Override + public Boolean fold() { + return (Boolean) EvaluatorMapper.super.fold(); + } + @Override public EvalOperator.ExpressionEvaluator.Factory toEvaluator(ToEvaluator toEvaluator) { return AutomataMatch.toEvaluator( diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFoldingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFoldingTests.java index c2e85cc43284a..c4f4dac67acd3 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFoldingTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ConstantFoldingTests.java @@ -17,11 +17,11 @@ import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And; import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Not; import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Or; -import org.elasticsearch.xpack.esql.core.expression.predicate.regex.RLike; import org.elasticsearch.xpack.esql.core.expression.predicate.regex.RLikePattern; -import org.elasticsearch.xpack.esql.core.expression.predicate.regex.WildcardLike; import org.elasticsearch.xpack.esql.core.expression.predicate.regex.WildcardPattern; import org.elasticsearch.xpack.esql.core.type.DataType; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.RLike; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.WildcardLike; import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Add; import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Div; import org.elasticsearch.xpack.esql.expression.predicate.operator.arithmetic.Mod; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFiltersTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFiltersTests.java index e159e5ed0bd7d..bc22fbb6bd828 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFiltersTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PushDownAndCombineFiltersTests.java @@ -199,7 +199,7 @@ public void testPushDownFilterOnAliasInEval() { public void testPushDownLikeRlikeFilter() { EsRelation relation = relation(); - org.elasticsearch.xpack.esql.core.expression.predicate.regex.RLike conditionA = rlike(getFieldAttribute("a"), "foo"); + RLike conditionA = rlike(getFieldAttribute("a"), "foo"); WildcardLike conditionB = wildcardLike(getFieldAttribute("b"), "bar"); Filter fa = new Filter(EMPTY, relation, conditionA); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatchTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatchTests.java index 20d638a113bf2..c7206c6971bde 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatchTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/ReplaceRegexMatchTests.java @@ -11,11 +11,11 @@ import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; import org.elasticsearch.xpack.esql.core.expression.predicate.nulls.IsNotNull; -import org.elasticsearch.xpack.esql.core.expression.predicate.regex.RLike; import org.elasticsearch.xpack.esql.core.expression.predicate.regex.RLikePattern; -import org.elasticsearch.xpack.esql.core.expression.predicate.regex.WildcardLike; import org.elasticsearch.xpack.esql.core.expression.predicate.regex.WildcardPattern; import org.elasticsearch.xpack.esql.core.util.StringUtils; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.RLike; +import org.elasticsearch.xpack.esql.expression.function.scalar.string.WildcardLike; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; import static java.util.Arrays.asList; From 6d6eac2c77a94dfdbee01cc1f609354c8c016d2a Mon Sep 17 00:00:00 2001 From: kanoshiou <73424326+kanoshiou@users.noreply.github.com> Date: Mon, 16 Dec 2024 22:20:58 +0800 Subject: [PATCH 08/44] ESQL: Drop null columns in text formats (#117643) This PR resolves the issue where, despite setting `drop_null_columns=true`, columns that are entirely null are still returned when using `format=txt`, `format=csv`, or `format=tsv`. Closes #116848 --- docs/changelog/117643.yaml | 6 +++ .../xpack/esql/qa/rest/RestEsqlTestCase.java | 6 +-- .../xpack/esql/action/EsqlQueryResponse.java | 2 +- .../xpack/esql/formatter/TextFormat.java | 22 +++++++-- .../xpack/esql/formatter/TextFormatter.java | 24 ++++++++-- .../xpack/esql/formatter/TextFormatTests.java | 47 ++++++++++++++----- .../esql/formatter/TextFormatterTests.java | 41 +++++++++++++--- 7 files changed, 117 insertions(+), 31 deletions(-) create mode 100644 docs/changelog/117643.yaml diff --git a/docs/changelog/117643.yaml b/docs/changelog/117643.yaml new file mode 100644 index 0000000000000..9105749377d2c --- /dev/null +++ b/docs/changelog/117643.yaml @@ -0,0 +1,6 @@ +pr: 117643 +summary: Drop null columns in text formats +area: ES|QL +type: bug +issues: + - 116848 diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java index 6a8779eef4efc..86f8a8c5363f6 100644 --- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java @@ -1119,7 +1119,7 @@ public void testAsyncGetWithoutContentType() throws IOException { var json = entityToMap(entity, requestObject.contentType()); checkKeepOnCompletion(requestObject, json, true); String id = (String) json.get("id"); - // results won't be returned since keepOnCompletion is true + // results won't be returned because wait_for_completion is provided a very small interval assertThat(id, is(not(emptyOrNullString()))); // issue an "async get" request with no Content-Type @@ -1274,11 +1274,11 @@ static String runEsqlAsTextWithFormat(RequestObjectBuilder builder, String forma switch (format) { case "txt" -> assertThat(initialValue, emptyOrNullString()); case "csv" -> { - assertEquals(initialValue, "\r\n"); + assertEquals("\r\n", initialValue); initialValue = ""; } case "tsv" -> { - assertEquals(initialValue, "\n"); + assertEquals("\n", initialValue); initialValue = ""; } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java index dc0e9fd1fb06d..4163a222b1a28 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponse.java @@ -218,7 +218,7 @@ public Iterator toXContentChunked(ToXContent.Params params }); } - private boolean[] nullColumns() { + public boolean[] nullColumns() { boolean[] nullColumns = new boolean[columns.size()]; for (int c = 0; c < nullColumns.length; c++) { nullColumns[c] = allColumnsAreNull(c); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormat.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormat.java index 5c0d6b138b326..7a7e4677b0dca 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormat.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormat.java @@ -39,7 +39,8 @@ public enum TextFormat implements MediaType { PLAIN_TEXT() { @Override public Iterator> format(RestRequest request, EsqlQueryResponse esqlResponse) { - return new TextFormatter(esqlResponse).format(hasHeader(request)); + boolean dropNullColumns = request.paramAsBoolean(DROP_NULL_COLUMNS_OPTION, false); + return new TextFormatter(esqlResponse, hasHeader(request), dropNullColumns).format(); } @Override @@ -282,15 +283,21 @@ public Set headerValues() { */ public static final String URL_PARAM_FORMAT = "format"; public static final String URL_PARAM_DELIMITER = "delimiter"; + public static final String DROP_NULL_COLUMNS_OPTION = "drop_null_columns"; public Iterator> format(RestRequest request, EsqlQueryResponse esqlResponse) { final var delimiter = delimiter(request); + boolean dropNullColumns = request.paramAsBoolean(DROP_NULL_COLUMNS_OPTION, false); + boolean[] dropColumns = dropNullColumns ? esqlResponse.nullColumns() : new boolean[esqlResponse.columns().size()]; return Iterators.concat( // if the header is requested return the info hasHeader(request) && esqlResponse.columns() != null - ? Iterators.single(writer -> row(writer, esqlResponse.columns().iterator(), ColumnInfo::name, delimiter)) + ? Iterators.single(writer -> row(writer, esqlResponse.columns().iterator(), ColumnInfo::name, delimiter, dropColumns)) : Collections.emptyIterator(), - Iterators.map(esqlResponse.values(), row -> writer -> row(writer, row, f -> Objects.toString(f, StringUtils.EMPTY), delimiter)) + Iterators.map( + esqlResponse.values(), + row -> writer -> row(writer, row, f -> Objects.toString(f, StringUtils.EMPTY), delimiter, dropColumns) + ) ); } @@ -313,9 +320,14 @@ public String contentType(RestRequest request) { } // utility method for consuming a row. - void row(Writer writer, Iterator row, Function toString, Character delimiter) throws IOException { + void row(Writer writer, Iterator row, Function toString, Character delimiter, boolean[] dropColumns) + throws IOException { boolean firstColumn = true; - while (row.hasNext()) { + for (int i = 0; row.hasNext(); i++) { + if (dropColumns[i]) { + row.next(); + continue; + } if (firstColumn) { firstColumn = false; } else { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormatter.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormatter.java index 0535e4adfe346..95b46958be351 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormatter.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/formatter/TextFormatter.java @@ -30,13 +30,17 @@ public class TextFormatter { private final EsqlQueryResponse response; private final int[] width; private final Function FORMATTER = Objects::toString; + private final boolean includeHeader; + private final boolean[] dropColumns; /** - * Create a new {@linkplain TextFormatter} for formatting responses. + * Create a new {@linkplain TextFormatter} for formatting responses */ - public TextFormatter(EsqlQueryResponse response) { + public TextFormatter(EsqlQueryResponse response, boolean includeHeader, boolean dropNullColumns) { this.response = response; var columns = response.columns(); + this.includeHeader = includeHeader; + this.dropColumns = dropNullColumns ? response.nullColumns() : new boolean[columns.size()]; // Figure out the column widths: // 1. Start with the widths of the column names width = new int[columns.size()]; @@ -58,12 +62,12 @@ public TextFormatter(EsqlQueryResponse response) { } /** - * Format the provided {@linkplain EsqlQueryResponse} optionally including the header lines. + * Format the provided {@linkplain EsqlQueryResponse} */ - public Iterator> format(boolean includeHeader) { + public Iterator> format() { return Iterators.concat( // The header lines - includeHeader && response.columns().size() > 0 ? Iterators.single(this::formatHeader) : Collections.emptyIterator(), + includeHeader && response.columns().isEmpty() == false ? Iterators.single(this::formatHeader) : Collections.emptyIterator(), // Now format the results. formatResults() ); @@ -71,6 +75,9 @@ public Iterator> format(boolean includeHead private void formatHeader(Writer writer) throws IOException { for (int i = 0; i < width.length; i++) { + if (dropColumns[i]) { + continue; + } if (i > 0) { writer.append('|'); } @@ -86,6 +93,9 @@ private void formatHeader(Writer writer) throws IOException { writer.append('\n'); for (int i = 0; i < width.length; i++) { + if (dropColumns[i]) { + continue; + } if (i > 0) { writer.append('+'); } @@ -98,6 +108,10 @@ private Iterator> formatResults() { return Iterators.map(response.values(), row -> writer -> { for (int i = 0; i < width.length; i++) { assert row.hasNext(); + if (dropColumns[i]) { + row.next(); + continue; + } if (i > 0) { writer.append('|'); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java index fe1ac52427627..ca47e0cb329b3 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatTests.java @@ -123,17 +123,17 @@ public void testTsvFormatWithEmptyData() { public void testCsvFormatWithRegularData() { String text = format(CSV, req(), regularData()); assertEquals(""" - string,number,location,location2\r - Along The River Bank,708,POINT (12.0 56.0),POINT (1234.0 5678.0)\r - Mind Train,280,POINT (-97.0 26.0),POINT (-9753.0 2611.0)\r + string,number,location,location2,null_field\r + Along The River Bank,708,POINT (12.0 56.0),POINT (1234.0 5678.0),\r + Mind Train,280,POINT (-97.0 26.0),POINT (-9753.0 2611.0),\r """, text); } public void testCsvFormatNoHeaderWithRegularData() { String text = format(CSV, reqWithParam("header", "absent"), regularData()); assertEquals(""" - Along The River Bank,708,POINT (12.0 56.0),POINT (1234.0 5678.0)\r - Mind Train,280,POINT (-97.0 26.0),POINT (-9753.0 2611.0)\r + Along The River Bank,708,POINT (12.0 56.0),POINT (1234.0 5678.0),\r + Mind Train,280,POINT (-97.0 26.0),POINT (-9753.0 2611.0),\r """, text); } @@ -146,14 +146,17 @@ public void testCsvFormatWithCustomDelimiterRegularData() { "number", "location", "location2", + "null_field", "Along The River Bank", "708", "POINT (12.0 56.0)", "POINT (1234.0 5678.0)", + "", "Mind Train", "280", "POINT (-97.0 26.0)", - "POINT (-9753.0 2611.0)" + "POINT (-9753.0 2611.0)", + "" ); List expectedTerms = terms.stream() .map(x -> x.contains(String.valueOf(delim)) ? '"' + x + '"' : x) @@ -167,6 +170,8 @@ public void testCsvFormatWithCustomDelimiterRegularData() { sb.append(expectedTerms.remove(0)); sb.append(delim); sb.append(expectedTerms.remove(0)); + sb.append(delim); + sb.append(expectedTerms.remove(0)); sb.append("\r\n"); } while (expectedTerms.size() > 0); assertEquals(sb.toString(), text); @@ -175,9 +180,9 @@ public void testCsvFormatWithCustomDelimiterRegularData() { public void testTsvFormatWithRegularData() { String text = format(TSV, req(), regularData()); assertEquals(""" - string\tnumber\tlocation\tlocation2 - Along The River Bank\t708\tPOINT (12.0 56.0)\tPOINT (1234.0 5678.0) - Mind Train\t280\tPOINT (-97.0 26.0)\tPOINT (-9753.0 2611.0) + string\tnumber\tlocation\tlocation2\tnull_field + Along The River Bank\t708\tPOINT (12.0 56.0)\tPOINT (1234.0 5678.0)\t + Mind Train\t280\tPOINT (-97.0 26.0)\tPOINT (-9753.0 2611.0)\t """, text); } @@ -245,6 +250,24 @@ public void testPlainTextEmptyCursorWithoutColumns() { ); } + public void testCsvFormatWithDropNullColumns() { + String text = format(CSV, reqWithParam("drop_null_columns", "true"), regularData()); + assertEquals(""" + string,number,location,location2\r + Along The River Bank,708,POINT (12.0 56.0),POINT (1234.0 5678.0)\r + Mind Train,280,POINT (-97.0 26.0),POINT (-9753.0 2611.0)\r + """, text); + } + + public void testTsvFormatWithDropNullColumns() { + String text = format(TSV, reqWithParam("drop_null_columns", "true"), regularData()); + assertEquals(""" + string\tnumber\tlocation\tlocation2 + Along The River Bank\t708\tPOINT (12.0 56.0)\tPOINT (1234.0 5678.0) + Mind Train\t280\tPOINT (-97.0 26.0)\tPOINT (-9753.0 2611.0) + """, text); + } + private static EsqlQueryResponse emptyData() { return new EsqlQueryResponse(singletonList(new ColumnInfoImpl("name", "keyword")), emptyList(), null, false, false, null); } @@ -256,7 +279,8 @@ private static EsqlQueryResponse regularData() { new ColumnInfoImpl("string", "keyword"), new ColumnInfoImpl("number", "integer"), new ColumnInfoImpl("location", "geo_point"), - new ColumnInfoImpl("location2", "cartesian_point") + new ColumnInfoImpl("location2", "cartesian_point"), + new ColumnInfoImpl("null_field", "keyword") ); BytesRefArray geoPoints = new BytesRefArray(2, BigArrays.NON_RECYCLING_INSTANCE); @@ -274,7 +298,8 @@ private static EsqlQueryResponse regularData() { blockFactory.newBytesRefBlockBuilder(2) .appendBytesRef(CARTESIAN.asWkb(new Point(1234, 5678))) .appendBytesRef(CARTESIAN.asWkb(new Point(-9753, 2611))) - .build() + .build(), + blockFactory.newConstantNullBlock(2) ) ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java index e735ba83168bb..4e90fe53d96d7 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/formatter/TextFormatterTests.java @@ -85,8 +85,6 @@ public class TextFormatterTests extends ESTestCase { new EsqlExecutionInfo(randomBoolean()) ); - TextFormatter formatter = new TextFormatter(esqlResponse); - /** * Tests for {@link TextFormatter#format} with header, values * of exactly the minimum column size, column names of exactly @@ -95,7 +93,7 @@ public class TextFormatterTests extends ESTestCase { * column size. */ public void testFormatWithHeader() { - String[] result = getTextBodyContent(formatter.format(true)).split("\n"); + String[] result = getTextBodyContent(new TextFormatter(esqlResponse, true, false).format()).split("\n"); assertThat(result, arrayWithSize(4)); assertEquals( " foo | bar |15charwidename!| null_field1 |superduperwidename!!!| baz |" @@ -119,6 +117,35 @@ public void testFormatWithHeader() { ); } + /** + * Tests for {@link TextFormatter#format} with drop_null_columns and + * truncation of long columns. + */ + public void testFormatWithDropNullColumns() { + String[] result = getTextBodyContent(new TextFormatter(esqlResponse, true, true).format()).split("\n"); + assertThat(result, arrayWithSize(4)); + assertEquals( + " foo | bar |15charwidename!|superduperwidename!!!| baz |" + + " date | location | location2 ", + result[0] + ); + assertEquals( + "---------------+---------------+---------------+---------------------+---------------+-------" + + "-----------------+------------------+----------------------", + result[1] + ); + assertEquals( + "15charwidedata!|1 |6.888 |12.0 |rabbit |" + + "1953-09-02T00:00:00.000Z|POINT (12.0 56.0) |POINT (1234.0 5678.0) ", + result[2] + ); + assertEquals( + "dog |2 |123124.888 |9912.0 |goat |" + + "2000-03-15T21:34:37.443Z|POINT (-97.0 26.0)|POINT (-9753.0 2611.0)", + result[3] + ); + } + /** * Tests for {@link TextFormatter#format} without header and * truncation of long columns. @@ -160,7 +187,7 @@ public void testFormatWithoutHeader() { new EsqlExecutionInfo(randomBoolean()) ); - String[] result = getTextBodyContent(new TextFormatter(response).format(false)).split("\n"); + String[] result = getTextBodyContent(new TextFormatter(response, false, false).format()).split("\n"); assertThat(result, arrayWithSize(2)); assertEquals( "doggie |4 |1.0 |null |77.0 |wombat |" @@ -199,8 +226,10 @@ public void testVeryLongPadding() { randomBoolean(), randomBoolean(), new EsqlExecutionInfo(randomBoolean()) - ) - ).format(false) + ), + false, + false + ).format() ) ); } From f2e5430682a194995f0dae87612ab758db0d377e Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Mon, 16 Dec 2024 09:24:47 -0500 Subject: [PATCH 09/44] Remove unused method (#118601) I was looking at some type conversion stuff, and noticed this method is never used. Back porting to reduce possible conflicts with the long term maintenance branch later. --- .../xpack/esql/core/planner/TranslatorHandler.java | 2 -- .../xpack/esql/planner/EsqlTranslatorHandler.java | 7 ------- 2 files changed, 9 deletions(-) diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/TranslatorHandler.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/TranslatorHandler.java index 1ccbb04f7a69c..b85544905595a 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/TranslatorHandler.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/TranslatorHandler.java @@ -12,7 +12,6 @@ import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; import org.elasticsearch.xpack.esql.core.expression.function.scalar.ScalarFunction; import org.elasticsearch.xpack.esql.core.querydsl.query.Query; -import org.elasticsearch.xpack.esql.core.type.DataType; import java.util.function.Supplier; @@ -34,5 +33,4 @@ default Query wrapFunctionQuery(ScalarFunction sf, Expression field, Supplier querySupplier) { if (field instanceof FieldAttribute fa) { From e6a27a91eeb4b1876287ce80cffc4efa7c4f3448 Mon Sep 17 00:00:00 2001 From: Alexis Charveriat Date: Mon, 16 Dec 2024 16:34:41 +0100 Subject: [PATCH 10/44] tier_preference and creation_date fields in monitoring template (#117851) * tier_preference and creation_date fields in monitoring template * Added index version * Update docs/changelog/117851.yaml * Updated changelog * Incremented STACK_MONITORING_REGISTRY_VERSION --- docs/changelog/117851.yaml | 5 +++++ .../src/main/resources/monitoring-es-mb.json | 11 +++++++++++ .../xpack/monitoring/MonitoringTemplateRegistry.java | 2 +- 3 files changed, 17 insertions(+), 1 deletion(-) create mode 100644 docs/changelog/117851.yaml diff --git a/docs/changelog/117851.yaml b/docs/changelog/117851.yaml new file mode 100644 index 0000000000000..21888cd6fb80f --- /dev/null +++ b/docs/changelog/117851.yaml @@ -0,0 +1,5 @@ +pr: 117851 +summary: Addition of `tier_preference`, `creation_date` and `version` fields in Elasticsearch monitoring template +area: Monitoring +type: enhancement +issues: [] diff --git a/x-pack/plugin/core/template-resources/src/main/resources/monitoring-es-mb.json b/x-pack/plugin/core/template-resources/src/main/resources/monitoring-es-mb.json index 2bf7607e86d32..793a8c3035d8e 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/monitoring-es-mb.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/monitoring-es-mb.json @@ -1517,6 +1517,17 @@ "ignore_above": 1024, "type": "keyword" }, + "tier_preference": { + "ignore_above": 1024, + "type": "keyword" + }, + "creation_date": { + "type": "date" + }, + "version": { + "ignore_above": 1024, + "type": "keyword" + }, "recovery": { "properties": { "stop_time": { 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 e0433ea6fdd71..cfd322d04e92f 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 @@ -77,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 = 8_00_00_99 + 18; + public static final int STACK_MONITORING_REGISTRY_VERSION = 8_00_00_99 + 19; 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"; From bded35bb39c4977c8255c2ffc3e06a7a9fa9b1dc Mon Sep 17 00:00:00 2001 From: Alexander Spies Date: Mon, 16 Dec 2024 17:14:16 +0100 Subject: [PATCH 11/44] ESQL: Add LOOKUP JOIN tests with null and mv join keys (#118761) Notable behavior: - `null` join keys never match anything, even a `null` from the left hand side. - If a lookup index document has multi-valued join keys, it will match with all left hand side rows that contain any of the multi-values. - If the left hand side has multi-values, it will match with any lookup index document whose join key contains any of the multi-values. --- .../resources/languages_non_unique_key.csv | 4 ++ .../src/main/resources/lookup-join.csv-spec | 67 +++++++++++++++++++ 2 files changed, 71 insertions(+) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/languages_non_unique_key.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/languages_non_unique_key.csv index 1578762f8d1cb..d6381b174d739 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/languages_non_unique_key.csv +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/languages_non_unique_key.csv @@ -8,3 +8,7 @@ language_code:integer,language_name:keyword,country:keyword 2,German, 4,Quenya, 5,,Atlantis +[6,7],Mv-Lang,Mv-Land +[7,8],Mv-Lang2,Mv-Land2 +,Null-Lang,Null-Land +,Null-Lang2,Null-Land2 diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec index c39f4ae7b4e0c..7fed4f377096f 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec @@ -178,6 +178,73 @@ language_code:integer | language_name:keyword | country:keyword 2 | [German, German, German] | [Austria, Germany, Switzerland] ; +nullJoinKeyOnTheDataNode +required_capability: join_lookup_v5 + +FROM employees +| WHERE emp_no < 10004 +| EVAL language_code = emp_no % 10, language_code = CASE(language_code == 3, null, language_code) +| LOOKUP JOIN languages_lookup_non_unique_key ON language_code +| SORT emp_no +| KEEP emp_no, language_code, language_name +; + +emp_no:integer | language_code:integer | language_name:keyword +10001 | 1 | [English, English, English] +10002 | 2 | [German, German, German] +10003 | null | null +; + + +mvJoinKeyOnTheDataNode +required_capability: join_lookup_v5 + +FROM employees +| WHERE 10003 < emp_no AND emp_no < 10008 +| EVAL language_code = emp_no % 10 +| LOOKUP JOIN languages_lookup_non_unique_key ON language_code +| SORT emp_no +| KEEP emp_no, language_code, language_name +; + +emp_no:integer | language_code:integer | language_name:keyword +10004 | 4 | Quenya +10005 | 5 | null +10006 | 6 | Mv-Lang +10007 | 7 | [Mv-Lang, Mv-Lang2] +; + +mvJoinKeyFromRow +required_capability: join_lookup_v5 + +ROW language_code = [4, 5, 6, 7] +| LOOKUP JOIN languages_lookup_non_unique_key ON language_code +| EVAL language_name = MV_SORT(language_name), country = MV_SORT(country) +| KEEP language_code, language_name, country +; + +language_code:integer | language_name:keyword | country:keyword +[4, 5, 6, 7] | [Mv-Lang, Mv-Lang2, Quenya] | [Atlantis, Mv-Land, Mv-Land2] +; + +mvJoinKeyFromRowExpanded +required_capability: join_lookup_v5 + +ROW language_code = [4, 5, 6, 7, 8] +| MV_EXPAND language_code +| LOOKUP JOIN languages_lookup_non_unique_key ON language_code +| EVAL language_name = MV_SORT(language_name), country = MV_SORT(country) +| KEEP language_code, language_name, country +; + +language_code:integer | language_name:keyword | country:keyword +4 | Quenya | null +5 | null | Atlantis +6 | Mv-Lang | Mv-Land +7 | [Mv-Lang, Mv-Lang2] | [Mv-Land, Mv-Land2] +8 | Mv-Lang2 | Mv-Land2 +; + lookupIPFromRow required_capability: join_lookup_v5 From 693bb794bce43eb20d07b08ceba5e796ded6efbd Mon Sep 17 00:00:00 2001 From: Salvatore Campagna <93581129+salvatore-campagna@users.noreply.github.com> Date: Mon, 16 Dec 2024 17:27:45 +0100 Subject: [PATCH 12/44] Expose shard changes action as a rest api (#118608) This PR adds a new REST API `/{index}/ccr/shard_changes` to retrieve shard-level changes, including translog operations, mapping versions, and sequence numbers. It is required for a new set of Rally benchmarks, which will be used to evaluate the impact of synthetic source on recovery time. The API accepts parameters like `from_seq_no`, `max_batch_size`, `poll_timeout`, and `max_operations_count` and is exposed only in snapshot builds. --- x-pack/plugin/ccr/build.gradle | 11 + .../xpack/ccr/rest/ShardChangesRestIT.java | 281 ++++++++++++++++ .../java/org/elasticsearch/xpack/ccr/Ccr.java | 50 +-- .../ccr/rest/RestShardChangesAction.java | 300 ++++++++++++++++++ 4 files changed, 623 insertions(+), 19 deletions(-) create mode 100644 x-pack/plugin/ccr/src/javaRestTest/java/org/elasticsearch/xpack/ccr/rest/ShardChangesRestIT.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestShardChangesAction.java diff --git a/x-pack/plugin/ccr/build.gradle b/x-pack/plugin/ccr/build.gradle index f673513950bb4..b5e96ac2a8b34 100644 --- a/x-pack/plugin/ccr/build.gradle +++ b/x-pack/plugin/ccr/build.gradle @@ -1,5 +1,6 @@ apply plugin: 'elasticsearch.internal-es-plugin' apply plugin: 'elasticsearch.internal-cluster-test' +apply plugin: 'elasticsearch.internal-java-rest-test' esplugin { name 'x-pack-ccr' description 'Elasticsearch Expanded Pack Plugin - CCR' @@ -33,6 +34,16 @@ tasks.named('internalClusterTestTestingConventions').configure { baseClass 'org.elasticsearch.test.ESIntegTestCase' } +tasks.named("javaRestTest").configure { + usesDefaultDistribution() +} + +restResources { + restApi { + include 'bulk', 'search', '_common', 'indices', 'index', 'cluster', 'data_stream' + } +} + addQaCheckDependencies(project) dependencies { diff --git a/x-pack/plugin/ccr/src/javaRestTest/java/org/elasticsearch/xpack/ccr/rest/ShardChangesRestIT.java b/x-pack/plugin/ccr/src/javaRestTest/java/org/elasticsearch/xpack/ccr/rest/ShardChangesRestIT.java new file mode 100644 index 0000000000000..e5dfea7b772f2 --- /dev/null +++ b/x-pack/plugin/ccr/src/javaRestTest/java/org/elasticsearch/xpack/ccr/rest/ShardChangesRestIT.java @@ -0,0 +1,281 @@ +/* + * 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.ccr.rest; + +import org.apache.http.util.EntityUtils; +import org.elasticsearch.Build; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.cluster.local.distribution.DistributionType; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.hamcrest.Matchers; +import org.junit.Before; +import org.junit.ClassRule; + +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +public class ShardChangesRestIT extends ESRestTestCase { + private static final String CCR_SHARD_CHANGES_ENDPOINT = "/%s/ccr/shard_changes"; + private static final String BULK_INDEX_ENDPOINT = "/%s/_bulk"; + + private static final String[] SHARD_RESPONSE_FIELDS = new String[] { + "took_in_millis", + "operations", + "shard_id", + "index", + "settings_version", + "max_seq_no_of_updates_or_deletes", + "number_of_operations", + "mapping_version", + "aliases_version", + "max_seq_no", + "global_checkpoint" }; + private static final String[] NAMES = { "skywalker", "leia", "obi-wan", "yoda", "chewbacca", "r2-d2", "c-3po", "darth-vader" }; + @ClassRule + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .distribution(DistributionType.DEFAULT) + .setting("xpack.security.enabled", "false") + .setting("xpack.license.self_generated.type", "trial") + .build(); + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + @Before + public void assumeSnapshotBuild() { + assumeTrue("/{index}/ccr/shard_changes endpoint only available in snapshot builds", Build.current().isSnapshot()); + } + + public void testShardChangesNoOperation() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + createIndex( + indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey(), "1s") + .build() + ); + assertTrue(indexExists(indexName)); + + final Request shardChangesRequest = new Request("GET", shardChangesEndpoint(indexName)); + assertOK(client().performRequest(shardChangesRequest)); + } + + public void testShardChangesDefaultParams() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + final Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey(), "1s") + .build(); + final String mappings = """ + { + "properties": { + "name": { + "type": "keyword" + } + } + } + """; + createIndex(indexName, settings, mappings); + assertTrue(indexExists(indexName)); + + assertOK(client().performRequest(bulkRequest(indexName, randomIntBetween(10, 20)))); + + final Request shardChangesRequest = new Request("GET", shardChangesEndpoint(indexName)); + final Response response = client().performRequest(shardChangesRequest); + assertOK(response); + assertShardChangesResponse( + XContentHelper.convertToMap(JsonXContent.jsonXContent, EntityUtils.toString(response.getEntity()), false) + ); + } + + public void testShardChangesWithAllParameters() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + createIndex( + indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey(), "1s") + .build() + ); + assertTrue(indexExists(indexName)); + + assertOK(client().performRequest(bulkRequest(indexName, randomIntBetween(100, 200)))); + + final Request shardChangesRequest = new Request("GET", shardChangesEndpoint(indexName)); + shardChangesRequest.addParameter("from_seq_no", "0"); + shardChangesRequest.addParameter("max_operations_count", "1"); + shardChangesRequest.addParameter("poll_timeout", "10s"); + shardChangesRequest.addParameter("max_batch_size", "1MB"); + + final Response response = client().performRequest(shardChangesRequest); + assertOK(response); + assertShardChangesResponse( + XContentHelper.convertToMap(JsonXContent.jsonXContent, EntityUtils.toString(response.getEntity()), false) + ); + } + + public void testShardChangesMultipleRequests() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + createIndex( + indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey(), "1s") + .build() + ); + assertTrue(indexExists(indexName)); + + assertOK(client().performRequest(bulkRequest(indexName, randomIntBetween(100, 200)))); + + final Request firstRequest = new Request("GET", shardChangesEndpoint(indexName)); + firstRequest.addParameter("from_seq_no", "0"); + firstRequest.addParameter("max_operations_count", "10"); + firstRequest.addParameter("poll_timeout", "10s"); + firstRequest.addParameter("max_batch_size", "1MB"); + + final Response firstResponse = client().performRequest(firstRequest); + assertOK(firstResponse); + assertShardChangesResponse( + XContentHelper.convertToMap(JsonXContent.jsonXContent, EntityUtils.toString(firstResponse.getEntity()), false) + ); + + final Request secondRequest = new Request("GET", shardChangesEndpoint(indexName)); + secondRequest.addParameter("from_seq_no", "10"); + secondRequest.addParameter("max_operations_count", "10"); + secondRequest.addParameter("poll_timeout", "10s"); + secondRequest.addParameter("max_batch_size", "1MB"); + + final Response secondResponse = client().performRequest(secondRequest); + assertOK(secondResponse); + assertShardChangesResponse( + XContentHelper.convertToMap(JsonXContent.jsonXContent, EntityUtils.toString(secondResponse.getEntity()), false) + ); + } + + public void testShardChangesInvalidFromSeqNo() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + assertTrue(indexExists(indexName)); + + final Request shardChangesRequest = new Request("GET", shardChangesEndpoint(indexName)); + shardChangesRequest.addParameter("from_seq_no", "-1"); + final ResponseException ex = assertThrows(ResponseException.class, () -> client().performRequest(shardChangesRequest)); + assertResponseException(ex, RestStatus.BAD_REQUEST, "Validation Failed: 1: fromSeqNo [-1] cannot be lower than 0"); + } + + public void testShardChangesInvalidMaxOperationsCount() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + assertTrue(indexExists(indexName)); + + final Request shardChangesRequest = new Request("GET", shardChangesEndpoint(indexName)); + shardChangesRequest.addParameter("max_operations_count", "-1"); + final ResponseException ex = assertThrows(ResponseException.class, () -> client().performRequest(shardChangesRequest)); + assertResponseException(ex, RestStatus.BAD_REQUEST, "Validation Failed: 1: maxOperationCount [-1] cannot be lower than 0"); + } + + public void testShardChangesNegativePollTimeout() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + assertTrue(indexExists(indexName)); + + final Request shardChangesRequest = new Request("GET", shardChangesEndpoint(indexName)); + shardChangesRequest.addParameter("poll_timeout", "-1s"); + assertOK(client().performRequest(shardChangesRequest)); + } + + public void testShardChangesInvalidMaxBatchSize() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + assertTrue(indexExists(indexName)); + + final Request shardChangesRequest = new Request("GET", shardChangesEndpoint(indexName)); + shardChangesRequest.addParameter("max_batch_size", "-1MB"); + final ResponseException ex = assertThrows(ResponseException.class, () -> client().performRequest(shardChangesRequest)); + assertResponseException( + ex, + RestStatus.BAD_REQUEST, + "failed to parse setting [max_batch_size] with value [-1MB] as a size in bytes" + ); + } + + public void testShardChangesMissingIndex() throws IOException { + final String indexName = randomAlphanumericOfLength(10).toLowerCase(Locale.ROOT); + assertFalse(indexExists(indexName)); + + final Request shardChangesRequest = new Request("GET", shardChangesEndpoint(indexName)); + final ResponseException ex = assertThrows(ResponseException.class, () -> client().performRequest(shardChangesRequest)); + assertResponseException(ex, RestStatus.BAD_REQUEST, "Failed to process shard changes for index [" + indexName + "]"); + } + + private static Request bulkRequest(final String indexName, int numberOfDocuments) { + final StringBuilder sb = new StringBuilder(); + + for (int i = 0; i < numberOfDocuments; i++) { + sb.append(String.format(Locale.ROOT, "{ \"index\": { \"_id\": \"%d\" } }\n{ \"name\": \"%s\" }\n", i + 1, randomFrom(NAMES))); + } + + final Request request = new Request("POST", bulkEndpoint(indexName)); + request.setJsonEntity(sb.toString()); + request.addParameter("refresh", "true"); + return request; + } + + private static String shardChangesEndpoint(final String indexName) { + return String.format(Locale.ROOT, CCR_SHARD_CHANGES_ENDPOINT, indexName); + } + + private static String bulkEndpoint(final String indexName) { + return String.format(Locale.ROOT, BULK_INDEX_ENDPOINT, indexName); + } + + private void assertResponseException(final ResponseException ex, final RestStatus restStatus, final String error) { + assertEquals(restStatus.getStatus(), ex.getResponse().getStatusLine().getStatusCode()); + assertThat(ex.getMessage(), Matchers.containsString(error)); + } + + private void assertShardChangesResponse(final Map shardChangesResponseBody) { + for (final String fieldName : SHARD_RESPONSE_FIELDS) { + final Object fieldValue = shardChangesResponseBody.get(fieldName); + assertNotNull("Field " + fieldName + " is missing or has a null value.", fieldValue); + + if ("operations".equals(fieldName)) { + if (fieldValue instanceof List operationsList) { + assertFalse("Field 'operations' is empty.", operationsList.isEmpty()); + + for (final Object operation : operationsList) { + assertNotNull("Operation is null.", operation); + if (operation instanceof Map operationMap) { + assertNotNull("seq_no is missing in operation.", operationMap.get("seq_no")); + assertNotNull("op_type is missing in operation.", operationMap.get("op_type")); + assertNotNull("primary_term is missing in operation.", operationMap.get("primary_term")); + } + } + } + } + } + } +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java index 87a4c2c7d4826..5305e179058b2 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.ccr; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.Build; import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionRequest; @@ -91,6 +92,7 @@ import org.elasticsearch.xpack.ccr.rest.RestPutFollowAction; import org.elasticsearch.xpack.ccr.rest.RestResumeAutoFollowPatternAction; import org.elasticsearch.xpack.ccr.rest.RestResumeFollowAction; +import org.elasticsearch.xpack.ccr.rest.RestShardChangesAction; import org.elasticsearch.xpack.ccr.rest.RestUnfollowAction; import org.elasticsearch.xpack.core.XPackFeatureUsage; import org.elasticsearch.xpack.core.XPackField; @@ -112,6 +114,7 @@ import org.elasticsearch.xpack.core.ccr.action.ShardFollowTask; import org.elasticsearch.xpack.core.ccr.action.UnfollowAction; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -140,7 +143,34 @@ public class Ccr extends Plugin implements ActionPlugin, PersistentTaskPlugin, E public static final String REQUESTED_OPS_MISSING_METADATA_KEY = "es.requested_operations_missing"; public static final TransportVersion TRANSPORT_VERSION_ACTION_WITH_SHARD_ID = TransportVersions.V_8_9_X; + private static final List BASE_REST_HANDLERS = Arrays.asList( + // stats API + new RestFollowStatsAction(), + new RestCcrStatsAction(), + new RestFollowInfoAction(), + // follow APIs + new RestPutFollowAction(), + new RestResumeFollowAction(), + new RestPauseFollowAction(), + new RestUnfollowAction(), + // auto-follow APIs + new RestDeleteAutoFollowPatternAction(), + new RestPutAutoFollowPatternAction(), + new RestGetAutoFollowPatternAction(), + new RestPauseAutoFollowPatternAction(), + new RestResumeAutoFollowPatternAction(), + // forget follower API + new RestForgetFollowerAction() + ); + private static final List REST_HANDLERS = Collections.unmodifiableList(BASE_REST_HANDLERS); + + private static final List SNAPSHOT_BUILD_REST_HANDLERS; + static { + List snapshotBuildHandlers = new ArrayList<>(BASE_REST_HANDLERS); + snapshotBuildHandlers.add(new RestShardChangesAction()); + SNAPSHOT_BUILD_REST_HANDLERS = Collections.unmodifiableList(snapshotBuildHandlers); + } private final boolean enabled; private final Settings settings; private final CcrLicenseChecker ccrLicenseChecker; @@ -272,25 +302,7 @@ public List getRestHandlers( return emptyList(); } - return Arrays.asList( - // stats API - new RestFollowStatsAction(), - new RestCcrStatsAction(), - new RestFollowInfoAction(), - // follow APIs - new RestPutFollowAction(), - new RestResumeFollowAction(), - new RestPauseFollowAction(), - new RestUnfollowAction(), - // auto-follow APIs - new RestDeleteAutoFollowPatternAction(), - new RestPutAutoFollowPatternAction(), - new RestGetAutoFollowPatternAction(), - new RestPauseAutoFollowPatternAction(), - new RestResumeAutoFollowPatternAction(), - // forget follower API - new RestForgetFollowerAction() - ); + return Build.current().isSnapshot() ? SNAPSHOT_BUILD_REST_HANDLERS : REST_HANDLERS; } public List getNamedWriteables() { diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestShardChangesAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestShardChangesAction.java new file mode 100644 index 0000000000000..84171ebce162f --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestShardChangesAction.java @@ -0,0 +1,300 @@ +/* + * 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.ccr.rest; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.RestActionListener; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xpack.ccr.Ccr; +import org.elasticsearch.xpack.ccr.action.ShardChangesAction; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; + +import static org.elasticsearch.rest.RestRequest.Method.GET; + +/** + * A REST handler that retrieves shard changes in a specific index whose name is provided as a parameter. + * It handles GET requests to the "/{index}/ccr/shard_changes" endpoint retrieving shard-level changes, + * such as translog operations, mapping version, settings version, aliases version, the global checkpoint, + * maximum sequence number and maximum sequence number of updates or deletes. + *

+ * Note: This handler is only available for snapshot builds. + */ +public class RestShardChangesAction extends BaseRestHandler { + + private static final long DEFAULT_FROM_SEQ_NO = 0L; + private static final ByteSizeValue DEFAULT_MAX_BATCH_SIZE = new ByteSizeValue(32, ByteSizeUnit.MB); + private static final TimeValue DEFAULT_POLL_TIMEOUT = new TimeValue(1, TimeUnit.MINUTES); + private static final int DEFAULT_MAX_OPERATIONS_COUNT = 1024; + private static final int DEFAULT_TIMEOUT_SECONDS = 60; + private static final TimeValue GET_INDEX_UUID_TIMEOUT = new TimeValue(DEFAULT_TIMEOUT_SECONDS, TimeUnit.SECONDS); + private static final TimeValue SHARD_STATS_TIMEOUT = new TimeValue(DEFAULT_TIMEOUT_SECONDS, TimeUnit.SECONDS); + private static final String INDEX_PARAM_NAME = "index"; + private static final String FROM_SEQ_NO_PARAM_NAME = "from_seq_no"; + private static final String MAX_BATCH_SIZE_PARAM_NAME = "max_batch_size"; + private static final String POLL_TIMEOUT_PARAM_NAME = "poll_timeout"; + private static final String MAX_OPERATIONS_COUNT_PARAM_NAME = "max_operations_count"; + + @Override + public String getName() { + return "ccr_shard_changes_action"; + } + + @Override + public List routes() { + return List.of(new Route(GET, "/{index}/ccr/shard_changes")); + } + + /** + * Prepares the request for retrieving shard changes. + * + * @param restRequest The REST request. + * @param client The NodeClient for executing the request. + * @return A RestChannelConsumer for handling the request. + * @throws IOException If an error occurs while preparing the request. + */ + @Override + protected RestChannelConsumer prepareRequest(final RestRequest restRequest, final NodeClient client) throws IOException { + final var indexName = restRequest.param(INDEX_PARAM_NAME); + final var fromSeqNo = restRequest.paramAsLong(FROM_SEQ_NO_PARAM_NAME, DEFAULT_FROM_SEQ_NO); + final var maxBatchSize = restRequest.paramAsSize(MAX_BATCH_SIZE_PARAM_NAME, DEFAULT_MAX_BATCH_SIZE); + final var pollTimeout = restRequest.paramAsTime(POLL_TIMEOUT_PARAM_NAME, DEFAULT_POLL_TIMEOUT); + final var maxOperationsCount = restRequest.paramAsInt(MAX_OPERATIONS_COUNT_PARAM_NAME, DEFAULT_MAX_OPERATIONS_COUNT); + + final CompletableFuture indexUUIDCompletableFuture = asyncGetIndexUUID( + client, + indexName, + client.threadPool().executor(Ccr.CCR_THREAD_POOL_NAME) + ); + final CompletableFuture shardStatsCompletableFuture = asyncShardStats( + client, + indexName, + client.threadPool().executor(Ccr.CCR_THREAD_POOL_NAME) + ); + + return channel -> CompletableFuture.allOf(indexUUIDCompletableFuture, shardStatsCompletableFuture).thenRun(() -> { + try { + final String indexUUID = indexUUIDCompletableFuture.get(DEFAULT_TIMEOUT_SECONDS, TimeUnit.SECONDS); + final ShardStats shardStats = shardStatsCompletableFuture.get(DEFAULT_TIMEOUT_SECONDS, TimeUnit.SECONDS); + final ShardId shardId = shardStats.getShardRouting().shardId(); + final String expectedHistoryUUID = shardStats.getCommitStats().getUserData().get(Engine.HISTORY_UUID_KEY); + + final ShardChangesAction.Request shardChangesRequest = shardChangesRequest( + indexName, + indexUUID, + shardId, + expectedHistoryUUID, + fromSeqNo, + maxBatchSize, + pollTimeout, + maxOperationsCount + ); + client.execute(ShardChangesAction.INSTANCE, shardChangesRequest, new RestActionListener<>(channel) { + @Override + protected void processResponse(final ShardChangesAction.Response response) { + channel.sendResponse(new RestResponse(RestStatus.OK, shardChangesResponseToXContent(response, indexName, shardId))); + } + }); + + } catch (InterruptedException | ExecutionException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException("Error while retrieving shard changes", e); + } catch (TimeoutException te) { + throw new IllegalStateException("Timeout while waiting for shard stats or index UUID", te); + } + }).exceptionally(ex -> { + channel.sendResponse(new RestResponse(RestStatus.BAD_REQUEST, "Failed to process shard changes for index [" + indexName + "]")); + return null; + }); + } + + /** + * Creates a ShardChangesAction.Request object with the provided parameters. + * + * @param indexName The name of the index for which to retrieve shard changes. + * @param indexUUID The UUID of the index. + * @param shardId The ShardId for which to retrieve shard changes. + * @param expectedHistoryUUID The expected history UUID of the shard. + * @param fromSeqNo The sequence number from which to start retrieving shard changes. + * @param maxBatchSize The maximum size of a batch of operations to retrieve. + * @param pollTimeout The maximum time to wait for shard changes. + * @param maxOperationsCount The maximum number of operations to retrieve in a single request. + * @return A ShardChangesAction.Request object with the provided parameters. + */ + private static ShardChangesAction.Request shardChangesRequest( + final String indexName, + final String indexUUID, + final ShardId shardId, + final String expectedHistoryUUID, + long fromSeqNo, + final ByteSizeValue maxBatchSize, + final TimeValue pollTimeout, + int maxOperationsCount + ) { + final ShardChangesAction.Request shardChangesRequest = new ShardChangesAction.Request( + new ShardId(new Index(indexName, indexUUID), shardId.id()), + expectedHistoryUUID + ); + shardChangesRequest.setFromSeqNo(fromSeqNo); + shardChangesRequest.setMaxBatchSize(maxBatchSize); + shardChangesRequest.setPollTimeout(pollTimeout); + shardChangesRequest.setMaxOperationCount(maxOperationsCount); + return shardChangesRequest; + } + + /** + * Converts the response to XContent JSOn format. + * + * @param response The ShardChangesAction response. + * @param indexName The name of the index. + * @param shardId The ShardId. + */ + private static XContentBuilder shardChangesResponseToXContent( + final ShardChangesAction.Response response, + final String indexName, + final ShardId shardId + ) { + try (XContentBuilder builder = XContentFactory.jsonBuilder()) { + builder.startObject(); + builder.field("index", indexName); + builder.field("shard_id", shardId); + builder.field("mapping_version", response.getMappingVersion()); + builder.field("settings_version", response.getSettingsVersion()); + builder.field("aliases_version", response.getAliasesVersion()); + builder.field("global_checkpoint", response.getGlobalCheckpoint()); + builder.field("max_seq_no", response.getMaxSeqNo()); + builder.field("max_seq_no_of_updates_or_deletes", response.getMaxSeqNoOfUpdatesOrDeletes()); + builder.field("took_in_millis", response.getTookInMillis()); + if (response.getOperations() != null && response.getOperations().length > 0) { + operationsToXContent(response, builder); + } + builder.endObject(); + + return builder; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Converts the operations from a ShardChangesAction response to XContent JSON format. + * + * @param response The ShardChangesAction response containing the operations to be converted. + * @param builder The XContentBuilder to which the converted operations will be added. + * @throws IOException If an error occurs while writing to the XContentBuilder. + */ + private static void operationsToXContent(final ShardChangesAction.Response response, final XContentBuilder builder) throws IOException { + builder.field("number_of_operations", response.getOperations().length); + builder.field("operations"); + builder.startArray(); + for (final Translog.Operation operation : response.getOperations()) { + builder.startObject(); + builder.field("op_type", operation.opType()); + builder.field("seq_no", operation.seqNo()); + builder.field("primary_term", operation.primaryTerm()); + builder.endObject(); + } + builder.endArray(); + } + + /** + * Execute an asynchronous task using a task supplier and an executor service. + * + * @param The type of data to be retrieved. + * @param task The supplier task that provides the data. + * @param executorService The executorService service for executing the asynchronous task. + * @param errorMessage The error message to be thrown if the task execution fails. + * @return A CompletableFuture that completes with the retrieved data. + */ + private static CompletableFuture supplyAsyncTask( + final Supplier task, + final ExecutorService executorService, + final String errorMessage + ) { + return CompletableFuture.supplyAsync(() -> { + try { + return task.get(); + } catch (Exception e) { + throw new ElasticsearchException(errorMessage, e); + } + }, executorService); + } + + /** + * Asynchronously retrieves the shard stats for a given index using an executor service. + * + * @param client The NodeClient for executing the asynchronous request. + * @param indexName The name of the index for which to retrieve shard statistics. + * @param executorService The executorService service for executing the asynchronous task. + * @return A CompletableFuture that completes with the retrieved ShardStats. + * @throws ElasticsearchException If an error occurs while retrieving shard statistics. + */ + private static CompletableFuture asyncShardStats( + final NodeClient client, + final String indexName, + final ExecutorService executorService + ) { + return supplyAsyncTask( + () -> Arrays.stream(client.admin().indices().prepareStats(indexName).clear().get(SHARD_STATS_TIMEOUT).getShards()) + .max(Comparator.comparingLong(shardStats -> shardStats.getCommitStats().getGeneration())) + .orElseThrow(() -> new ElasticsearchException("Unable to retrieve shard stats for index: " + indexName)), + executorService, + "Error while retrieving shard stats for index [" + indexName + "]" + ); + } + + /** + * Asynchronously retrieves the index UUID for a given index using an executor service. + * + * @param client The NodeClient for executing the asynchronous request. + * @param indexName The name of the index for which to retrieve the index UUID. + * @param executorService The executorService service for executing the asynchronous task. + * @return A CompletableFuture that completes with the retrieved index UUID. + * @throws ElasticsearchException If an error occurs while retrieving the index UUID. + */ + private static CompletableFuture asyncGetIndexUUID( + final NodeClient client, + final String indexName, + final ExecutorService executorService + ) { + return supplyAsyncTask( + () -> client.admin() + .indices() + .prepareGetIndex() + .setIndices(indexName) + .get(GET_INDEX_UUID_TIMEOUT) + .getSetting(indexName, IndexMetadata.SETTING_INDEX_UUID), + executorService, + "Error while retrieving index UUID for index [" + indexName + "]" + ); + } +} From 8d1f4565a6b4ef169e926d356519963fdbe78436 Mon Sep 17 00:00:00 2001 From: Mark Vieira Date: Mon, 16 Dec 2024 08:45:06 -0800 Subject: [PATCH 13/44] Update IronBank hardening manifest maintainers (#118175) --- .../docker/src/docker/iron_bank/hardening_manifest.yaml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/distribution/docker/src/docker/iron_bank/hardening_manifest.yaml b/distribution/docker/src/docker/iron_bank/hardening_manifest.yaml index f4364c5008c09..e3bdac51cc5c5 100644 --- a/distribution/docker/src/docker/iron_bank/hardening_manifest.yaml +++ b/distribution/docker/src/docker/iron_bank/hardening_manifest.yaml @@ -50,9 +50,12 @@ resources: # List of project maintainers maintainers: - - name: "Rory Hunter" - email: "rory.hunter@elastic.co" - username: "rory" + - name: "Mark Vieira" + email: "mark.vieira@elastic.co" + username: "mark-vieira" + - name: "Rene Gröschke" + email: "rene.groschke@elastic.co" + username: "breskeby" - email: "klepal_alexander@bah.com" name: "Alexander Klepal" username: "alexander.klepal" From bf1c0fe0778f1c5af90e6ad1b3da38f444ad114c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Slobodan=20Adamovi=C4=87?= Date: Mon, 16 Dec 2024 19:15:28 +0100 Subject: [PATCH 14/44] Make reserved built-in roles queryable (#117581) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR makes reserved [built-in roles](https://www.elastic.co/guide/en/elasticsearch/reference/current/built-in-roles.html) queryable via [Query Role API](https://www.elastic.co/guide/en/elasticsearch/reference/current/security-api-query-role.html) by indexing them into the `.security` index. Currently, the built-in roles were only available via [Get Role API](https://www.elastic.co/guide/en/elasticsearch/reference/current/security-api-get-role.html). The built-in roles are synced into the `.security` index on cluster recovery. The `.security` index will be created (if it's not existing) before built-in roles are synced. In order to avoid concurrent updates, the built-in roles will only be synced by a master node. Once the built-in roles are synced, the information about indexed roles is kept in the cluster state as part of the `.security` index's metadata. The map containing role names and their digests is persisted as part of `queryable_built_in_roles_digest` property: ``` GET /_cluster/state/metadata/.security "queryable_built_in_roles_digest": { "superuser": "lRRmA3kPO1/ztr3ESAlTetOuDjgUC3fKcGS3ZCqM+6k=", ... } ``` Important: The reserved roles stored in the `.security` index are only intended to be used for querying and retrieving. The role resolution and mapping during authentication will remain the same and give a priority to static/file role definitions. This is ensured by the [order in which role providers (built-in, file and native) are invoked](https://github.com/elastic/elasticsearch/blob/71c252c274aa967d5a66f7d081291ac5d87d27a9/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/RoleProviders.java#L77-L81). It’s important to note this because there can be a short period of time where we have a temporary inconsistency between actual built-in role definitions and what is stored in the `.security` index. --- Note: The functionality is temporarily hidden behind the `es.queryable_built_in_roles_enabled` system property. By default, the flag is disabled and will become enabled in a followup PR. The reason for this is to keep this PR as small as possible and to avoid the need to adjust a large number of tests that don't expect `.security` index to exist. Testing: To run and test locally execute `./gradlew run -Dtests.jvm.argline="-Des.queryable_built_in_roles_enabled=true"`. To query all reserved built-in roles execute: ``` POST /_security/_query/role { "query": { "bool": { "must": { "term": { "metadata._reserved": true } } } } } ``` --- docs/changelog/117581.yaml | 5 + .../test/rest/ESRestTestCase.java | 2 +- .../security/qa/security-basic/build.gradle | 17 +- .../xpack/security/QueryRoleIT.java | 2 +- .../security/QueryableReservedRolesIT.java | 354 ++++++++++++ .../src/main/java/module-info.java | 6 + .../role/QueryableBuiltInRolesTestPlugin.java | 22 + .../security/src/main/java/module-info.java | 2 + .../xpack/security/Security.java | 23 +- .../xpack/security/SecurityFeatures.java | 8 +- .../action/role/TransportGetRolesAction.java | 12 +- .../security/authz/store/FileRolesStore.java | 9 + .../authz/store/NativeRolesStore.java | 50 +- .../rest/action/role/RestQueryRoleAction.java | 3 + .../support/FeatureNotEnabledException.java | 1 + .../support/QueryableBuiltInRoles.java | 52 ++ .../QueryableBuiltInRolesProviderFactory.java | 23 + .../QueryableBuiltInRolesSynchronizer.java | 532 ++++++++++++++++++ .../support/QueryableBuiltInRolesUtils.java | 101 ++++ .../QueryableReservedRolesProvider.java | 56 ++ .../support/SecurityIndexManager.java | 2 +- .../QueryableBuiltInRolesUtilsTests.java | 296 ++++++++++ .../QueryableReservedRolesProviderTests.java | 31 + 23 files changed, 1585 insertions(+), 24 deletions(-) create mode 100644 docs/changelog/117581.yaml create mode 100644 x-pack/plugin/security/qa/security-basic/src/javaRestTest/java/org/elasticsearch/xpack/security/QueryableReservedRolesIT.java create mode 100644 x-pack/plugin/security/qa/security-basic/src/main/java/module-info.java create mode 100644 x-pack/plugin/security/qa/security-basic/src/main/java/org/elasticsearch/xpack/security/role/QueryableBuiltInRolesTestPlugin.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRoles.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesProviderFactory.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesSynchronizer.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesUtils.java create mode 100644 x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableReservedRolesProvider.java create mode 100644 x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesUtilsTests.java create mode 100644 x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/QueryableReservedRolesProviderTests.java diff --git a/docs/changelog/117581.yaml b/docs/changelog/117581.yaml new file mode 100644 index 0000000000000..b88017f45e9c9 --- /dev/null +++ b/docs/changelog/117581.yaml @@ -0,0 +1,5 @@ +pr: 117581 +summary: Make reserved built-in roles queryable +area: Authorization +type: enhancement +issues: [] diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 4428afaaeabe5..fa525705a9b39 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -1138,7 +1138,7 @@ protected static void wipeAllIndices(boolean preserveSecurityIndices) throws IOE } } - private static boolean ignoreSystemIndexAccessWarnings(List warnings) { + protected static boolean ignoreSystemIndexAccessWarnings(List warnings) { for (String warning : warnings) { if (warning.startsWith("this request accesses system indices:")) { SUITE_LOGGER.warn("Ignoring system index access warning during test cleanup: {}", warning); diff --git a/x-pack/plugin/security/qa/security-basic/build.gradle b/x-pack/plugin/security/qa/security-basic/build.gradle index 8740354646346..e6caf943dc023 100644 --- a/x-pack/plugin/security/qa/security-basic/build.gradle +++ b/x-pack/plugin/security/qa/security-basic/build.gradle @@ -4,20 +4,31 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ - +apply plugin: 'elasticsearch.base-internal-es-plugin' apply plugin: 'elasticsearch.internal-java-rest-test' + +esplugin { + name 'queryable-reserved-roles-test' + description 'A test plugin for testing that changes to reserved roles are made queryable' + classname 'org.elasticsearch.xpack.security.role.QueryableBuiltInRolesTestPlugin' + extendedPlugins = ['x-pack-core', 'x-pack-security'] +} dependencies { javaRestTestImplementation(testArtifact(project(xpackModule('security')))) javaRestTestImplementation(testArtifact(project(xpackModule('core')))) + compileOnly project(':x-pack:plugin:core') + compileOnly project(':x-pack:plugin:security') + clusterPlugins project(':x-pack:plugin:security:qa:security-basic') } tasks.named('javaRestTest') { usesDefaultDistribution() } +tasks.named("javadoc").configure { enabled = false } -if (buildParams.inFipsJvm){ +if (buildParams.inFipsJvm) { // This test cluster is using a BASIC license and FIPS 140 mode is not supported in BASIC - tasks.named("javaRestTest").configure{enabled = false } + tasks.named("javaRestTest").configure { enabled = false } } diff --git a/x-pack/plugin/security/qa/security-basic/src/javaRestTest/java/org/elasticsearch/xpack/security/QueryRoleIT.java b/x-pack/plugin/security/qa/security-basic/src/javaRestTest/java/org/elasticsearch/xpack/security/QueryRoleIT.java index 1588749b9a331..311510352d805 100644 --- a/x-pack/plugin/security/qa/security-basic/src/javaRestTest/java/org/elasticsearch/xpack/security/QueryRoleIT.java +++ b/x-pack/plugin/security/qa/security-basic/src/javaRestTest/java/org/elasticsearch/xpack/security/QueryRoleIT.java @@ -496,7 +496,7 @@ private RoleDescriptor createRole( ); } - private void assertQuery(String body, int total, Consumer>> roleVerifier) throws IOException { + static void assertQuery(String body, int total, Consumer>> roleVerifier) throws IOException { assertQuery(client(), body, total, roleVerifier); } diff --git a/x-pack/plugin/security/qa/security-basic/src/javaRestTest/java/org/elasticsearch/xpack/security/QueryableReservedRolesIT.java b/x-pack/plugin/security/qa/security-basic/src/javaRestTest/java/org/elasticsearch/xpack/security/QueryableReservedRolesIT.java new file mode 100644 index 0000000000000..7adff21d8df4f --- /dev/null +++ b/x-pack/plugin/security/qa/security-basic/src/javaRestTest/java/org/elasticsearch/xpack/security/QueryableReservedRolesIT.java @@ -0,0 +1,354 @@ +/* + * 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.security; + +import com.carrotsearch.randomizedtesting.annotations.TestCaseOrdering; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.SecureString; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.test.AnnotationTestOrdering; +import org.elasticsearch.test.AnnotationTestOrdering.Order; +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.cluster.MutableSettingsProvider; +import org.elasticsearch.test.cluster.local.distribution.DistributionType; +import org.elasticsearch.test.cluster.local.model.User; +import org.elasticsearch.test.cluster.util.resource.Resource; +import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.ObjectPath; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; +import org.elasticsearch.xpack.core.security.test.TestRestrictedIndices; +import org.elasticsearch.xpack.security.support.QueryableBuiltInRolesSynchronizer; +import org.elasticsearch.xpack.security.support.SecurityMigrations; +import org.junit.BeforeClass; +import org.junit.ClassRule; + +import java.io.IOException; +import java.io.InputStream; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.elasticsearch.xpack.core.security.test.TestRestrictedIndices.INTERNAL_SECURITY_MAIN_INDEX_7; +import static org.elasticsearch.xpack.security.QueryRoleIT.assertQuery; +import static org.elasticsearch.xpack.security.QueryRoleIT.waitForMigrationCompletion; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.iterableWithSize; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; +import static org.hamcrest.Matchers.oneOf; + +@TestCaseOrdering(AnnotationTestOrdering.class) +public class QueryableReservedRolesIT extends ESRestTestCase { + + protected static final String REST_USER = "security_test_user"; + private static final SecureString REST_PASSWORD = new SecureString("security-test-password".toCharArray()); + private static final String ADMIN_USER = "admin_user"; + private static final SecureString ADMIN_PASSWORD = new SecureString("admin-password".toCharArray()); + protected static final String READ_SECURITY_USER = "read_security_user"; + private static final SecureString READ_SECURITY_PASSWORD = new SecureString("read-security-password".toCharArray()); + + @BeforeClass + public static void setup() { + new ReservedRolesStore(); + } + + @Override + protected boolean preserveClusterUponCompletion() { + return true; + } + + private static MutableSettingsProvider clusterSettings = new MutableSettingsProvider() { + { + put("xpack.license.self_generated.type", "basic"); + put("xpack.security.enabled", "true"); + put("xpack.security.http.ssl.enabled", "false"); + put("xpack.security.transport.ssl.enabled", "false"); + } + }; + + @ClassRule + public static ElasticsearchCluster cluster = ElasticsearchCluster.local() + .distribution(DistributionType.DEFAULT) + .nodes(2) + .settings(clusterSettings) + .rolesFile(Resource.fromClasspath("roles.yml")) + .user(ADMIN_USER, ADMIN_PASSWORD.toString(), User.ROOT_USER_ROLE, true) + .user(REST_USER, REST_PASSWORD.toString(), "security_test_role", false) + .user(READ_SECURITY_USER, READ_SECURITY_PASSWORD.toString(), "read_security_user_role", false) + .systemProperty("es.queryable_built_in_roles_enabled", "true") + .plugin("queryable-reserved-roles-test") + .build(); + + private static Set PREVIOUS_RESERVED_ROLES; + private static Set CONFIGURED_RESERVED_ROLES; + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + @Override + protected Settings restAdminSettings() { + String token = basicAuthHeaderValue(ADMIN_USER, ADMIN_PASSWORD); + return Settings.builder().put(ThreadContext.PREFIX + ".Authorization", token).build(); + } + + @Override + protected Settings restClientSettings() { + String token = basicAuthHeaderValue(REST_USER, REST_PASSWORD); + return Settings.builder().put(ThreadContext.PREFIX + ".Authorization", token).build(); + } + + @Order(10) + public void testQueryDeleteOrUpdateReservedRoles() throws Exception { + waitForMigrationCompletion(adminClient(), SecurityMigrations.ROLE_METADATA_FLATTENED_MIGRATION_VERSION); + + final String[] allReservedRoles = ReservedRolesStore.names().toArray(new String[0]); + assertQuery(client(), """ + { "query": { "bool": { "must": { "term": { "metadata._reserved": true } } } }, "size": 100 } + """, allReservedRoles.length, roles -> { + assertThat(roles, iterableWithSize(allReservedRoles.length)); + for (var role : roles) { + assertThat((String) role.get("name"), is(oneOf(allReservedRoles))); + } + }); + + final String roleName = randomFrom(allReservedRoles); + assertQuery(client(), String.format(""" + { "query": { "bool": { "must": { "term": { "name": "%s" } } } } } + """, roleName), 1, roles -> { + assertThat(roles, iterableWithSize(1)); + assertThat((String) roles.get(0).get("name"), equalTo(roleName)); + }); + + assertCannotDeleteReservedRoles(); + assertCannotCreateOrUpdateReservedRole(roleName); + } + + @Order(11) + public void testGetReservedRoles() throws Exception { + final String[] allReservedRoles = ReservedRolesStore.names().toArray(new String[0]); + final String roleName = randomFrom(allReservedRoles); + Request request = new Request("GET", "/_security/role/" + roleName); + Response response = adminClient().performRequest(request); + assertOK(response); + var responseMap = responseAsMap(response); + assertThat(responseMap.size(), equalTo(1)); + assertThat(responseMap.containsKey(roleName), is(true)); + } + + @Order(20) + public void testRestartForConfiguringReservedRoles() throws Exception { + configureReservedRoles(List.of("editor", "viewer", "kibana_system", "apm_system", "beats_system", "logstash_system")); + cluster.restart(false); + closeClients(); + } + + @Order(30) + public void testConfiguredReservedRoles() throws Exception { + assert CONFIGURED_RESERVED_ROLES != null; + + // Test query roles API + assertBusy(() -> { + assertQuery(client(), """ + { "query": { "bool": { "must": { "term": { "metadata._reserved": true } } } }, "size": 100 } + """, CONFIGURED_RESERVED_ROLES.size(), roles -> { + assertThat(roles, iterableWithSize(CONFIGURED_RESERVED_ROLES.size())); + for (var role : roles) { + assertThat((String) role.get("name"), is(oneOf(CONFIGURED_RESERVED_ROLES.toArray(new String[0])))); + } + }); + }, 30, TimeUnit.SECONDS); + + // Test get roles API + assertBusy(() -> { + final Response response = adminClient().performRequest(new Request("GET", "/_security/role")); + assertOK(response); + final Map responseMap = responseAsMap(response); + assertThat(responseMap.keySet(), equalTo(CONFIGURED_RESERVED_ROLES)); + }); + } + + @Order(40) + public void testRestartForConfiguringReservedRolesAndClosingIndex() throws Exception { + configureReservedRoles(List.of("editor", "viewer")); + closeSecurityIndex(); + cluster.restart(false); + closeClients(); + } + + @Order(50) + public void testConfiguredReservedRolesAfterClosingAndOpeningIndex() throws Exception { + assert CONFIGURED_RESERVED_ROLES != null; + assert PREVIOUS_RESERVED_ROLES != null; + assertThat(PREVIOUS_RESERVED_ROLES, is(not(equalTo(CONFIGURED_RESERVED_ROLES)))); + + // Test configured roles did not get updated because the security index is closed + assertMetadataContainsBuiltInRoles(PREVIOUS_RESERVED_ROLES); + + // Open the security index + openSecurityIndex(); + + // Test that the roles are now updated after index got opened + assertBusy(() -> { + assertQuery(client(), """ + { "query": { "bool": { "must": { "term": { "metadata._reserved": true } } } }, "size": 100 } + """, CONFIGURED_RESERVED_ROLES.size(), roles -> { + assertThat(roles, iterableWithSize(CONFIGURED_RESERVED_ROLES.size())); + for (var role : roles) { + assertThat((String) role.get("name"), is(oneOf(CONFIGURED_RESERVED_ROLES.toArray(new String[0])))); + } + }); + }, 30, TimeUnit.SECONDS); + + } + + @Order(60) + public void testDeletingAndCreatingSecurityIndexTriggersSynchronization() throws Exception { + deleteSecurityIndex(); + + assertBusy(this::assertSecurityIndexDeleted, 30, TimeUnit.SECONDS); + + // Creating a user will trigger .security index creation + createUser("superman", "superman", "superuser"); + + // Test that the roles are now updated after index got created + assertBusy(() -> { + assertQuery(client(), """ + { "query": { "bool": { "must": { "term": { "metadata._reserved": true } } } }, "size": 100 } + """, CONFIGURED_RESERVED_ROLES.size(), roles -> { + assertThat(roles, iterableWithSize(CONFIGURED_RESERVED_ROLES.size())); + for (var role : roles) { + assertThat((String) role.get("name"), is(oneOf(CONFIGURED_RESERVED_ROLES.toArray(new String[0])))); + } + }); + }, 30, TimeUnit.SECONDS); + } + + private void createUser(String name, String password, String role) throws IOException { + Request request = new Request("PUT", "/_security/user/" + name); + request.setJsonEntity("{ \"password\": \"" + password + "\", \"roles\": [ \"" + role + "\"] }"); + assertOK(adminClient().performRequest(request)); + } + + private void deleteSecurityIndex() throws IOException { + final Request deleteRequest = new Request("DELETE", INTERNAL_SECURITY_MAIN_INDEX_7); + deleteRequest.setOptions(RequestOptions.DEFAULT.toBuilder().setWarningsHandler(ESRestTestCase::ignoreSystemIndexAccessWarnings)); + final Response response = adminClient().performRequest(deleteRequest); + try (InputStream is = response.getEntity().getContent()) { + assertTrue((boolean) XContentHelper.convertToMap(XContentType.JSON.xContent(), is, true).get("acknowledged")); + } + } + + private void assertMetadataContainsBuiltInRoles(Set builtInRoles) throws IOException { + final Request request = new Request("GET", "_cluster/state/metadata/" + INTERNAL_SECURITY_MAIN_INDEX_7); + final Response response = adminClient().performRequest(request); + assertOK(response); + final Map builtInRolesDigests = ObjectPath.createFromResponse(response) + .evaluate("metadata.indices.\\.security-7." + QueryableBuiltInRolesSynchronizer.METADATA_QUERYABLE_BUILT_IN_ROLES_DIGEST_KEY); + assertThat(builtInRolesDigests.keySet(), equalTo(builtInRoles)); + } + + private void assertSecurityIndexDeleted() throws IOException { + final Request request = new Request("GET", "_cluster/state/metadata/" + INTERNAL_SECURITY_MAIN_INDEX_7); + final Response response = adminClient().performRequest(request); + assertOK(response); + final Map securityIndexMetadata = ObjectPath.createFromResponse(response) + .evaluate("metadata.indices.\\.security-7"); + assertThat(securityIndexMetadata, is(nullValue())); + } + + private void configureReservedRoles(List reservedRoles) throws Exception { + PREVIOUS_RESERVED_ROLES = CONFIGURED_RESERVED_ROLES; + CONFIGURED_RESERVED_ROLES = new HashSet<>(); + CONFIGURED_RESERVED_ROLES.add("superuser"); // superuser must always be included + CONFIGURED_RESERVED_ROLES.addAll(reservedRoles); + clusterSettings.put("xpack.security.reserved_roles.include", Strings.collectionToCommaDelimitedString(CONFIGURED_RESERVED_ROLES)); + } + + private void closeSecurityIndex() throws Exception { + Request request = new Request("POST", "/" + TestRestrictedIndices.INTERNAL_SECURITY_MAIN_INDEX_7 + "/_close"); + request.setOptions( + expectWarnings( + "this request accesses system indices: [.security-7], but in a future major version, " + + "direct access to system indices will be prevented by default" + ) + ); + Response response = adminClient().performRequest(request); + assertOK(response); + } + + private void openSecurityIndex() throws Exception { + Request request = new Request("POST", "/" + TestRestrictedIndices.INTERNAL_SECURITY_MAIN_INDEX_7 + "/_open"); + request.setOptions( + expectWarnings( + "this request accesses system indices: [.security-7], but in a future major version, " + + "direct access to system indices will be prevented by default" + ) + ); + Response response = adminClient().performRequest(request); + assertOK(response); + } + + private void assertCannotDeleteReservedRoles() throws Exception { + { + String roleName = randomFrom(ReservedRolesStore.names()); + Request request = new Request("DELETE", "/_security/role/" + roleName); + var e = expectThrows(ResponseException.class, () -> adminClient().performRequest(request)); + assertThat(e.getMessage(), containsString("role [" + roleName + "] is reserved and cannot be deleted")); + } + { + Request request = new Request("DELETE", "/_security/role/"); + request.setJsonEntity( + """ + { + "names": [%s] + } + """.formatted( + ReservedRolesStore.names().stream().map(name -> "\"" + name + "\"").reduce((a, b) -> a + ", " + b).orElse("") + ) + ); + Response response = adminClient().performRequest(request); + assertOK(response); + String responseAsString = responseAsMap(response).toString(); + for (String roleName : ReservedRolesStore.names()) { + assertThat(responseAsString, containsString("role [" + roleName + "] is reserved and cannot be deleted")); + } + } + } + + private void assertCannotCreateOrUpdateReservedRole(String roleName) throws Exception { + Request request = new Request(randomBoolean() ? "PUT" : "POST", "/_security/role/" + roleName); + request.setJsonEntity(""" + { + "cluster": ["all"], + "indices": [ + { + "names": ["*"], + "privileges": ["all"] + } + ] + } + """); + var e = expectThrows(ResponseException.class, () -> adminClient().performRequest(request)); + assertThat(e.getMessage(), containsString("Role [" + roleName + "] is reserved and may not be used.")); + } + +} diff --git a/x-pack/plugin/security/qa/security-basic/src/main/java/module-info.java b/x-pack/plugin/security/qa/security-basic/src/main/java/module-info.java new file mode 100644 index 0000000000000..00c8e480cfbaf --- /dev/null +++ b/x-pack/plugin/security/qa/security-basic/src/main/java/module-info.java @@ -0,0 +1,6 @@ +module org.elasticsearch.internal.security { + requires org.elasticsearch.base; + requires org.elasticsearch.server; + requires org.elasticsearch.xcore; + requires org.elasticsearch.security; +} diff --git a/x-pack/plugin/security/qa/security-basic/src/main/java/org/elasticsearch/xpack/security/role/QueryableBuiltInRolesTestPlugin.java b/x-pack/plugin/security/qa/security-basic/src/main/java/org/elasticsearch/xpack/security/role/QueryableBuiltInRolesTestPlugin.java new file mode 100644 index 0000000000000..ba5538d992cfb --- /dev/null +++ b/x-pack/plugin/security/qa/security-basic/src/main/java/org/elasticsearch/xpack/security/role/QueryableBuiltInRolesTestPlugin.java @@ -0,0 +1,22 @@ +/* + * 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.security.role; + +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; + +import java.util.List; + +public class QueryableBuiltInRolesTestPlugin extends Plugin { + + @Override + public List> getSettings() { + return List.of(ReservedRolesStore.INCLUDED_RESERVED_ROLES_SETTING); + } +} diff --git a/x-pack/plugin/security/src/main/java/module-info.java b/x-pack/plugin/security/src/main/java/module-info.java index a072b34da7e96..947211559b0c2 100644 --- a/x-pack/plugin/security/src/main/java/module-info.java +++ b/x-pack/plugin/security/src/main/java/module-info.java @@ -70,6 +70,8 @@ exports org.elasticsearch.xpack.security.slowlog to org.elasticsearch.server; exports org.elasticsearch.xpack.security.authc.support to org.elasticsearch.internal.security; exports org.elasticsearch.xpack.security.rest.action.apikey to org.elasticsearch.internal.security; + exports org.elasticsearch.xpack.security.support to org.elasticsearch.internal.security; + exports org.elasticsearch.xpack.security.authz.store to org.elasticsearch.internal.security; provides org.elasticsearch.index.SlowLogFieldProvider with org.elasticsearch.xpack.security.slowlog.SecuritySlowLogFieldProvider; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java index ef66392a87260..fd530a338b26c 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java @@ -411,6 +411,8 @@ import org.elasticsearch.xpack.security.rest.action.user.RestSetEnabledAction; import org.elasticsearch.xpack.security.support.CacheInvalidatorRegistry; import org.elasticsearch.xpack.security.support.ExtensionComponents; +import org.elasticsearch.xpack.security.support.QueryableBuiltInRolesProviderFactory; +import org.elasticsearch.xpack.security.support.QueryableBuiltInRolesSynchronizer; import org.elasticsearch.xpack.security.support.ReloadableSecurityComponent; import org.elasticsearch.xpack.security.support.SecurityIndexManager; import org.elasticsearch.xpack.security.support.SecurityMigrationExecutor; @@ -461,6 +463,7 @@ import static org.elasticsearch.xpack.core.security.SecurityField.FIELD_LEVEL_SECURITY_FEATURE; import static org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore.INCLUDED_RESERVED_ROLES_SETTING; import static org.elasticsearch.xpack.security.operator.OperatorPrivileges.OPERATOR_PRIVILEGES_ENABLED; +import static org.elasticsearch.xpack.security.support.QueryableBuiltInRolesSynchronizer.QUERYABLE_BUILT_IN_ROLES_ENABLED; import static org.elasticsearch.xpack.security.transport.SSLEngineUtils.extractClientCertificates; public class Security extends Plugin @@ -631,7 +634,7 @@ public class Security extends Plugin private final SetOnce reservedRoleNameCheckerFactory = new SetOnce<>(); private final SetOnce fileRoleValidator = new SetOnce<>(); private final SetOnce secondaryAuthActions = new SetOnce<>(); - + private final SetOnce queryableRolesProviderFactory = new SetOnce<>(); private final SetOnce securityMigrationExecutor = new SetOnce<>(); // Node local retry count for migration jobs that's checked only on the master node to make sure @@ -1202,6 +1205,23 @@ Collection createComponents( reservedRoleMappingAction.set(new ReservedRoleMappingAction()); + if (QUERYABLE_BUILT_IN_ROLES_ENABLED) { + if (queryableRolesProviderFactory.get() == null) { + queryableRolesProviderFactory.set(new QueryableBuiltInRolesProviderFactory.Default()); + } + components.add( + new QueryableBuiltInRolesSynchronizer( + clusterService, + featureService, + queryableRolesProviderFactory.get(), + nativeRolesStore, + reservedRolesStore, + fileRolesStore.get(), + threadPool + ) + ); + } + cacheInvalidatorRegistry.validate(); final List reloadableComponents = new ArrayList<>(); @@ -2317,6 +2337,7 @@ public void loadExtensions(ExtensionLoader loader) { loadSingletonExtensionAndSetOnce(loader, grantApiKeyRequestTranslator, RestGrantApiKeyAction.RequestTranslator.class); loadSingletonExtensionAndSetOnce(loader, fileRoleValidator, FileRoleValidator.class); loadSingletonExtensionAndSetOnce(loader, secondaryAuthActions, SecondaryAuthActions.class); + loadSingletonExtensionAndSetOnce(loader, queryableRolesProviderFactory, QueryableBuiltInRolesProviderFactory.class); } private void loadSingletonExtensionAndSetOnce(ExtensionLoader loader, SetOnce setOnce, Class clazz) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityFeatures.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityFeatures.java index 53ecafa280715..84749d895a44e 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityFeatures.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/SecurityFeatures.java @@ -12,6 +12,7 @@ import java.util.Set; +import static org.elasticsearch.xpack.security.support.QueryableBuiltInRolesSynchronizer.QUERYABLE_BUILT_IN_ROLES_FEATURE; import static org.elasticsearch.xpack.security.support.SecuritySystemIndices.SECURITY_MIGRATION_FRAMEWORK; import static org.elasticsearch.xpack.security.support.SecuritySystemIndices.SECURITY_ROLES_METADATA_FLATTENED; import static org.elasticsearch.xpack.security.support.SecuritySystemIndices.SECURITY_ROLE_MAPPING_CLEANUP; @@ -20,6 +21,11 @@ public class SecurityFeatures implements FeatureSpecification { @Override public Set getFeatures() { - return Set.of(SECURITY_ROLE_MAPPING_CLEANUP, SECURITY_ROLES_METADATA_FLATTENED, SECURITY_MIGRATION_FRAMEWORK); + return Set.of( + SECURITY_ROLE_MAPPING_CLEANUP, + SECURITY_ROLES_METADATA_FLATTENED, + SECURITY_MIGRATION_FRAMEWORK, + QUERYABLE_BUILT_IN_ROLES_FEATURE + ); } } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/role/TransportGetRolesAction.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/role/TransportGetRolesAction.java index e019f168cf8c0..cdeac51e1f492 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/role/TransportGetRolesAction.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/role/TransportGetRolesAction.java @@ -20,11 +20,9 @@ import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; import org.elasticsearch.xpack.security.authz.store.NativeRolesStore; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashSet; -import java.util.List; +import java.util.LinkedHashSet; import java.util.Set; import java.util.stream.Collectors; @@ -51,8 +49,8 @@ protected void doExecute(Task task, final GetRolesRequest request, final ActionL return; } - final Set rolesToSearchFor = new HashSet<>(); - final List reservedRoles = new ArrayList<>(); + final Set rolesToSearchFor = new LinkedHashSet<>(); + final Set reservedRoles = new LinkedHashSet<>(); if (specificRolesRequested) { for (String role : requestedRoles) { if (ReservedRolesStore.isReserved(role)) { @@ -80,10 +78,10 @@ protected void doExecute(Task task, final GetRolesRequest request, final ActionL } private void getNativeRoles(Set rolesToSearchFor, ActionListener listener) { - getNativeRoles(rolesToSearchFor, new ArrayList<>(), listener); + getNativeRoles(rolesToSearchFor, new LinkedHashSet<>(), listener); } - private void getNativeRoles(Set rolesToSearchFor, List foundRoles, ActionListener listener) { + private void getNativeRoles(Set rolesToSearchFor, Set foundRoles, ActionListener listener) { nativeRolesStore.getRoleDescriptors(rolesToSearchFor, ActionListener.wrap((retrievalResult) -> { if (retrievalResult.isSuccess()) { foundRoles.addAll(retrievalResult.getDescriptors()); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/FileRolesStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/FileRolesStore.java index 7618135c8662f..87378ac0b9f25 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/FileRolesStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/FileRolesStore.java @@ -44,6 +44,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -173,6 +174,14 @@ public Path getFile() { return file; } + /** + * @return a map of all file role definitions. The returned map is unmodifiable. + */ + public Map getAllRoleDescriptors() { + final Map localPermissions = permissions; + return Collections.unmodifiableMap(localPermissions); + } + // package private for testing Set getAllRoleNames() { return permissions.keySet(); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java index 23a1fc188e4a0..0a5865ecfe9bf 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/store/NativeRolesStore.java @@ -63,13 +63,13 @@ import org.elasticsearch.xpack.core.security.authz.privilege.ConfigurableClusterPrivileges; import org.elasticsearch.xpack.core.security.authz.store.RoleRetrievalResult; import org.elasticsearch.xpack.core.security.authz.support.DLSRoleQueryValidator; -import org.elasticsearch.xpack.core.security.support.NativeRealmValidationUtil; import org.elasticsearch.xpack.security.authz.ReservedRoleNameChecker; import org.elasticsearch.xpack.security.support.SecurityIndexManager; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -169,6 +169,10 @@ public NativeRolesStore( this.enabled = settings.getAsBoolean(NATIVE_ROLES_ENABLED, true); } + public boolean isEnabled() { + return enabled; + } + @Override public void accept(Set names, ActionListener listener) { getRoleDescriptors(names, listener); @@ -263,6 +267,10 @@ public boolean isMetadataSearchable() { } public void queryRoleDescriptors(SearchSourceBuilder searchSourceBuilder, ActionListener listener) { + if (enabled == false) { + listener.onResponse(QueryRoleResult.EMPTY); + return; + } SearchRequest searchRequest = new SearchRequest(new String[] { SECURITY_MAIN_ALIAS }, searchSourceBuilder); SecurityIndexManager frozenSecurityIndex = securityIndex.defensiveCopy(); if (frozenSecurityIndex.indexExists() == false) { @@ -345,6 +353,15 @@ public void deleteRoles( final List roleNames, WriteRequest.RefreshPolicy refreshPolicy, final ActionListener listener + ) { + deleteRoles(roleNames, refreshPolicy, true, listener); + } + + public void deleteRoles( + final Collection roleNames, + WriteRequest.RefreshPolicy refreshPolicy, + boolean validateRoleNames, + final ActionListener listener ) { if (enabled == false) { listener.onFailure(new IllegalStateException("Native role management is disabled")); @@ -355,7 +372,7 @@ public void deleteRoles( Map validationErrorByRoleName = new HashMap<>(); for (String roleName : roleNames) { - if (reservedRoleNameChecker.isReserved(roleName)) { + if (validateRoleNames && reservedRoleNameChecker.isReserved(roleName)) { validationErrorByRoleName.put( roleName, new IllegalArgumentException("role [" + roleName + "] is reserved and cannot be deleted") @@ -402,7 +419,7 @@ public void onFailure(Exception e) { } private void bulkResponseAndRefreshRolesCache( - List roleNames, + Collection roleNames, BulkResponse bulkResponse, Map validationErrorByRoleName, ActionListener listener @@ -430,7 +447,7 @@ private void bulkResponseAndRefreshRolesCache( } private void bulkResponseWithOnlyValidationErrors( - List roleNames, + Collection roleNames, Map validationErrorByRoleName, ActionListener listener ) { @@ -542,7 +559,16 @@ public void onFailure(Exception e) { public void putRoles( final WriteRequest.RefreshPolicy refreshPolicy, - final List roles, + final Collection roles, + final ActionListener listener + ) { + putRoles(refreshPolicy, roles, true, listener); + } + + public void putRoles( + final WriteRequest.RefreshPolicy refreshPolicy, + final Collection roles, + boolean validateRoleDescriptors, final ActionListener listener ) { if (enabled == false) { @@ -555,7 +581,7 @@ public void putRoles( for (RoleDescriptor role : roles) { Exception validationException; try { - validationException = validateRoleDescriptor(role); + validationException = validateRoleDescriptors ? validateRoleDescriptor(role) : null; } catch (Exception e) { validationException = e; } @@ -621,8 +647,6 @@ private DeleteRequest createRoleDeleteRequest(final String roleName) { // Package private for testing XContentBuilder createRoleXContentBuilder(RoleDescriptor role) throws IOException { - assert NativeRealmValidationUtil.validateRoleName(role.getName(), false) == null - : "Role name was invalid or reserved: " + role.getName(); assert false == role.hasRestriction() : "restriction is not supported for native roles"; XContentBuilder builder = jsonBuilder().startObject(); @@ -671,7 +695,11 @@ public void usageStats(ActionListener> listener) { client.prepareMultiSearch() .add( client.prepareSearch(SECURITY_MAIN_ALIAS) - .setQuery(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) + .setQuery( + QueryBuilders.boolQuery() + .must(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) + .mustNot(QueryBuilders.termQuery("metadata_flattened._reserved", true)) + ) .setTrackTotalHits(true) .setSize(0) ) @@ -680,6 +708,7 @@ public void usageStats(ActionListener> listener) { .setQuery( QueryBuilders.boolQuery() .must(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) + .mustNot(QueryBuilders.termQuery("metadata_flattened._reserved", true)) .must( QueryBuilders.boolQuery() .should(existsQuery("indices.field_security.grant")) @@ -697,6 +726,7 @@ public void usageStats(ActionListener> listener) { .setQuery( QueryBuilders.boolQuery() .must(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) + .mustNot(QueryBuilders.termQuery("metadata_flattened._reserved", true)) .filter(existsQuery("indices.query")) ) .setTrackTotalHits(true) @@ -708,6 +738,7 @@ public void usageStats(ActionListener> listener) { .setQuery( QueryBuilders.boolQuery() .must(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) + .mustNot(QueryBuilders.termQuery("metadata_flattened._reserved", true)) .filter(existsQuery("remote_indices")) ) .setTrackTotalHits(true) @@ -718,6 +749,7 @@ public void usageStats(ActionListener> listener) { .setQuery( QueryBuilders.boolQuery() .must(QueryBuilders.termQuery(RoleDescriptor.Fields.TYPE.getPreferredName(), ROLE_TYPE)) + .mustNot(QueryBuilders.termQuery("metadata_flattened._reserved", true)) .filter(existsQuery("remote_cluster")) ) .setTrackTotalHits(true) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/role/RestQueryRoleAction.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/role/RestQueryRoleAction.java index c2dc7166bd3b6..3637159479463 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/role/RestQueryRoleAction.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/role/RestQueryRoleAction.java @@ -14,6 +14,8 @@ import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.license.XPackLicenseState; import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.Scope; +import org.elasticsearch.rest.ServerlessScope; import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.search.searchafter.SearchAfterBuilder; import org.elasticsearch.search.sort.FieldSortBuilder; @@ -32,6 +34,7 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg; +@ServerlessScope(Scope.INTERNAL) public final class RestQueryRoleAction extends NativeRoleBaseRestHandler { @SuppressWarnings("unchecked") diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/FeatureNotEnabledException.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/FeatureNotEnabledException.java index 87c23284c5819..8ba3ebad8a851 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/FeatureNotEnabledException.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/FeatureNotEnabledException.java @@ -29,6 +29,7 @@ public enum Feature { } } + @SuppressWarnings("this-escape") public FeatureNotEnabledException(Feature feature, String message, Object... args) { super(message, args); addMetadata(DISABLED_FEATURE_METADATA, feature.featureName); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRoles.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRoles.java new file mode 100644 index 0000000000000..ec38e4951f45c --- /dev/null +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRoles.java @@ -0,0 +1,52 @@ +/* + * 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.security.support; + +import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; + +import java.util.Collection; +import java.util.Map; + +/** + * A class that holds the built-in roles and their hash digests. + */ +public record QueryableBuiltInRoles(Map rolesDigest, Collection roleDescriptors) { + + /** + * A listener that is notified when the built-in roles change. + */ + public interface Listener { + + /** + * Called when the built-in roles change. + * + * @param roles the new built-in roles. + */ + void onRolesChanged(QueryableBuiltInRoles roles); + + } + + /** + * A provider that provides the built-in roles and can notify subscribed listeners when the built-in roles change. + */ + public interface Provider { + + /** + * @return the built-in roles. + */ + QueryableBuiltInRoles getRoles(); + + /** + * Adds a listener to be notified when the built-in roles change. + * + * @param listener the listener to add. + */ + void addListener(Listener listener); + + } +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesProviderFactory.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesProviderFactory.java new file mode 100644 index 0000000000000..c29b64836d1a5 --- /dev/null +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesProviderFactory.java @@ -0,0 +1,23 @@ +/* + * 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.security.support; + +import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; +import org.elasticsearch.xpack.security.authz.store.FileRolesStore; + +public interface QueryableBuiltInRolesProviderFactory { + + QueryableBuiltInRoles.Provider createProvider(ReservedRolesStore reservedRolesStore, FileRolesStore fileRolesStore); + + class Default implements QueryableBuiltInRolesProviderFactory { + @Override + public QueryableBuiltInRoles.Provider createProvider(ReservedRolesStore reservedRolesStore, FileRolesStore fileRolesStore) { + return new QueryableReservedRolesProvider(reservedRolesStore); + } + } +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesSynchronizer.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesSynchronizer.java new file mode 100644 index 0000000000000..60163434e212f --- /dev/null +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesSynchronizer.java @@ -0,0 +1,532 @@ +/* + * 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.security.support; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.TransportActions; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.ClusterStateTaskListener; +import org.elasticsearch.cluster.NotMasterException; +import org.elasticsearch.cluster.SimpleBatchedExecutor; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.cluster.service.MasterServiceTaskQueue; +import org.elasticsearch.common.Priority; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.component.LifecycleListener; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Strings; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.features.FeatureService; +import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.engine.DocumentMissingException; +import org.elasticsearch.index.engine.VersionConflictEngineException; +import org.elasticsearch.indices.IndexClosedException; +import org.elasticsearch.indices.IndexPrimaryShardNotAllocatedException; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.security.action.role.BulkRolesResponse; +import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; +import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; +import org.elasticsearch.xpack.security.authz.store.FileRolesStore; +import org.elasticsearch.xpack.security.authz.store.NativeRolesStore; + +import java.util.Collection; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; + +import static java.util.stream.Collectors.toMap; +import static java.util.stream.Collectors.toSet; +import static org.elasticsearch.xpack.security.support.QueryableBuiltInRolesUtils.determineRolesToDelete; +import static org.elasticsearch.xpack.security.support.QueryableBuiltInRolesUtils.determineRolesToUpsert; +import static org.elasticsearch.xpack.security.support.SecuritySystemIndices.SECURITY_MAIN_ALIAS; + +/** + * Synchronizes built-in roles to the .security index. + * The .security index is created if it does not exist. + *

+ * The synchronization is executed only on the elected master node + * after the cluster has recovered and roles need to be synced. + * The goal is to reduce the potential for conflicting operations. + * While in most cases, there should be only a single node that’s + * attempting to create/update/delete roles, it’s still possible + * that the master node changes in the middle of the syncing process. + */ +public final class QueryableBuiltInRolesSynchronizer implements ClusterStateListener { + + private static final Logger logger = LogManager.getLogger(QueryableBuiltInRolesSynchronizer.class); + + /** + * This is a temporary feature flag to allow enabling the synchronization of built-in roles to the .security index. + * Initially, it is disabled by default due to the number of tests that need to be adjusted now that .security index + * is created earlier in the cluster lifecycle. + *

+ * Once all tests are adjusted, this flag will be set to enabled by default and later removed altogether. + */ + public static final boolean QUERYABLE_BUILT_IN_ROLES_ENABLED; + static { + final var propertyValue = System.getProperty("es.queryable_built_in_roles_enabled"); + if (propertyValue == null || propertyValue.isEmpty() || "false".equals(propertyValue)) { + QUERYABLE_BUILT_IN_ROLES_ENABLED = false; + } else if ("true".equals(propertyValue)) { + QUERYABLE_BUILT_IN_ROLES_ENABLED = true; + } else { + throw new IllegalStateException( + "system property [es.queryable_built_in_roles_enabled] may only be set to [true] or [false], but was [" + + propertyValue + + "]" + ); + } + } + + public static final NodeFeature QUERYABLE_BUILT_IN_ROLES_FEATURE = new NodeFeature("security.queryable_built_in_roles"); + + /** + * Index metadata key of the digest of built-in roles indexed in the .security index. + *

+ * The value is a map of built-in role names to their digests (calculated by sha256 of the role definition). + */ + public static final String METADATA_QUERYABLE_BUILT_IN_ROLES_DIGEST_KEY = "queryable_built_in_roles_digest"; + + private static final SimpleBatchedExecutor> MARK_ROLES_AS_SYNCED_TASK_EXECUTOR = + new SimpleBatchedExecutor<>() { + @Override + public Tuple> executeTask(MarkRolesAsSyncedTask task, ClusterState clusterState) { + return task.execute(clusterState); + } + + @Override + public void taskSucceeded(MarkRolesAsSyncedTask task, Map value) { + task.success(value); + } + }; + + private final MasterServiceTaskQueue markRolesAsSyncedTaskQueue; + + private final ClusterService clusterService; + private final FeatureService featureService; + private final QueryableBuiltInRoles.Provider rolesProvider; + private final NativeRolesStore nativeRolesStore; + private final Executor executor; + private final AtomicBoolean synchronizationInProgress = new AtomicBoolean(false); + + private volatile boolean securityIndexDeleted = false; + + /** + * Constructs a new built-in roles synchronizer. + * + * @param clusterService the cluster service to register as a listener + * @param featureService the feature service to check if the cluster has the queryable built-in roles feature + * @param rolesProviderFactory the factory to create the built-in roles provider + * @param nativeRolesStore the native roles store to sync the built-in roles to + * @param reservedRolesStore the reserved roles store to fetch the built-in roles from + * @param fileRolesStore the file roles store to fetch the built-in roles from + * @param threadPool the thread pool + */ + public QueryableBuiltInRolesSynchronizer( + ClusterService clusterService, + FeatureService featureService, + QueryableBuiltInRolesProviderFactory rolesProviderFactory, + NativeRolesStore nativeRolesStore, + ReservedRolesStore reservedRolesStore, + FileRolesStore fileRolesStore, + ThreadPool threadPool + ) { + this.clusterService = clusterService; + this.featureService = featureService; + this.rolesProvider = rolesProviderFactory.createProvider(reservedRolesStore, fileRolesStore); + this.nativeRolesStore = nativeRolesStore; + this.executor = threadPool.generic(); + this.markRolesAsSyncedTaskQueue = clusterService.createTaskQueue( + "mark-built-in-roles-as-synced-task-queue", + Priority.LOW, + MARK_ROLES_AS_SYNCED_TASK_EXECUTOR + ); + this.rolesProvider.addListener(this::builtInRolesChanged); + this.clusterService.addLifecycleListener(new LifecycleListener() { + @Override + public void beforeStop() { + clusterService.removeListener(QueryableBuiltInRolesSynchronizer.this); + } + + @Override + public void beforeStart() { + clusterService.addListener(QueryableBuiltInRolesSynchronizer.this); + } + }); + } + + private void builtInRolesChanged(QueryableBuiltInRoles roles) { + logger.debug("Built-in roles changed, attempting to sync to .security index"); + final ClusterState state = clusterService.state(); + if (shouldSyncBuiltInRoles(state)) { + syncBuiltInRoles(roles); + } + } + + @Override + public void clusterChanged(ClusterChangedEvent event) { + final ClusterState state = event.state(); + if (isSecurityIndexDeleted(event)) { + this.securityIndexDeleted = true; + logger.trace("Received security index deletion event, skipping built-in roles synchronization"); + return; + } else if (isSecurityIndexCreatedOrRecovered(event)) { + this.securityIndexDeleted = false; + logger.trace("Security index has been created/recovered, attempting to sync built-in roles"); + } + if (shouldSyncBuiltInRoles(state)) { + final QueryableBuiltInRoles roles = rolesProvider.getRoles(); + syncBuiltInRoles(roles); + } + } + + private void syncBuiltInRoles(final QueryableBuiltInRoles roles) { + if (synchronizationInProgress.compareAndSet(false, true)) { + final Map indexedRolesDigests = readIndexedBuiltInRolesDigests(clusterService.state()); + if (roles.rolesDigest().equals(indexedRolesDigests)) { + logger.debug("Security index already contains the latest built-in roles indexed, skipping synchronization"); + return; + } + executor.execute(() -> doSyncBuiltinRoles(indexedRolesDigests, roles, ActionListener.wrap(v -> { + logger.info("Successfully synced [" + roles.roleDescriptors().size() + "] built-in roles to .security index"); + synchronizationInProgress.set(false); + }, e -> { + handleException(e); + synchronizationInProgress.set(false); + }))); + } + } + + private static void handleException(Exception e) { + if (e instanceof BulkRolesResponseException bulkException) { + final boolean isBulkDeleteFailure = bulkException instanceof BulkDeleteRolesResponseException; + for (final Map.Entry bulkFailure : bulkException.getFailures().entrySet()) { + final String logMessage = Strings.format( + "Failed to [%s] built-in role [%s]", + isBulkDeleteFailure ? "delete" : "create/update", + bulkFailure.getKey() + ); + if (isExpectedFailure(bulkFailure.getValue())) { + logger.info(logMessage, bulkFailure.getValue()); + } else { + logger.warn(logMessage, bulkFailure.getValue()); + } + } + } else if (isExpectedFailure(e)) { + logger.info("Failed to sync built-in roles to .security index", e); + } else { + logger.warn("Failed to sync built-in roles to .security index due to unexpected exception", e); + } + } + + /** + * Some failures are expected and should not be logged as errors. + * These exceptions are either: + * - transient (e.g. connection errors), + * - recoverable (e.g. no longer master, index reallocating or caused by concurrent operations) + * - not recoverable but expected (e.g. index closed). + * + * @param e to check + * @return {@code true} if the exception is expected and should not be logged as an error + */ + private static boolean isExpectedFailure(final Exception e) { + final Throwable cause = ExceptionsHelper.unwrapCause(e); + return ExceptionsHelper.isNodeOrShardUnavailableTypeException(cause) + || TransportActions.isShardNotAvailableException(cause) + || cause instanceof IndexClosedException + || cause instanceof IndexPrimaryShardNotAllocatedException + || cause instanceof NotMasterException + || cause instanceof ResourceAlreadyExistsException + || cause instanceof VersionConflictEngineException + || cause instanceof DocumentMissingException + || cause instanceof FailedToMarkBuiltInRolesAsSyncedException; + } + + private boolean shouldSyncBuiltInRoles(final ClusterState state) { + if (false == state.nodes().isLocalNodeElectedMaster()) { + logger.trace("Local node is not the master, skipping built-in roles synchronization"); + return false; + } + if (false == state.clusterRecovered()) { + logger.trace("Cluster state has not recovered yet, skipping built-in roles synchronization"); + return false; + } + if (nativeRolesStore.isEnabled() == false) { + logger.trace("Native roles store is not enabled, skipping built-in roles synchronization"); + return false; + } + if (state.nodes().getDataNodes().isEmpty()) { + logger.trace("No data nodes in the cluster, skipping built-in roles synchronization"); + return false; + } + if (state.nodes().isMixedVersionCluster()) { + // To keep things simple and avoid potential overwrites with an older version of built-in roles, + // we only sync built-in roles if all nodes are on the same version. + logger.trace("Not all nodes are on the same version, skipping built-in roles synchronization"); + return false; + } + if (false == featureService.clusterHasFeature(state, QUERYABLE_BUILT_IN_ROLES_FEATURE)) { + logger.trace("Not all nodes support queryable built-in roles feature, skipping built-in roles synchronization"); + return false; + } + if (securityIndexDeleted) { + logger.trace("Security index is deleted, skipping built-in roles synchronization"); + return false; + } + if (isSecurityIndexClosed(state)) { + logger.trace("Security index is closed, skipping built-in roles synchronization"); + return false; + } + return true; + } + + private void doSyncBuiltinRoles( + final Map indexedRolesDigests, + final QueryableBuiltInRoles roles, + final ActionListener listener + ) { + final Set rolesToUpsert = determineRolesToUpsert(roles, indexedRolesDigests); + final Set rolesToDelete = determineRolesToDelete(roles, indexedRolesDigests); + + assert Sets.intersection(rolesToUpsert.stream().map(RoleDescriptor::getName).collect(toSet()), rolesToDelete).isEmpty() + : "The roles to upsert and delete should not have any common roles"; + + if (rolesToUpsert.isEmpty() && rolesToDelete.isEmpty()) { + logger.debug("No changes to built-in roles to sync to .security index"); + listener.onResponse(null); + return; + } + + indexRoles(rolesToUpsert, listener.delegateFailureAndWrap((l1, indexResponse) -> { + deleteRoles(rolesToDelete, l1.delegateFailureAndWrap((l2, deleteResponse) -> { + markRolesAsSynced(indexedRolesDigests, roles.rolesDigest(), l2); + })); + })); + } + + private void deleteRoles(final Set rolesToDelete, final ActionListener listener) { + if (rolesToDelete.isEmpty()) { + listener.onResponse(null); + return; + } + nativeRolesStore.deleteRoles(rolesToDelete, WriteRequest.RefreshPolicy.IMMEDIATE, false, ActionListener.wrap(deleteResponse -> { + final Map deleteFailure = deleteResponse.getItems() + .stream() + .filter(BulkRolesResponse.Item::isFailed) + .collect(toMap(BulkRolesResponse.Item::getRoleName, BulkRolesResponse.Item::getCause)); + if (deleteFailure.isEmpty()) { + listener.onResponse(null); + } else { + listener.onFailure(new BulkDeleteRolesResponseException(deleteFailure)); + } + }, listener::onFailure)); + } + + private void indexRoles(final Collection rolesToUpsert, final ActionListener listener) { + if (rolesToUpsert.isEmpty()) { + listener.onResponse(null); + return; + } + nativeRolesStore.putRoles(WriteRequest.RefreshPolicy.IMMEDIATE, rolesToUpsert, false, ActionListener.wrap(response -> { + final Map indexFailures = response.getItems() + .stream() + .filter(BulkRolesResponse.Item::isFailed) + .collect(toMap(BulkRolesResponse.Item::getRoleName, BulkRolesResponse.Item::getCause)); + if (indexFailures.isEmpty()) { + listener.onResponse(null); + } else { + listener.onFailure(new BulkIndexRolesResponseException(indexFailures)); + } + }, listener::onFailure)); + } + + private boolean isSecurityIndexDeleted(final ClusterChangedEvent event) { + final IndexMetadata previousSecurityIndexMetadata = resolveSecurityIndexMetadata(event.previousState().metadata()); + final IndexMetadata currentSecurityIndexMetadata = resolveSecurityIndexMetadata(event.state().metadata()); + return previousSecurityIndexMetadata != null && currentSecurityIndexMetadata == null; + } + + private boolean isSecurityIndexCreatedOrRecovered(final ClusterChangedEvent event) { + final IndexMetadata previousSecurityIndexMetadata = resolveSecurityIndexMetadata(event.previousState().metadata()); + final IndexMetadata currentSecurityIndexMetadata = resolveSecurityIndexMetadata(event.state().metadata()); + return previousSecurityIndexMetadata == null && currentSecurityIndexMetadata != null; + } + + private boolean isSecurityIndexClosed(final ClusterState state) { + final IndexMetadata indexMetadata = resolveSecurityIndexMetadata(state.metadata()); + return indexMetadata != null && indexMetadata.getState() == IndexMetadata.State.CLOSE; + } + + /** + * This method marks the built-in roles as synced in the .security index + * by setting the new roles digests in the metadata of the .security index. + *

+ * The marking is done as a compare and swap operation to ensure that the roles + * are marked as synced only when new roles are indexed. The operation is idempotent + * and will succeed if the expected roles digests are equal to the digests in the + * .security index or if they are equal to the new roles digests. + */ + private void markRolesAsSynced( + final Map expectedRolesDigests, + final Map newRolesDigests, + final ActionListener listener + ) { + final IndexMetadata securityIndexMetadata = resolveSecurityIndexMetadata(clusterService.state().metadata()); + if (securityIndexMetadata == null) { + listener.onFailure(new IndexNotFoundException(SECURITY_MAIN_ALIAS)); + return; + } + final Index concreteSecurityIndex = securityIndexMetadata.getIndex(); + markRolesAsSyncedTaskQueue.submitTask( + "mark built-in roles as synced task", + new MarkRolesAsSyncedTask(listener.delegateFailureAndWrap((l, response) -> { + if (newRolesDigests.equals(response) == false) { + logger.debug( + () -> Strings.format( + "Another master node most probably indexed a newer versions of built-in roles in the meantime. " + + "Expected: [%s], Actual: [%s]", + newRolesDigests, + response + ) + ); + l.onFailure( + new FailedToMarkBuiltInRolesAsSyncedException( + "Failed to mark built-in roles as synced. The expected role digests have changed." + ) + ); + } else { + l.onResponse(null); + } + }), concreteSecurityIndex.getName(), expectedRolesDigests, newRolesDigests), + null + ); + } + + private Map readIndexedBuiltInRolesDigests(final ClusterState state) { + final IndexMetadata indexMetadata = resolveSecurityIndexMetadata(state.metadata()); + if (indexMetadata == null) { + return null; + } + return indexMetadata.getCustomData(METADATA_QUERYABLE_BUILT_IN_ROLES_DIGEST_KEY); + } + + private static IndexMetadata resolveSecurityIndexMetadata(final Metadata metadata) { + return SecurityIndexManager.resolveConcreteIndex(SECURITY_MAIN_ALIAS, metadata); + } + + static class MarkRolesAsSyncedTask implements ClusterStateTaskListener { + + private final ActionListener> listener; + private final String concreteSecurityIndexName; + private final Map expectedRoleDigests; + private final Map newRoleDigests; + + MarkRolesAsSyncedTask( + ActionListener> listener, + String concreteSecurityIndexName, + @Nullable Map expectedRoleDigests, + @Nullable Map newRoleDigests + ) { + this.listener = listener; + this.concreteSecurityIndexName = concreteSecurityIndexName; + this.expectedRoleDigests = expectedRoleDigests; + this.newRoleDigests = newRoleDigests; + } + + Tuple> execute(ClusterState state) { + IndexMetadata indexMetadata = state.metadata().index(concreteSecurityIndexName); + if (indexMetadata == null) { + throw new IndexNotFoundException(concreteSecurityIndexName); + } + Map existingRoleDigests = indexMetadata.getCustomData(METADATA_QUERYABLE_BUILT_IN_ROLES_DIGEST_KEY); + if (Objects.equals(expectedRoleDigests, existingRoleDigests)) { + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexMetadata); + if (newRoleDigests != null) { + indexMetadataBuilder.putCustom(METADATA_QUERYABLE_BUILT_IN_ROLES_DIGEST_KEY, newRoleDigests); + } else { + indexMetadataBuilder.removeCustom(METADATA_QUERYABLE_BUILT_IN_ROLES_DIGEST_KEY); + } + indexMetadataBuilder.version(indexMetadataBuilder.version() + 1); + ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(state.metadata().indices()); + builder.put(concreteSecurityIndexName, indexMetadataBuilder.build()); + return new Tuple<>( + ClusterState.builder(state).metadata(Metadata.builder(state.metadata()).indices(builder.build()).build()).build(), + newRoleDigests + ); + } else { + // returns existing value when expectation is not met + return new Tuple<>(state, existingRoleDigests); + } + } + + void success(Map value) { + listener.onResponse(value); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + } + + private static class BulkDeleteRolesResponseException extends BulkRolesResponseException { + + BulkDeleteRolesResponseException(Map failures) { + super("Failed to bulk delete built-in roles", failures); + } + + } + + private static class BulkIndexRolesResponseException extends BulkRolesResponseException { + + BulkIndexRolesResponseException(Map failures) { + super("Failed to bulk create/update built-in roles", failures); + } + + } + + private abstract static class BulkRolesResponseException extends RuntimeException { + + private final Map failures; + + BulkRolesResponseException(String message, Map failures) { + super(message); + assert failures != null && failures.isEmpty() == false; + this.failures = failures; + failures.values().forEach(this::addSuppressed); + } + + Map getFailures() { + return failures; + } + + } + + private static class FailedToMarkBuiltInRolesAsSyncedException extends RuntimeException { + + FailedToMarkBuiltInRolesAsSyncedException(String message) { + super(message); + } + + } + +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesUtils.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesUtils.java new file mode 100644 index 0000000000000..2d2eb345594ed --- /dev/null +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesUtils.java @@ -0,0 +1,101 @@ +/* + * 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.security.support; + +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.hash.MessageDigests; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.Maps; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.util.Base64; +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import static org.elasticsearch.xcontent.ToXContent.EMPTY_PARAMS; + +/** + * Utility class which provides helper method for calculating the hash of a role descriptor, + * determining the roles to upsert and the roles to delete. + */ +public final class QueryableBuiltInRolesUtils { + + /** + * Calculates the hash of the given role descriptor by serializing it by calling {@link RoleDescriptor#writeTo(StreamOutput)} method + * and then SHA256 hashing the bytes. + * + * @param roleDescriptor the role descriptor to hash + * @return the base64 encoded SHA256 hash of the role descriptor + */ + public static String calculateHash(final RoleDescriptor roleDescriptor) { + final MessageDigest hash = MessageDigests.sha256(); + try (XContentBuilder jsonBuilder = XContentFactory.jsonBuilder()) { + roleDescriptor.toXContent(jsonBuilder, EMPTY_PARAMS); + final Map flattenMap = Maps.flatten( + XContentHelper.convertToMap(BytesReference.bytes(jsonBuilder), true, XContentType.JSON).v2(), + false, + true + ); + hash.update(flattenMap.toString().getBytes(StandardCharsets.UTF_8)); + } catch (IOException e) { + throw new IllegalStateException("failed to compute digest for [" + roleDescriptor.getName() + "] role", e); + } + // HEX vs Base64 encoding is a trade-off between readability and space efficiency + // opting for Base64 here to reduce the size of the cluster state + return Base64.getEncoder().encodeToString(hash.digest()); + } + + /** + * Determines the roles to delete by comparing the indexed roles with the roles in the built-in roles. + * @return the set of roles to delete + */ + public static Set determineRolesToDelete(final QueryableBuiltInRoles roles, final Map indexedRolesDigests) { + assert roles != null; + if (indexedRolesDigests == null) { + // nothing indexed, nothing to delete + return Set.of(); + } + final Set rolesToDelete = Sets.difference(indexedRolesDigests.keySet(), roles.rolesDigest().keySet()); + return Collections.unmodifiableSet(rolesToDelete); + } + + /** + * Determines the roles to upsert by comparing the indexed roles and their digests with the current built-in roles. + * @return the set of roles to upsert (create or update) + */ + public static Set determineRolesToUpsert( + final QueryableBuiltInRoles roles, + final Map indexedRolesDigests + ) { + assert roles != null; + final Set rolesToUpsert = new HashSet<>(); + for (RoleDescriptor role : roles.roleDescriptors()) { + final String roleDigest = roles.rolesDigest().get(role.getName()); + if (indexedRolesDigests == null || indexedRolesDigests.containsKey(role.getName()) == false) { + rolesToUpsert.add(role); // a new role to create + } else if (indexedRolesDigests.get(role.getName()).equals(roleDigest) == false) { + rolesToUpsert.add(role); // an existing role that needs to be updated + } + } + return Collections.unmodifiableSet(rolesToUpsert); + } + + private QueryableBuiltInRolesUtils() { + throw new IllegalAccessError("not allowed"); + } +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableReservedRolesProvider.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableReservedRolesProvider.java new file mode 100644 index 0000000000000..710e94b7ac879 --- /dev/null +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/QueryableReservedRolesProvider.java @@ -0,0 +1,56 @@ +/* + * 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.security.support; + +import org.elasticsearch.common.util.CachedSupplier; +import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; +import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; + +import java.util.Collection; +import java.util.Collections; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +/** + * A provider of the built-in reserved roles. + *

+ * This provider fetches all reserved roles from the {@link ReservedRolesStore} and calculates their hashes lazily. + * The reserved roles are static and do not change during runtime, hence this provider will never notify any listeners. + *

+ */ +public final class QueryableReservedRolesProvider implements QueryableBuiltInRoles.Provider { + + private final Supplier reservedRolesSupplier; + + /** + * Constructs a new reserved roles provider. + * + * @param reservedRolesStore the store to fetch the reserved roles from. + * Having a store reference here is necessary to ensure that static fields are initialized. + */ + public QueryableReservedRolesProvider(ReservedRolesStore reservedRolesStore) { + this.reservedRolesSupplier = CachedSupplier.wrap(() -> { + final Collection roleDescriptors = Collections.unmodifiableCollection(ReservedRolesStore.roleDescriptors()); + return new QueryableBuiltInRoles( + roleDescriptors.stream() + .collect(Collectors.toUnmodifiableMap(RoleDescriptor::getName, QueryableBuiltInRolesUtils::calculateHash)), + roleDescriptors + ); + }); + } + + @Override + public QueryableBuiltInRoles getRoles() { + return reservedRolesSupplier.get(); + } + + @Override + public void addListener(QueryableBuiltInRoles.Listener listener) { + // no-op: reserved roles are static and do not change + } +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java index f3222a74b530c..78f7209c06e3a 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/support/SecurityIndexManager.java @@ -586,7 +586,7 @@ private static int readMappingVersion(String indexName, MappingMetadata mappingM * Resolves a concrete index name or alias to a {@link IndexMetadata} instance. Requires * that if supplied with an alias, the alias resolves to at most one concrete index. */ - private static IndexMetadata resolveConcreteIndex(final String indexOrAliasName, final Metadata metadata) { + public static IndexMetadata resolveConcreteIndex(final String indexOrAliasName, final Metadata metadata) { final IndexAbstraction indexAbstraction = metadata.getIndicesLookup().get(indexOrAliasName); if (indexAbstraction != null) { final List indices = indexAbstraction.getIndices(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesUtilsTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesUtilsTests.java new file mode 100644 index 0000000000000..5b4787f25ae7f --- /dev/null +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/QueryableBuiltInRolesUtilsTests.java @@ -0,0 +1,296 @@ +/* + * 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.security.support; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; +import org.elasticsearch.xpack.core.security.authz.RoleDescriptorTestHelper; +import org.elasticsearch.xpack.core.security.authz.permission.RemoteClusterPermissionGroup; +import org.elasticsearch.xpack.core.security.authz.permission.RemoteClusterPermissions; +import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; +import org.junit.BeforeClass; + +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + +import static org.elasticsearch.xpack.core.security.support.MetadataUtils.RESERVED_METADATA_KEY; +import static org.elasticsearch.xpack.security.support.QueryableBuiltInRolesUtils.determineRolesToDelete; +import static org.elasticsearch.xpack.security.support.QueryableBuiltInRolesUtils.determineRolesToUpsert; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; + +public class QueryableBuiltInRolesUtilsTests extends ESTestCase { + + @BeforeClass + public static void setupReservedRolesStore() { + new ReservedRolesStore(); // initialize the store + } + + public void testCalculateHash() { + assertThat( + QueryableBuiltInRolesUtils.calculateHash(ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR), + equalTo("bWEFdFo4WX229wdhdecfiz5QHMYEssh3ex8hizRgg+Q=") + ); + } + + public void testEmptyOrNullRolesToUpsertOrDelete() { + // test empty roles and index digests + final QueryableBuiltInRoles emptyRoles = new QueryableBuiltInRoles(Map.of(), Set.of()); + assertThat(determineRolesToDelete(emptyRoles, Map.of()), is(empty())); + assertThat(determineRolesToUpsert(emptyRoles, Map.of()), is(empty())); + + // test empty roles and null indexed digests + assertThat(determineRolesToDelete(emptyRoles, null), is(empty())); + assertThat(determineRolesToUpsert(emptyRoles, null), is(empty())); + } + + public void testNoRolesToUpsertOrDelete() { + { + QueryableBuiltInRoles currentBuiltInRoles = buildQueryableBuiltInRoles( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor") + ) + ); + + // no roles to delete or upsert since the built-in roles are the same as the indexed roles + assertThat(determineRolesToDelete(currentBuiltInRoles, currentBuiltInRoles.rolesDigest()), is(empty())); + assertThat(determineRolesToUpsert(currentBuiltInRoles, currentBuiltInRoles.rolesDigest()), is(empty())); + } + { + QueryableBuiltInRoles currentBuiltInRoles = buildQueryableBuiltInRoles( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor"), + supermanRole("monitor", "read") + ) + ); + + Map digests = buildDigests( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor"), + supermanRole("monitor", "read") + ) + ); + + // no roles to delete or upsert since the built-in roles are the same as the indexed roles + assertThat(determineRolesToDelete(currentBuiltInRoles, digests), is(empty())); + assertThat(determineRolesToUpsert(currentBuiltInRoles, digests), is(empty())); + } + { + final RoleDescriptor randomRole = RoleDescriptorTestHelper.randomRoleDescriptor(); + final QueryableBuiltInRoles currentBuiltInRoles = buildQueryableBuiltInRoles(Set.of(randomRole)); + final Map digests = buildDigests( + Set.of( + new RoleDescriptor( + randomRole.getName(), + randomRole.getClusterPrivileges(), + randomRole.getIndicesPrivileges(), + randomRole.getApplicationPrivileges(), + randomRole.getConditionalClusterPrivileges(), + randomRole.getRunAs(), + randomRole.getMetadata(), + randomRole.getTransientMetadata(), + randomRole.getRemoteIndicesPrivileges(), + randomRole.getRemoteClusterPermissions(), + randomRole.getRestriction(), + randomRole.getDescription() + ) + ) + ); + + assertThat(determineRolesToDelete(currentBuiltInRoles, digests), is(empty())); + assertThat(determineRolesToUpsert(currentBuiltInRoles, digests), is(empty())); + } + } + + public void testRolesToDeleteOnly() { + Map indexedDigests = buildDigests( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor"), + supermanRole("monitor", "read", "view_index_metadata", "read_cross_cluster") + ) + ); + + QueryableBuiltInRoles currentBuiltInRoles = buildQueryableBuiltInRoles( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor") + ) + ); + + // superman is the only role that needs to be deleted since it is not in a current built-in role + assertThat(determineRolesToDelete(currentBuiltInRoles, indexedDigests), containsInAnyOrder("superman")); + assertThat(determineRolesToUpsert(currentBuiltInRoles, indexedDigests), is(empty())); + + // passing empty built-in roles should result in all indexed roles needing to be deleted + QueryableBuiltInRoles emptyBuiltInRoles = new QueryableBuiltInRoles(Map.of(), Set.of()); + assertThat( + determineRolesToDelete(emptyBuiltInRoles, indexedDigests), + containsInAnyOrder("superman", "viewer", "editor", "superuser") + ); + assertThat(determineRolesToUpsert(emptyBuiltInRoles, indexedDigests), is(empty())); + } + + public void testRolesToUpdateOnly() { + Map indexedDigests = buildDigests( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor"), + supermanRole("monitor", "read", "write") + ) + ); + + RoleDescriptor updatedSupermanRole = supermanRole("monitor", "read", "view_index_metadata", "read_cross_cluster"); + QueryableBuiltInRoles currentBuiltInRoles = buildQueryableBuiltInRoles( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor"), + updatedSupermanRole + ) + ); + + // superman is the only role that needs to be updated since its definition has changed + assertThat(determineRolesToDelete(currentBuiltInRoles, indexedDigests), is(empty())); + assertThat(determineRolesToUpsert(currentBuiltInRoles, indexedDigests), containsInAnyOrder(updatedSupermanRole)); + assertThat(currentBuiltInRoles.rolesDigest().get("superman"), is(not(equalTo(indexedDigests.get("superman"))))); + } + + public void testRolesToCreateOnly() { + Map indexedDigests = buildDigests( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor") + ) + ); + + RoleDescriptor newSupermanRole = supermanRole("monitor", "read", "view_index_metadata", "read_cross_cluster"); + QueryableBuiltInRoles currentBuiltInRoles = buildQueryableBuiltInRoles( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor"), + newSupermanRole + ) + ); + + // superman is the only role that needs to be created since it is not in the indexed roles + assertThat(determineRolesToDelete(currentBuiltInRoles, indexedDigests), is(empty())); + assertThat(determineRolesToUpsert(currentBuiltInRoles, indexedDigests), containsInAnyOrder(newSupermanRole)); + + // passing empty indexed roles should result in all roles needing to be created + assertThat(determineRolesToDelete(currentBuiltInRoles, Map.of()), is(empty())); + assertThat( + determineRolesToUpsert(currentBuiltInRoles, Map.of()), + containsInAnyOrder(currentBuiltInRoles.roleDescriptors().toArray(new RoleDescriptor[0])) + ); + } + + public void testRolesToUpsertAndDelete() { + Map indexedDigests = buildDigests( + Set.of( + ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, + ReservedRolesStore.roleDescriptor("viewer"), + ReservedRolesStore.roleDescriptor("editor") + ) + ); + + RoleDescriptor newSupermanRole = supermanRole("monitor"); + QueryableBuiltInRoles currentBuiltInRoles = buildQueryableBuiltInRoles( + Set.of(ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR, newSupermanRole) + ); + + // superman is the only role that needs to be updated since its definition has changed + assertThat(determineRolesToDelete(currentBuiltInRoles, indexedDigests), containsInAnyOrder("viewer", "editor")); + assertThat(determineRolesToUpsert(currentBuiltInRoles, indexedDigests), containsInAnyOrder(newSupermanRole)); + } + + private static RoleDescriptor supermanRole(String... indicesPrivileges) { + return new RoleDescriptor( + "superman", + new String[] { "all" }, + new RoleDescriptor.IndicesPrivileges[] { + RoleDescriptor.IndicesPrivileges.builder().indices("*").privileges("all").allowRestrictedIndices(false).build(), + RoleDescriptor.IndicesPrivileges.builder() + .indices("*") + .privileges(indicesPrivileges) + .allowRestrictedIndices(true) + .build() }, + new RoleDescriptor.ApplicationResourcePrivileges[] { + RoleDescriptor.ApplicationResourcePrivileges.builder().application("*").privileges("*").resources("*").build() }, + null, + new String[] { "*" }, + randomlyOrderedSupermanMetadata(), + Collections.emptyMap(), + new RoleDescriptor.RemoteIndicesPrivileges[] { + new RoleDescriptor.RemoteIndicesPrivileges( + RoleDescriptor.IndicesPrivileges.builder().indices("*").privileges("all").allowRestrictedIndices(false).build(), + "*" + ), + new RoleDescriptor.RemoteIndicesPrivileges( + RoleDescriptor.IndicesPrivileges.builder() + .indices("*") + .privileges(indicesPrivileges) + .allowRestrictedIndices(true) + .build(), + "*" + ) }, + new RemoteClusterPermissions().addGroup( + new RemoteClusterPermissionGroup( + RemoteClusterPermissions.getSupportedRemoteClusterPermissions().toArray(new String[0]), + new String[] { "*" } + ) + ), + null, + "Grants full access to cluster management and data indices." + ); + } + + private static Map randomlyOrderedSupermanMetadata() { + final LinkedHashMap metadata = new LinkedHashMap<>(); + if (randomBoolean()) { + metadata.put("foo", "bar"); + metadata.put("baz", "qux"); + metadata.put(RESERVED_METADATA_KEY, true); + } else { + metadata.put(RESERVED_METADATA_KEY, true); + metadata.put("foo", "bar"); + metadata.put("baz", "qux"); + } + return metadata; + } + + private static QueryableBuiltInRoles buildQueryableBuiltInRoles(Set roles) { + final Map digests = buildDigests(roles); + return new QueryableBuiltInRoles(digests, roles); + } + + private static Map buildDigests(Set roles) { + final Map digests = new HashMap<>(); + for (RoleDescriptor role : roles) { + digests.put(role.getName(), QueryableBuiltInRolesUtils.calculateHash(role)); + } + return digests; + } +} diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/QueryableReservedRolesProviderTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/QueryableReservedRolesProviderTests.java new file mode 100644 index 0000000000000..7beb078795b29 --- /dev/null +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/support/QueryableReservedRolesProviderTests.java @@ -0,0 +1,31 @@ +/* + * 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.security.support; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.security.authz.RoleDescriptor; +import org.elasticsearch.xpack.core.security.authz.store.ReservedRolesStore; + +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.equalTo; + +public class QueryableReservedRolesProviderTests extends ESTestCase { + + public void testReservedRoleProvider() { + QueryableReservedRolesProvider provider = new QueryableReservedRolesProvider(new ReservedRolesStore()); + assertNotNull(provider.getRoles()); + assertThat(provider.getRoles(), equalTo(provider.getRoles())); + assertThat(provider.getRoles().rolesDigest().size(), equalTo(ReservedRolesStore.roleDescriptors().size())); + assertThat( + provider.getRoles().rolesDigest().keySet(), + equalTo(ReservedRolesStore.roleDescriptors().stream().map(RoleDescriptor::getName).collect(Collectors.toSet())) + ); + } + +} From cf1f2cbc34e97b8d280ee74e824ac1106c24cef0 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Mon, 16 Dec 2024 14:08:11 -0500 Subject: [PATCH 15/44] Unmute ml/sparse_vector_search/Test sparse_vector search with query vector and pruning config (#118788) * Unmute ml/sparse_vector_search/Test sparse_vector search with query vector and pruning config * Revert "Unmute ml/sparse_vector_search/Test sparse_vector search with query vector and pruning config" This reverts commit 48e76936e622072da7c214842613d22b9830dc45. * Unmute test, this time without formatting the entire file --- muted-tests.yml | 3 --- 1 file changed, 3 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 2689f02cc92cd..be3805d887bbe 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -40,9 +40,6 @@ tests: - class: org.elasticsearch.packaging.test.WindowsServiceTests method: test33JavaChanged issue: https://github.com/elastic/elasticsearch/issues/113177 -- class: org.elasticsearch.smoketest.MlWithSecurityIT - method: test {yaml=ml/sparse_vector_search/Test sparse_vector search with query vector and pruning config} - issue: https://github.com/elastic/elasticsearch/issues/108997 - class: org.elasticsearch.packaging.test.WindowsServiceTests method: test80JavaOptsInEnvVar issue: https://github.com/elastic/elasticsearch/issues/113219 From 8e988689438f9579dc23427efc85d320e8289ce3 Mon Sep 17 00:00:00 2001 From: Craig Taverner Date: Mon, 16 Dec 2024 21:26:45 +0100 Subject: [PATCH 16/44] Support multi-index LOOKUP JOIN and various bug fixes (#118429) While working on supporting multi-index LOOKUP JOIN; various bugs were fixed: * Previously we just used '*' for lookup-join indices, because the fieldnames were sometimes not being correctly determined. The problem was with KEEP referencing fields from the right that had previously been defined on the left as aliases, including using the ROW command. We normally don't want to ask for aliases, but if they could be shadowed by a lookup join, we need to keep them. * With both single and multi-index LOOKUP JOIN we need to mark each index as potentially wildcard fields, if the KEEP commands occur before the LOOKUP JOIN. --- .../esql/qa/mixed/MixedClusterEsqlSpecIT.java | 4 +- .../xpack/esql/ccq/MultiClusterSpecIT.java | 8 +- .../rest/RequestIndexFilteringTestCase.java | 11 + .../src/main/resources/lookup-join.csv-spec | 566 +++++++++++++++--- .../src/main/resources/message_types.csv | 2 + .../xpack/esql/action/EsqlCapabilities.java | 2 +- .../xpack/esql/analysis/Analyzer.java | 12 +- .../xpack/esql/analysis/AnalyzerContext.java | 12 +- .../xpack/esql/session/EsqlSession.java | 269 ++++----- .../elasticsearch/xpack/esql/CsvTests.java | 2 +- .../esql/analysis/AnalyzerTestUtils.java | 4 +- .../xpack/esql/analysis/AnalyzerTests.java | 6 +- .../xpack/esql/analysis/VerifierTests.java | 2 +- .../optimizer/LogicalPlanOptimizerTests.java | 37 +- .../optimizer/PhysicalPlanOptimizerTests.java | 4 +- .../session/IndexResolverFieldNamesTests.java | 226 ++++++- 16 files changed, 902 insertions(+), 265 deletions(-) diff --git a/x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/MixedClusterEsqlSpecIT.java b/x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/MixedClusterEsqlSpecIT.java index 1120a69cc5166..5efe7ffc800a2 100644 --- a/x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/MixedClusterEsqlSpecIT.java +++ b/x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/MixedClusterEsqlSpecIT.java @@ -21,7 +21,7 @@ import java.util.List; import static org.elasticsearch.xpack.esql.CsvTestUtils.isEnabled; -import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V5; +import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V6; import static org.elasticsearch.xpack.esql.qa.rest.EsqlSpecTestCase.Mode.ASYNC; public class MixedClusterEsqlSpecIT extends EsqlSpecTestCase { @@ -96,7 +96,7 @@ protected boolean supportsInferenceTestService() { @Override protected boolean supportsIndexModeLookup() throws IOException { - return hasCapabilities(List.of(JOIN_LOOKUP_V5.capabilityName())); + return hasCapabilities(List.of(JOIN_LOOKUP_V6.capabilityName())); } @Override diff --git a/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java b/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java index 5c7f981c93a97..dd75776973c3d 100644 --- a/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java +++ b/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java @@ -48,7 +48,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.classpathResources; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.INLINESTATS; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.INLINESTATS_V2; -import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V5; +import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V6; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_PLANNING_V1; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.METADATA_FIELDS_REMOTE_TEST; import static org.elasticsearch.xpack.esql.qa.rest.EsqlSpecTestCase.Mode.SYNC; @@ -124,7 +124,7 @@ protected void shouldSkipTest(String testName) throws IOException { assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(INLINESTATS.capabilityName())); assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(INLINESTATS_V2.capabilityName())); assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_PLANNING_V1.capabilityName())); - assumeFalse("LOOKUP JOIN not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_LOOKUP_V5.capabilityName())); + assumeFalse("LOOKUP JOIN not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_LOOKUP_V6.capabilityName())); } private TestFeatureService remoteFeaturesService() throws IOException { @@ -283,8 +283,8 @@ protected boolean supportsInferenceTestService() { @Override protected boolean supportsIndexModeLookup() throws IOException { - // CCS does not yet support JOIN_LOOKUP_V5 and clusters falsely report they have this capability - // return hasCapabilities(List.of(JOIN_LOOKUP_V5.capabilityName())); + // CCS does not yet support JOIN_LOOKUP_V6 and clusters falsely report they have this capability + // return hasCapabilities(List.of(JOIN_LOOKUP_V6.capabilityName())); return false; } } diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java index 406997b66dbf0..2aae4c94c33fe 100644 --- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java @@ -14,6 +14,7 @@ import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.esql.AssertWarnings; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; import org.junit.After; import org.junit.Assert; @@ -219,6 +220,16 @@ public void testIndicesDontExist() throws IOException { assertEquals(404, e.getResponse().getStatusLine().getStatusCode()); assertThat(e.getMessage(), containsString("index_not_found_exception")); assertThat(e.getMessage(), containsString("no such index [foo]")); + + if (EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()) { + e = expectThrows( + ResponseException.class, + () -> runEsql(timestampFilter("gte", "2020-01-01").query("FROM test1 | LOOKUP JOIN foo ON id1")) + ); + assertEquals(400, e.getResponse().getStatusLine().getStatusCode()); + assertThat(e.getMessage(), containsString("verification_exception")); + assertThat(e.getMessage(), containsString("Unknown index [foo]")); + } } private static RestEsqlTestCase.RequestObjectBuilder timestampFilter(String op, String date) throws IOException { diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec index 7fed4f377096f..8b8d24b1bb156 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec @@ -3,8 +3,12 @@ // Reuses the sample dataset and commands from enrich.csv-spec // +############################################### +# Tests with languages_lookup index +############################################### + basicOnTheDataNode -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | EVAL language_code = languages @@ -21,7 +25,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; basicRow -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW language_code = 1 | LOOKUP JOIN languages_lookup ON language_code @@ -32,7 +36,7 @@ language_code:integer | language_name:keyword ; basicOnTheCoordinator -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | SORT emp_no @@ -49,7 +53,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; subsequentEvalOnTheDataNode -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | EVAL language_code = languages @@ -67,7 +71,7 @@ emp_no:integer | language_code:integer | language_name:keyword | language_code_x ; subsequentEvalOnTheCoordinator -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | SORT emp_no @@ -85,7 +89,7 @@ emp_no:integer | language_code:integer | language_name:keyword | language_code_x ; sortEvalBeforeLookup -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | SORT emp_no @@ -102,7 +106,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; nonUniqueLeftKeyOnTheDataNode -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | WHERE emp_no <= 10030 @@ -126,7 +130,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; nonUniqueRightKeyOnTheDataNode -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | EVAL language_code = emp_no % 10 @@ -146,7 +150,7 @@ emp_no:integer | language_code:integer | language_name:keyword | country:k ; nonUniqueRightKeyOnTheCoordinator -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | SORT emp_no @@ -166,7 +170,7 @@ emp_no:integer | language_code:integer | language_name:keyword | country:k ; nonUniqueRightKeyFromRow -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW language_code = 2 | LOOKUP JOIN languages_lookup_non_unique_key ON language_code @@ -178,8 +182,73 @@ language_code:integer | language_name:keyword | country:keyword 2 | [German, German, German] | [Austria, Germany, Switzerland] ; +############################################### +# Filtering tests with languages_lookup index +############################################### + +lookupWithFilterOnLeftSideField +required_capability: join_lookup_v6 + +FROM employees +| EVAL language_code = languages +| LOOKUP JOIN languages_lookup ON language_code +| SORT emp_no +| KEEP emp_no, language_code, language_name +| WHERE emp_no >= 10091 AND emp_no < 10094 +; + +emp_no:integer | language_code:integer | language_name:keyword +10091 | 3 | Spanish +10092 | 1 | English +10093 | 3 | Spanish +; + +lookupMessageWithFilterOnRightSideField-Ignore +required_capability: join_lookup_v6 + +FROM sample_data +| LOOKUP JOIN message_types_lookup ON message +| WHERE type == "Error" +| KEEP @timestamp, client_ip, event_duration, message, type +| SORT @timestamp DESC +; + +@timestamp:date | client_ip:ip | event_duration:long | message:keyword | type:keyword +2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Connection error | Error +2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Connection error | Error +2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Connection error | Error +; + +lookupWithFieldAndRightSideAfterStats +required_capability: join_lookup_v6 + +FROM sample_data +| LOOKUP JOIN message_types_lookup ON message +| STATS count = count(message) BY type +| WHERE type == "Error" +; + +count:long | type:keyword +3 | Error +; + +lookupWithFieldOnJoinKey-Ignore +required_capability: join_lookup_v6 + +FROM employees +| EVAL language_code = languages +| LOOKUP JOIN languages_lookup ON language_code +| WHERE language_code > 1 AND language_name IS NOT NULL +| KEEP emp_no, language_code, language_name +; + +emp_no:integer | language_code:integer | language_name:keyword +10001 | 2 | French +10003 | 4 | German +; + nullJoinKeyOnTheDataNode -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | WHERE emp_no < 10004 @@ -197,7 +266,7 @@ emp_no:integer | language_code:integer | language_name:keyword mvJoinKeyOnTheDataNode -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM employees | WHERE 10003 < emp_no AND emp_no < 10008 @@ -215,7 +284,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; mvJoinKeyFromRow -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW language_code = [4, 5, 6, 7] | LOOKUP JOIN languages_lookup_non_unique_key ON language_code @@ -228,7 +297,7 @@ language_code:integer | language_name:keyword | country:keyword ; mvJoinKeyFromRowExpanded -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW language_code = [4, 5, 6, 7, 8] | MV_EXPAND language_code @@ -245,10 +314,26 @@ language_code:integer | language_name:keyword | country:keyword 8 | Mv-Lang2 | Mv-Land2 ; +############################################### +# Tests with clientips_lookup index +############################################### + lookupIPFromRow -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", right = "right" +| LOOKUP JOIN clientips_lookup ON client_ip +; + +left:keyword | client_ip:keyword | right:keyword | env:keyword +left | 172.21.0.5 | right | Development +; + +lookupIPFromKeepRow +required_capability: join_lookup_v6 ROW left = "left", client_ip = "172.21.0.5", right = "right" +| KEEP left, client_ip, right | LOOKUP JOIN clientips_lookup ON client_ip ; @@ -257,7 +342,7 @@ left | 172.21.0.5 | right | Development ; lookupIPFromRowWithShadowing -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW left = "left", client_ip = "172.21.0.5", env = "env", right = "right" | LOOKUP JOIN clientips_lookup ON client_ip @@ -268,7 +353,7 @@ left | 172.21.0.5 | right | Development ; lookupIPFromRowWithShadowingKeep -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW left = "left", client_ip = "172.21.0.5", env = "env", right = "right" | EVAL client_ip = client_ip::keyword @@ -281,7 +366,7 @@ left | 172.21.0.5 | right | Development ; lookupIPFromRowWithShadowingKeepReordered -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW left = "left", client_ip = "172.21.0.5", env = "env", right = "right" | EVAL client_ip = client_ip::keyword @@ -294,7 +379,7 @@ right | Development | 172.21.0.5 ; lookupIPFromIndex -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -313,7 +398,7 @@ ignoreOrder:true ; lookupIPFromIndexKeep -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -332,8 +417,30 @@ ignoreOrder:true 2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 | QA ; +lookupIPFromIndexKeepKeep +required_capability: join_lookup_v6 + +FROM sample_data +| KEEP client_ip, event_duration, @timestamp, message +| RENAME @timestamp AS timestamp, message AS msg +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| KEEP timestamp, client_ip, event_duration, msg, env +; +ignoreOrder:true + +timestamp:date | client_ip:keyword | event_duration:long | msg:keyword | env:keyword +2023-10-23T13:55:01.543Z | 172.21.3.15 | 1756467 | Connected to 10.1.0.1 | Production +2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Connection error | Production +2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Connection error | Production +2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Connection error | Production +2023-10-23T13:33:34.937Z | 172.21.0.5 | 1232382 | Disconnected | Development +2023-10-23T12:27:28.948Z | 172.21.2.113 | 2764889 | Connected to 10.1.0.2 | QA +2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 | QA +; + lookupIPFromIndexStats -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -349,7 +456,7 @@ count:long | env:keyword ; lookupIPFromIndexStatsKeep -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -365,10 +472,43 @@ count:long | env:keyword 1 | Development ; +statsAndLookupIPFromIndex +required_capability: join_lookup_v6 + +FROM sample_data +| EVAL client_ip = client_ip::keyword +| STATS count = count(client_ip) BY client_ip +| LOOKUP JOIN clientips_lookup ON client_ip +| SORT count DESC, client_ip ASC, env ASC +; + +count:long | client_ip:keyword | env:keyword +4 | 172.21.3.15 | Production +1 | 172.21.0.5 | Development +1 | 172.21.2.113 | QA +1 | 172.21.2.162 | QA +; + +############################################### +# Tests with message_types_lookup index +############################################### + lookupMessageFromRow -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 + +ROW left = "left", message = "Connected to 10.1.0.1", right = "right" +| LOOKUP JOIN message_types_lookup ON message +; + +left:keyword | message:keyword | right:keyword | type:keyword +left | Connected to 10.1.0.1 | right | Success +; + +lookupMessageFromKeepRow +required_capability: join_lookup_v6 ROW left = "left", message = "Connected to 10.1.0.1", right = "right" +| KEEP left, message, right | LOOKUP JOIN message_types_lookup ON message ; @@ -377,7 +517,7 @@ left | Connected to 10.1.0.1 | right | Success ; lookupMessageFromRowWithShadowing -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW left = "left", message = "Connected to 10.1.0.1", type = "unknown", right = "right" | LOOKUP JOIN message_types_lookup ON message @@ -388,7 +528,7 @@ left | Connected to 10.1.0.1 | right | Success ; lookupMessageFromRowWithShadowingKeep -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 ROW left = "left", message = "Connected to 10.1.0.1", type = "unknown", right = "right" | LOOKUP JOIN message_types_lookup ON message @@ -400,7 +540,7 @@ left | Connected to 10.1.0.1 | right | Success ; lookupMessageFromIndex -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -418,7 +558,7 @@ ignoreOrder:true ; lookupMessageFromIndexKeep -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -436,8 +576,28 @@ ignoreOrder:true 2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 | Success ; +lookupMessageFromIndexKeepKeep +required_capability: join_lookup_v6 + +FROM sample_data +| KEEP client_ip, event_duration, @timestamp, message +| LOOKUP JOIN message_types_lookup ON message +| KEEP @timestamp, client_ip, event_duration, message, type +; +ignoreOrder:true + +@timestamp:date | client_ip:ip | event_duration:long | message:keyword | type:keyword +2023-10-23T13:55:01.543Z | 172.21.3.15 | 1756467 | Connected to 10.1.0.1 | Success +2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Connection error | Error +2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Connection error | Error +2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Connection error | Error +2023-10-23T13:33:34.937Z | 172.21.0.5 | 1232382 | Disconnected | Disconnected +2023-10-23T12:27:28.948Z | 172.21.2.113 | 2764889 | Connected to 10.1.0.2 | Success +2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 | Success +; + lookupMessageFromIndexKeepReordered -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -456,7 +616,7 @@ Success | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 ; lookupMessageFromIndexStats -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -471,7 +631,7 @@ count:long | type:keyword ; lookupMessageFromIndexStatsKeep -required_capability: join_lookup_v5 +required_capability: join_lookup_v6 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -486,67 +646,333 @@ count:long | type:keyword 1 | Disconnected ; -// -// Filtering tests -// +statsAndLookupMessageFromIndex +required_capability: join_lookup_v6 -lookupWithFilterOnLeftSideField -required_capability: join_lookup_v5 +FROM sample_data +| STATS count = count(message) BY message +| LOOKUP JOIN message_types_lookup ON message +| KEEP count, type, message +| SORT count DESC, message ASC +; -FROM employees -| EVAL language_code = languages -| LOOKUP JOIN languages_lookup ON language_code -| SORT emp_no -| KEEP emp_no, language_code, language_name -| WHERE emp_no >= 10091 AND emp_no < 10094 +count:long | type:keyword | message:keyword +3 | Error | Connection error +1 | Success | Connected to 10.1.0.1 +1 | Success | Connected to 10.1.0.2 +1 | Success | Connected to 10.1.0.3 +1 | Disconnected | Disconnected ; -emp_no:integer | language_code:integer | language_name:keyword -10091 | 3 | Spanish -10092 | 1 | English -10093 | 3 | Spanish +lookupMessageFromIndexTwice +required_capability: join_lookup_v6 + +FROM sample_data +| LOOKUP JOIN message_types_lookup ON message +| RENAME message AS message1, type AS type1 +| EVAL message = client_ip::keyword +| LOOKUP JOIN message_types_lookup ON message +| RENAME message AS message2, type AS type2 ; +ignoreOrder:true -lookupMessageWithFilterOnRightSideField-Ignore -required_capability: join_lookup_v5 +@timestamp:date | client_ip:ip | event_duration:long | message1:keyword | type1:keyword | message2:keyword | type2:keyword +2023-10-23T13:55:01.543Z | 172.21.3.15 | 1756467 | Connected to 10.1.0.1 | Success | 172.21.3.15 | null +2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Connection error | Error | 172.21.3.15 | null +2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Connection error | Error | 172.21.3.15 | null +2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Connection error | Error | 172.21.3.15 | null +2023-10-23T13:33:34.937Z | 172.21.0.5 | 1232382 | Disconnected | Disconnected | 172.21.0.5 | null +2023-10-23T12:27:28.948Z | 172.21.2.113 | 2764889 | Connected to 10.1.0.2 | Success | 172.21.2.113 | null +2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 | Success | 172.21.2.162 | null +; + +lookupMessageFromIndexTwiceKeep +required_capability: join_lookup_v6 FROM sample_data | LOOKUP JOIN message_types_lookup ON message -| WHERE type == "Error" -| KEEP @timestamp, client_ip, event_duration, message, type -| SORT @timestamp DESC +| RENAME message AS message1, type AS type1 +| EVAL message = client_ip::keyword +| LOOKUP JOIN message_types_lookup ON message +| RENAME message AS message2, type AS type2 +| KEEP @timestamp, client_ip, event_duration, message1, type1, message2, type2 ; +ignoreOrder:true -@timestamp:date | client_ip:ip | event_duration:long | message:keyword | type:keyword -2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Connection error | Error -2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Connection error | Error -2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Connection error | Error +@timestamp:date | client_ip:ip | event_duration:long | message1:keyword | type1:keyword | message2:keyword | type2:keyword +2023-10-23T13:55:01.543Z | 172.21.3.15 | 1756467 | Connected to 10.1.0.1 | Success | 172.21.3.15 | null +2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Connection error | Error | 172.21.3.15 | null +2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Connection error | Error | 172.21.3.15 | null +2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Connection error | Error | 172.21.3.15 | null +2023-10-23T13:33:34.937Z | 172.21.0.5 | 1232382 | Disconnected | Disconnected | 172.21.0.5 | null +2023-10-23T12:27:28.948Z | 172.21.2.113 | 2764889 | Connected to 10.1.0.2 | Success | 172.21.2.113 | null +2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 | Success | 172.21.2.162 | null ; -lookupWithFieldAndRightSideAfterStats -required_capability: join_lookup_v5 +############################################### +# Tests with clientips_lookup and message_types_lookup indexes +############################################### + +lookupIPAndMessageFromRow +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", right = "right" +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +; + +left:keyword | client_ip:keyword | message:keyword | right:keyword | env:keyword | type:keyword +left | 172.21.0.5 | Connected to 10.1.0.1 | right | Development | Success +; + +lookupIPAndMessageFromRowKeepBefore +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", right = "right" +| KEEP left, client_ip, message, right +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +; + +left:keyword | client_ip:keyword | message:keyword | right:keyword | env:keyword | type:keyword +left | 172.21.0.5 | Connected to 10.1.0.1 | right | Development | Success +; + +lookupIPAndMessageFromRowKeepBetween +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", right = "right" +| LOOKUP JOIN clientips_lookup ON client_ip +| KEEP left, client_ip, message, right, env +| LOOKUP JOIN message_types_lookup ON message +; + +left:keyword | client_ip:keyword | message:keyword | right:keyword | env:keyword | type:keyword +left | 172.21.0.5 | Connected to 10.1.0.1 | right | Development | Success +; + +lookupIPAndMessageFromRowKeepAfter +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", right = "right" +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +| KEEP left, client_ip, message, right, env, type +; + +left:keyword | client_ip:keyword | message:keyword | right:keyword | env:keyword | type:keyword +left | 172.21.0.5 | Connected to 10.1.0.1 | right | Development | Success +; + +lookupIPAndMessageFromRowWithShadowing +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", type = "type", right = "right" +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +; + +left:keyword | client_ip:keyword | message:keyword | right:keyword | env:keyword | type:keyword +left | 172.21.0.5 | Connected to 10.1.0.1 | right | Development | Success +; + +lookupIPAndMessageFromRowWithShadowingKeep +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", right = "right" +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +| KEEP left, client_ip, message, right, env, type +; + +left:keyword | client_ip:keyword | message:keyword | right:keyword | env:keyword | type:keyword +left | 172.21.0.5 | Connected to 10.1.0.1 | right | Development | Success +; + +lookupIPAndMessageFromRowWithShadowingKeepKeep +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", right = "right" +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| KEEP left, client_ip, message, right, env +| LOOKUP JOIN message_types_lookup ON message +| KEEP left, client_ip, message, right, env, type +; + +left:keyword | client_ip:keyword | message:keyword | right:keyword | env:keyword | type:keyword +left | 172.21.0.5 | Connected to 10.1.0.1 | right | Development | Success +; + +lookupIPAndMessageFromRowWithShadowingKeepKeepKeep +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", right = "right" +| EVAL client_ip = client_ip::keyword +| KEEP left, client_ip, message, right, env +| LOOKUP JOIN clientips_lookup ON client_ip +| KEEP left, client_ip, message, right, env +| LOOKUP JOIN message_types_lookup ON message +| KEEP left, client_ip, message, right, env, type +; + +left:keyword | client_ip:keyword | message:keyword | right:keyword | env:keyword | type:keyword +left | 172.21.0.5 | Connected to 10.1.0.1 | right | Development | Success +; + +lookupIPAndMessageFromRowWithShadowingKeepReordered +required_capability: join_lookup_v6 + +ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", right = "right" +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +| KEEP right, env, type, client_ip +; + +right:keyword | env:keyword | type:keyword | client_ip:keyword +right | Development | Success | 172.21.0.5 +; + +lookupIPAndMessageFromIndex +required_capability: join_lookup_v6 FROM sample_data +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip | LOOKUP JOIN message_types_lookup ON message -| STATS count = count(message) BY type -| WHERE type == "Error" ; +ignoreOrder:true -count:long | type:keyword -3 | Error +@timestamp:date | event_duration:long | message:keyword | client_ip:keyword | env:keyword | type:keyword +2023-10-23T13:55:01.543Z | 1756467 | Connected to 10.1.0.1 | 172.21.3.15 | Production | Success +2023-10-23T13:53:55.832Z | 5033755 | Connection error | 172.21.3.15 | Production | Error +2023-10-23T13:52:55.015Z | 8268153 | Connection error | 172.21.3.15 | Production | Error +2023-10-23T13:51:54.732Z | 725448 | Connection error | 172.21.3.15 | Production | Error +2023-10-23T13:33:34.937Z | 1232382 | Disconnected | 172.21.0.5 | Development | Disconnected +2023-10-23T12:27:28.948Z | 2764889 | Connected to 10.1.0.2 | 172.21.2.113 | QA | Success +2023-10-23T12:15:03.360Z | 3450233 | Connected to 10.1.0.3 | 172.21.2.162 | QA | Success ; -lookupWithFieldOnJoinKey-Ignore -required_capability: join_lookup_v5 +lookupIPAndMessageFromIndexKeep +required_capability: join_lookup_v6 -FROM employees -| EVAL language_code = languages -| LOOKUP JOIN languages_lookup ON language_code -| WHERE language_code > 1 AND language_name IS NOT NULL -| KEEP emp_no, language_code, language_name +FROM sample_data +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +| KEEP @timestamp, client_ip, event_duration, message, env, type ; +ignoreOrder:true -emp_no:integer | language_code:integer | language_name:keyword -10001 | 2 | French -10003 | 4 | German +@timestamp:date | client_ip:keyword | event_duration:long | message:keyword | env:keyword | type:keyword +2023-10-23T13:55:01.543Z | 172.21.3.15 | 1756467 | Connected to 10.1.0.1 | Production | Success +2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Connection error | Production | Error +2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Connection error | Production | Error +2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Connection error | Production | Error +2023-10-23T13:33:34.937Z | 172.21.0.5 | 1232382 | Disconnected | Development | Disconnected +2023-10-23T12:27:28.948Z | 172.21.2.113 | 2764889 | Connected to 10.1.0.2 | QA | Success +2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 | QA | Success +; + +lookupIPAndMessageFromIndexStats +required_capability: join_lookup_v6 + +FROM sample_data +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +| STATS count = count(*) BY env, type +| SORT count DESC, env ASC, type ASC +; + +count:long | env:keyword | type:keyword +3 | Production | Error +2 | QA | Success +1 | Development | Disconnected +1 | Production | Success +; + +lookupIPAndMessageFromIndexStatsKeep +required_capability: join_lookup_v6 + +FROM sample_data +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +| KEEP client_ip, env, type +| STATS count = count(*) BY env, type +| SORT count DESC, env ASC, type ASC +; + +count:long | env:keyword | type:keyword +3 | Production | Error +2 | QA | Success +1 | Development | Disconnected +1 | Production | Success +; + +statsAndLookupIPAndMessageFromIndex +required_capability: join_lookup_v6 + +FROM sample_data +| EVAL client_ip = client_ip::keyword +| STATS count = count(*) BY client_ip, message +| LOOKUP JOIN clientips_lookup ON client_ip +| LOOKUP JOIN message_types_lookup ON message +| SORT count DESC, client_ip ASC, message ASC +; + +count:long | client_ip:keyword | message:keyword | env:keyword | type:keyword +3 | 172.21.3.15 | Connection error | Production | Error +1 | 172.21.0.5 | Disconnected | Development | Disconnected +1 | 172.21.2.113 | Connected to 10.1.0.2 | QA | Success +1 | 172.21.2.162 | Connected to 10.1.0.3 | QA | Success +1 | 172.21.3.15 | Connected to 10.1.0.1 | Production | Success +; + +lookupIPAndMessageFromIndexChainedEvalKeep +required_capability: join_lookup_v6 + +FROM sample_data +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| EVAL message = CONCAT(env, " environment") +| LOOKUP JOIN message_types_lookup ON message +| KEEP @timestamp, client_ip, event_duration, message, type +; +ignoreOrder:true + +@timestamp:date | client_ip:keyword | event_duration:long | message:keyword | type:keyword +2023-10-23T13:55:01.543Z | 172.21.3.15 | 1756467 | Production environment | Production +2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Production environment | Production +2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Production environment | Production +2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Production environment | Production +2023-10-23T13:33:34.937Z | 172.21.0.5 | 1232382 | Development environment | Development +2023-10-23T12:27:28.948Z | 172.21.2.113 | 2764889 | QA environment | null +2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | QA environment | null +; + +lookupIPAndMessageFromIndexChainedRenameKeep +required_capability: join_lookup_v6 + +FROM sample_data +| EVAL client_ip = client_ip::keyword +| LOOKUP JOIN clientips_lookup ON client_ip +| RENAME env AS message +| LOOKUP JOIN message_types_lookup ON message +| KEEP @timestamp, client_ip, event_duration, message, type ; +ignoreOrder:true + +@timestamp:date | client_ip:keyword | event_duration:long | message:keyword | type:keyword +2023-10-23T13:55:01.543Z | 172.21.3.15 | 1756467 | Production | null +2023-10-23T13:53:55.832Z | 172.21.3.15 | 5033755 | Production | null +2023-10-23T13:52:55.015Z | 172.21.3.15 | 8268153 | Production | null +2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Production | null +2023-10-23T13:33:34.937Z | 172.21.0.5 | 1232382 | Development | null +2023-10-23T12:27:28.948Z | 172.21.2.113 | 2764889 | QA | null +2023-10-23T12:15:03.360Z | 172.21.2.162 | 3450233 | QA | null +; + diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/message_types.csv b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/message_types.csv index 8e00485771445..bb4b58046b843 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/message_types.csv +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/message_types.csv @@ -4,3 +4,5 @@ Disconnected,Disconnected Connected to 10.1.0.1,Success Connected to 10.1.0.2,Success Connected to 10.1.0.3,Success +Production environment,Production +Development environment,Development diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index e9a0f89e4f448..235d0dcbe4164 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -547,7 +547,7 @@ public enum Cap { /** * LOOKUP JOIN */ - JOIN_LOOKUP_V5(Build.current().isSnapshot()), + JOIN_LOOKUP_V6(Build.current().isSnapshot()), /** * Fix for https://github.com/elastic/elasticsearch/issues/117054 diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java index cf91c7df9a034..d59745f03f608 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java @@ -198,12 +198,16 @@ private static class ResolveTable extends ParameterizedAnalyzerRule lookupResolution, EnrichResolution enrichResolution ) { // Currently for tests only, since most do not test lookups @@ -26,12 +28,6 @@ public AnalyzerContext( IndexResolution indexResolution, EnrichResolution enrichResolution ) { - this( - configuration, - functionRegistry, - indexResolution, - IndexResolution.invalid("AnalyzerContext constructed without any lookup join resolution"), - enrichResolution - ); + this(configuration, functionRegistry, indexResolution, Map.of(), enrichResolution); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java index 83480f6651abf..c0290fa2b1d73 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java @@ -13,7 +13,6 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.compute.data.Block; @@ -77,10 +76,12 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; @@ -282,10 +283,10 @@ public void analyzedPlan( return; } - TriFunction analyzeAction = (indices, lookupIndices, policies) -> { + Function analyzeAction = (l) -> { planningMetrics.gatherPreAnalysisMetrics(parsed); Analyzer analyzer = new Analyzer( - new AnalyzerContext(configuration, functionRegistry, indices, lookupIndices, policies), + new AnalyzerContext(configuration, functionRegistry, l.indices, l.lookupIndices, l.enrichResolution), verifier ); LogicalPlan plan = analyzer.analyze(parsed); @@ -301,110 +302,77 @@ public void analyzedPlan( EsqlSessionCCSUtils.checkForCcsLicense(indices, indicesExpressionGrouper, verifier.licenseState()); - // TODO: make a separate call for lookup indices final Set targetClusters = enrichPolicyResolver.groupIndicesPerCluster( indices.stream().flatMap(t -> Arrays.stream(Strings.commaDelimitedListToStringArray(t.id().index()))).toArray(String[]::new) ).keySet(); - SubscribableListener.newForked(l -> enrichPolicyResolver.resolvePolicies(targetClusters, unresolvedPolicies, l)) - .andThen((l, enrichResolution) -> { - // we need the match_fields names from enrich policies and THEN, with an updated list of fields, we call field_caps API - var enrichMatchFields = enrichResolution.resolvedEnrichPolicies() - .stream() - .map(ResolvedEnrichPolicy::matchField) - .collect(Collectors.toSet()); - // get the field names from the parsed plan combined with the ENRICH match fields from the ENRICH policy - var fieldNames = fieldNames(parsed, enrichMatchFields); - ListenerResult listenerResult = new ListenerResult(null, null, enrichResolution, fieldNames); - - // first resolve the lookup indices, then the main indices - preAnalyzeLookupIndices(preAnalysis.lookupIndices, listenerResult, l); - }) - .andThen((l, listenerResult) -> { - // resolve the main indices - preAnalyzeIndices(preAnalysis.indices, executionInfo, listenerResult, requestFilter, l); - }) - .andThen((l, listenerResult) -> { - // TODO in follow-PR (for skip_unavailable handling of missing concrete indexes) add some tests for - // invalid index resolution to updateExecutionInfo - if (listenerResult.indices.isValid()) { - // CCS indices and skip_unavailable cluster values can stop the analysis right here - if (analyzeCCSIndices(executionInfo, targetClusters, unresolvedPolicies, listenerResult, logicalPlanListener, l)) - return; - } - // whatever tuple we have here (from CCS-special handling or from the original pre-analysis), pass it on to the next step - l.onResponse(listenerResult); - }) - .andThen((l, listenerResult) -> { - // first attempt (maybe the only one) at analyzing the plan - analyzeAndMaybeRetry(analyzeAction, requestFilter, listenerResult, logicalPlanListener, l); - }) - .andThen((l, listenerResult) -> { - assert requestFilter != null : "The second pre-analysis shouldn't take place when there is no index filter in the request"; - - // "reset" execution information for all ccs or non-ccs (local) clusters, since we are performing the indices - // resolving one more time (the first attempt failed and the query had a filter) - for (String clusterAlias : executionInfo.clusterAliases()) { - executionInfo.swapCluster(clusterAlias, (k, v) -> null); - } - - // here the requestFilter is set to null, performing the pre-analysis after the first step failed - preAnalyzeIndices(preAnalysis.indices, executionInfo, listenerResult, null, l); - }) - .andThen((l, listenerResult) -> { - assert requestFilter != null : "The second analysis shouldn't take place when there is no index filter in the request"; - LOGGER.debug("Analyzing the plan (second attempt, without filter)"); - LogicalPlan plan; - try { - plan = analyzeAction.apply(listenerResult.indices, listenerResult.lookupIndices, listenerResult.enrichResolution); - } catch (Exception e) { - l.onFailure(e); - return; - } - LOGGER.debug("Analyzed plan (second attempt, without filter):\n{}", plan); - l.onResponse(plan); - }) - .addListener(logicalPlanListener); - } + var listener = SubscribableListener.newForked( + l -> enrichPolicyResolver.resolvePolicies(targetClusters, unresolvedPolicies, l) + ).andThen((l, enrichResolution) -> resolveFieldNames(parsed, enrichResolution, l)); + // first resolve the lookup indices, then the main indices + for (TableInfo lookupIndex : preAnalysis.lookupIndices) { + listener = listener.andThen((l, preAnalysisResult) -> { preAnalyzeLookupIndex(lookupIndex, preAnalysisResult, l); }); + } + listener.andThen((l, result) -> { + // resolve the main indices + preAnalyzeIndices(preAnalysis.indices, executionInfo, result, requestFilter, l); + }).andThen((l, result) -> { + // TODO in follow-PR (for skip_unavailable handling of missing concrete indexes) add some tests for + // invalid index resolution to updateExecutionInfo + if (result.indices.isValid()) { + // CCS indices and skip_unavailable cluster values can stop the analysis right here + if (analyzeCCSIndices(executionInfo, targetClusters, unresolvedPolicies, result, logicalPlanListener, l)) return; + } + // whatever tuple we have here (from CCS-special handling or from the original pre-analysis), pass it on to the next step + l.onResponse(result); + }).andThen((l, result) -> { + // first attempt (maybe the only one) at analyzing the plan + analyzeAndMaybeRetry(analyzeAction, requestFilter, result, logicalPlanListener, l); + }).andThen((l, result) -> { + assert requestFilter != null : "The second pre-analysis shouldn't take place when there is no index filter in the request"; + + // "reset" execution information for all ccs or non-ccs (local) clusters, since we are performing the indices + // resolving one more time (the first attempt failed and the query had a filter) + for (String clusterAlias : executionInfo.clusterAliases()) { + executionInfo.swapCluster(clusterAlias, (k, v) -> null); + } - private void preAnalyzeLookupIndices(List indices, ListenerResult listenerResult, ActionListener listener) { - if (indices.size() > 1) { - // Note: JOINs on more than one index are not yet supported - listener.onFailure(new MappingException("More than one LOOKUP JOIN is not supported")); - } else if (indices.size() == 1) { - TableInfo tableInfo = indices.get(0); - TableIdentifier table = tableInfo.id(); - // call the EsqlResolveFieldsAction (field-caps) to resolve indices and get field types - indexResolver.resolveAsMergedMapping( - table.index(), - Set.of("*"), // TODO: for LOOKUP JOIN, this currently declares all lookup index fields relevant and might fetch too many. - null, - listener.map(indexResolution -> listenerResult.withLookupIndexResolution(indexResolution)) - ); - // TODO: Verify that the resolved index actually has indexMode: "lookup" - } else { + // here the requestFilter is set to null, performing the pre-analysis after the first step failed + preAnalyzeIndices(preAnalysis.indices, executionInfo, result, null, l); + }).andThen((l, result) -> { + assert requestFilter != null : "The second analysis shouldn't take place when there is no index filter in the request"; + LOGGER.debug("Analyzing the plan (second attempt, without filter)"); + LogicalPlan plan; try { - // No lookup indices specified - listener.onResponse( - new ListenerResult( - listenerResult.indices, - IndexResolution.invalid("[none specified]"), - listenerResult.enrichResolution, - listenerResult.fieldNames - ) - ); - } catch (Exception ex) { - listener.onFailure(ex); + plan = analyzeAction.apply(result); + } catch (Exception e) { + l.onFailure(e); + return; } - } + LOGGER.debug("Analyzed plan (second attempt, without filter):\n{}", plan); + l.onResponse(plan); + }).addListener(logicalPlanListener); + } + + private void preAnalyzeLookupIndex(TableInfo tableInfo, PreAnalysisResult result, ActionListener listener) { + TableIdentifier table = tableInfo.id(); + Set fieldNames = result.wildcardJoinIndices().contains(table.index()) ? IndexResolver.ALL_FIELDS : result.fieldNames; + // call the EsqlResolveFieldsAction (field-caps) to resolve indices and get field types + indexResolver.resolveAsMergedMapping( + table.index(), + fieldNames, + null, + listener.map(indexResolution -> result.addLookupIndexResolution(table.index(), indexResolution)) + ); + // TODO: Verify that the resolved index actually has indexMode: "lookup" } private void preAnalyzeIndices( List indices, EsqlExecutionInfo executionInfo, - ListenerResult listenerResult, + PreAnalysisResult result, QueryBuilder requestFilter, - ActionListener listener + ActionListener listener ) { // TODO we plan to support joins in the future when possible, but for now we'll just fail early if we see one if (indices.size() > 1) { @@ -412,7 +380,7 @@ private void preAnalyzeIndices( listener.onFailure(new MappingException("Queries with multiple indices are not supported")); } else if (indices.size() == 1) { // known to be unavailable from the enrich policy API call - Map unavailableClusters = listenerResult.enrichResolution.getUnavailableClusters(); + Map unavailableClusters = result.enrichResolution.getUnavailableClusters(); TableInfo tableInfo = indices.get(0); TableIdentifier table = tableInfo.id(); @@ -445,34 +413,20 @@ private void preAnalyzeIndices( String indexExpressionToResolve = EsqlSessionCCSUtils.createIndexExpressionFromAvailableClusters(executionInfo); if (indexExpressionToResolve.isEmpty()) { // if this was a pure remote CCS request (no local indices) and all remotes are offline, return an empty IndexResolution - listener.onResponse( - new ListenerResult( - IndexResolution.valid(new EsIndex(table.index(), Map.of(), Map.of())), - listenerResult.lookupIndices, - listenerResult.enrichResolution, - listenerResult.fieldNames - ) - ); + listener.onResponse(result.withIndexResolution(IndexResolution.valid(new EsIndex(table.index(), Map.of(), Map.of())))); } else { // call the EsqlResolveFieldsAction (field-caps) to resolve indices and get field types indexResolver.resolveAsMergedMapping( indexExpressionToResolve, - listenerResult.fieldNames, + result.fieldNames, requestFilter, - listener.map(indexResolution -> listenerResult.withIndexResolution(indexResolution)) + listener.map(indexResolution -> result.withIndexResolution(indexResolution)) ); } } else { try { // occurs when dealing with local relations (row a = 1) - listener.onResponse( - new ListenerResult( - IndexResolution.invalid("[none specified]"), - listenerResult.lookupIndices, - listenerResult.enrichResolution, - listenerResult.fieldNames - ) - ); + listener.onResponse(result.withIndexResolution(IndexResolution.invalid("[none specified]"))); } catch (Exception ex) { listener.onFailure(ex); } @@ -483,11 +437,11 @@ private boolean analyzeCCSIndices( EsqlExecutionInfo executionInfo, Set targetClusters, Set unresolvedPolicies, - ListenerResult listenerResult, + PreAnalysisResult result, ActionListener logicalPlanListener, - ActionListener l + ActionListener l ) { - IndexResolution indexResolution = listenerResult.indices; + IndexResolution indexResolution = result.indices; EsqlSessionCCSUtils.updateExecutionInfoWithClustersWithNoMatchingIndices(executionInfo, indexResolution); EsqlSessionCCSUtils.updateExecutionInfoWithUnavailableClusters(executionInfo, indexResolution.unavailableClusters()); if (executionInfo.isCrossClusterSearch() && executionInfo.getClusterStateCount(EsqlExecutionInfo.Cluster.Status.RUNNING) == 0) { @@ -509,7 +463,7 @@ private boolean analyzeCCSIndices( enrichPolicyResolver.resolvePolicies( newClusters, unresolvedPolicies, - l.map(enrichResolution -> listenerResult.withEnrichResolution(enrichResolution)) + l.map(enrichResolution -> result.withEnrichResolution(enrichResolution)) ); return true; } @@ -517,11 +471,11 @@ private boolean analyzeCCSIndices( } private static void analyzeAndMaybeRetry( - TriFunction analyzeAction, + Function analyzeAction, QueryBuilder requestFilter, - ListenerResult listenerResult, + PreAnalysisResult result, ActionListener logicalPlanListener, - ActionListener l + ActionListener l ) { LogicalPlan plan = null; var filterPresentMessage = requestFilter == null ? "without" : "with"; @@ -529,7 +483,7 @@ private static void analyzeAndMaybeRetry( LOGGER.debug("Analyzing the plan ({} attempt, {} filter)", attemptMessage, filterPresentMessage); try { - plan = analyzeAction.apply(listenerResult.indices, listenerResult.lookupIndices, listenerResult.enrichResolution); + plan = analyzeAction.apply(result); } catch (Exception e) { if (e instanceof VerificationException ve) { LOGGER.debug( @@ -544,7 +498,7 @@ private static void analyzeAndMaybeRetry( } else { // interested only in a VerificationException, but this time we are taking out the index filter // to try and make the index resolution work without any index filtering. In the next step... to be continued - l.onResponse(listenerResult); + l.onResponse(result); } } else { // if the query failed with any other type of exception, then just pass the exception back to the user @@ -557,10 +511,24 @@ private static void analyzeAndMaybeRetry( logicalPlanListener.onResponse(plan); } - static Set fieldNames(LogicalPlan parsed, Set enrichPolicyMatchFields) { + private static void resolveFieldNames(LogicalPlan parsed, EnrichResolution enrichResolution, ActionListener l) { + try { + // we need the match_fields names from enrich policies and THEN, with an updated list of fields, we call field_caps API + var enrichMatchFields = enrichResolution.resolvedEnrichPolicies() + .stream() + .map(ResolvedEnrichPolicy::matchField) + .collect(Collectors.toSet()); + // get the field names from the parsed plan combined with the ENRICH match fields from the ENRICH policy + l.onResponse(fieldNames(parsed, enrichMatchFields, new PreAnalysisResult(enrichResolution))); + } catch (Exception ex) { + l.onFailure(ex); + } + } + + static PreAnalysisResult fieldNames(LogicalPlan parsed, Set enrichPolicyMatchFields, PreAnalysisResult result) { if (false == parsed.anyMatch(plan -> plan instanceof Aggregate || plan instanceof Project)) { // no explicit columns selection, for example "from employees" - return IndexResolver.ALL_FIELDS; + return result.withFieldNames(IndexResolver.ALL_FIELDS); } Holder projectAll = new Holder<>(false); @@ -571,7 +539,7 @@ static Set fieldNames(LogicalPlan parsed, Set enrichPolicyMatchF projectAll.set(true); }); if (projectAll.get()) { - return IndexResolver.ALL_FIELDS; + return result.withFieldNames(IndexResolver.ALL_FIELDS); } AttributeSet references = new AttributeSet(); @@ -579,6 +547,7 @@ static Set fieldNames(LogicalPlan parsed, Set enrichPolicyMatchF // ie "from test | eval lang = languages + 1 | keep *l" should consider both "languages" and "*l" as valid fields to ask for AttributeSet keepCommandReferences = new AttributeSet(); AttributeSet keepJoinReferences = new AttributeSet(); + Set wildcardJoinIndices = new java.util.HashSet<>(); parsed.forEachDown(p -> {// go over each plan top-down if (p instanceof RegexExtract re) { // for Grok and Dissect @@ -596,10 +565,16 @@ static Set fieldNames(LogicalPlan parsed, Set enrichPolicyMatchF enrichRefs.removeIf(attr -> attr instanceof EmptyAttribute); references.addAll(enrichRefs); } else if (p instanceof LookupJoin join) { - keepJoinReferences.addAll(join.config().matchFields()); // TODO: why is this empty if (join.config().type() instanceof JoinTypes.UsingJoinType usingJoinType) { keepJoinReferences.addAll(usingJoinType.columns()); } + if (keepCommandReferences.isEmpty()) { + // No KEEP commands after the JOIN, so we need to mark this index for "*" field resolution + wildcardJoinIndices.add(((UnresolvedRelation) join.right()).table().index()); + } else { + // Keep commands can reference the join columns with names that shadow aliases, so we block their removal + keepJoinReferences.addAll(keepCommandReferences); + } } else { references.addAll(p.references()); if (p instanceof UnresolvedRelation ur && ur.indexMode() == IndexMode.TIME_SERIES) { @@ -634,6 +609,10 @@ static Set fieldNames(LogicalPlan parsed, Set enrichPolicyMatchF }); // Add JOIN ON column references afterward to avoid Alias removal references.addAll(keepJoinReferences); + // If any JOIN commands need wildcard field-caps calls, persist the index names + if (wildcardJoinIndices.isEmpty() == false) { + result = result.withWildcardJoinIndices(wildcardJoinIndices); + } // remove valid metadata attributes because they will be filtered out by the IndexResolver anyway // otherwise, in some edge cases, we will fail to ask for "*" (all fields) instead @@ -642,12 +621,12 @@ static Set fieldNames(LogicalPlan parsed, Set enrichPolicyMatchF if (fieldNames.isEmpty() && enrichPolicyMatchFields.isEmpty()) { // there cannot be an empty list of fields, we'll ask the simplest and lightest one instead: _index - return IndexResolver.INDEX_METADATA_FIELD; + return result.withFieldNames(IndexResolver.INDEX_METADATA_FIELD); } else { fieldNames.addAll(subfields(fieldNames)); fieldNames.addAll(enrichPolicyMatchFields); fieldNames.addAll(subfields(enrichPolicyMatchFields)); - return fieldNames; + return result.withFieldNames(fieldNames); } } @@ -706,22 +685,36 @@ public PhysicalPlan optimizedPhysicalPlan(LogicalPlan optimizedPlan) { return plan; } - private record ListenerResult( + record PreAnalysisResult( IndexResolution indices, - IndexResolution lookupIndices, + Map lookupIndices, EnrichResolution enrichResolution, - Set fieldNames + Set fieldNames, + Set wildcardJoinIndices ) { - ListenerResult withEnrichResolution(EnrichResolution newEnrichResolution) { - return new ListenerResult(indices(), lookupIndices(), newEnrichResolution, fieldNames()); + PreAnalysisResult(EnrichResolution newEnrichResolution) { + this(null, new HashMap<>(), newEnrichResolution, Set.of(), Set.of()); } - ListenerResult withIndexResolution(IndexResolution newIndexResolution) { - return new ListenerResult(newIndexResolution, lookupIndices(), enrichResolution(), fieldNames()); + PreAnalysisResult withEnrichResolution(EnrichResolution newEnrichResolution) { + return new PreAnalysisResult(indices(), lookupIndices(), newEnrichResolution, fieldNames(), wildcardJoinIndices()); } - ListenerResult withLookupIndexResolution(IndexResolution newIndexResolution) { - return new ListenerResult(indices(), newIndexResolution, enrichResolution(), fieldNames()); + PreAnalysisResult withIndexResolution(IndexResolution newIndexResolution) { + return new PreAnalysisResult(newIndexResolution, lookupIndices(), enrichResolution(), fieldNames(), wildcardJoinIndices()); } - }; + + PreAnalysisResult addLookupIndexResolution(String index, IndexResolution newIndexResolution) { + lookupIndices.put(index, newIndexResolution); + return this; + } + + PreAnalysisResult withFieldNames(Set newFields) { + return new PreAnalysisResult(indices(), lookupIndices(), enrichResolution(), newFields, wildcardJoinIndices()); + } + + public PreAnalysisResult withWildcardJoinIndices(Set wildcardJoinIndices) { + return new PreAnalysisResult(indices(), lookupIndices(), enrichResolution(), fieldNames(), wildcardJoinIndices); + } + } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java index c11ef8615eb72..f553c15ef69fa 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java @@ -263,7 +263,7 @@ public final void test() throws Throwable { ); assumeFalse( "lookup join disabled for csv tests", - testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.JOIN_LOOKUP_V5.capabilityName()) + testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.JOIN_LOOKUP_V6.capabilityName()) ); assumeFalse( "can't use TERM function in csv tests", diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java index 5e79e40b7e938..85dd36ba0aaa5 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTestUtils.java @@ -123,8 +123,8 @@ public static IndexResolution expandedDefaultIndexResolution() { return loadMapping("mapping-default.json", "test"); } - public static IndexResolution defaultLookupResolution() { - return loadMapping("mapping-languages.json", "languages_lookup", IndexMode.LOOKUP); + public static Map defaultLookupResolution() { + return Map.of("languages_lookup", loadMapping("mapping-languages.json", "languages_lookup", IndexMode.LOOKUP)); } public static EnrichResolution defaultEnrichResolution() { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java index cfff245b19244..4e02119b31744 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java @@ -2139,7 +2139,7 @@ public void testLookupMatchTypeWrong() { } public void testLookupJoinUnknownIndex() { - assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); String errorMessage = "Unknown index [foobar]"; IndexResolution missingLookupIndex = IndexResolution.invalid(errorMessage); @@ -2149,7 +2149,7 @@ public void testLookupJoinUnknownIndex() { EsqlTestUtils.TEST_CFG, new EsqlFunctionRegistry(), analyzerDefaultMapping(), - missingLookupIndex, + Map.of("foobar", missingLookupIndex), defaultEnrichResolution() ), TEST_VERIFIER @@ -2168,7 +2168,7 @@ public void testLookupJoinUnknownIndex() { } public void testLookupJoinUnknownField() { - assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); String query = "FROM test | LOOKUP JOIN languages_lookup ON last_name"; String errorMessage = "1:45: Unknown column [last_name] in right side of join"; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 4b916106165fb..58180aafedc0b 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1964,7 +1964,7 @@ public void testSortByAggregate() { } public void testLookupJoinDataTypeMismatch() { - assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); query("FROM test | EVAL language_code = languages | LOOKUP JOIN languages_lookup ON language_code"); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index 1d10ebab267ce..c4d7b30115c2d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -219,11 +219,6 @@ public static void init() { enrichResolution = new EnrichResolution(); AnalyzerTestUtils.loadEnrichPolicyResolution(enrichResolution, "languages_idx", "id", "languages_idx", "mapping-languages.json"); - var lookupMapping = loadMapping("mapping-languages.json"); - IndexResolution lookupResolution = IndexResolution.valid( - new EsIndex("language_code", lookupMapping, Map.of("language_code", IndexMode.LOOKUP)) - ); - // Most tests used data from the test index, so we load it here, and use it in the plan() function. mapping = loadMapping("mapping-basic.json"); EsIndex test = new EsIndex("test", mapping, Map.of("test", IndexMode.STANDARD)); @@ -4911,7 +4906,7 @@ public void testPlanSanityCheck() throws Exception { } public void testPlanSanityCheckWithBinaryPlans() throws Exception { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); var plan = optimizedPlan(""" FROM test @@ -5913,15 +5908,15 @@ public void testLookupStats() { * | \_Limit[1000[INTEGER]] * | \_Filter[languages{f}#10 > 1[INTEGER]] * | \_EsRelation[test][_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, ge..] - * \_EsRelation[language_code][LOOKUP][language_code{f}#18, language_name{f}#19] + * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19] */ public void testLookupJoinPushDownFilterOnJoinKeyWithRename() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); String query = """ FROM test | RENAME languages AS language_code - | LOOKUP JOIN language_code ON language_code + | LOOKUP JOIN languages_lookup ON language_code | WHERE language_code > 1 """; var plan = optimizedPlan(query); @@ -5956,15 +5951,15 @@ public void testLookupJoinPushDownFilterOnJoinKeyWithRename() { * | \_Limit[1000[INTEGER]] * | \_Filter[emp_no{f}#7 > 1[INTEGER]] * | \_EsRelation[test][_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, ge..] - * \_EsRelation[language_code][LOOKUP][language_code{f}#18, language_name{f}#19] + * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19] */ public void testLookupJoinPushDownFilterOnLeftSideField() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); String query = """ FROM test | RENAME languages AS language_code - | LOOKUP JOIN language_code ON language_code + | LOOKUP JOIN languages_lookup ON language_code | WHERE emp_no > 1 """; @@ -6000,15 +5995,15 @@ public void testLookupJoinPushDownFilterOnLeftSideField() { * |_EsqlProject[[_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, gender{f}#9, hire_date{f}#14, job{f}#15, job.raw{f}#16, lang * uages{f}#10 AS language_code, last_name{f}#11, long_noidx{f}#17, salary{f}#12]] * | \_EsRelation[test][_meta_field{f}#13, emp_no{f}#7, first_name{f}#8, ge..] - * \_EsRelation[language_code][LOOKUP][language_code{f}#18, language_name{f}#19] + * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19] */ public void testLookupJoinPushDownDisabledForLookupField() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); String query = """ FROM test | RENAME languages AS language_code - | LOOKUP JOIN language_code ON language_code + | LOOKUP JOIN languages_lookup ON language_code | WHERE language_name == "English" """; @@ -6045,15 +6040,15 @@ public void testLookupJoinPushDownDisabledForLookupField() { * guages{f}#11 AS language_code, last_name{f}#12, long_noidx{f}#18, salary{f}#13]] * | \_Filter[emp_no{f}#8 > 1[INTEGER]] * | \_EsRelation[test][_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, ge..] - * \_EsRelation[language_code][LOOKUP][language_code{f}#19, language_name{f}#20] + * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#19, language_name{f}#20] */ public void testLookupJoinPushDownSeparatedForConjunctionBetweenLeftAndRightField() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); String query = """ FROM test | RENAME languages AS language_code - | LOOKUP JOIN language_code ON language_code + | LOOKUP JOIN languages_lookup ON language_code | WHERE language_name == "English" AND emp_no > 1 """; @@ -6098,15 +6093,15 @@ public void testLookupJoinPushDownSeparatedForConjunctionBetweenLeftAndRightFiel * |_EsqlProject[[_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, gender{f}#10, hire_date{f}#15, job{f}#16, job.raw{f}#17, lan * guages{f}#11 AS language_code, last_name{f}#12, long_noidx{f}#18, salary{f}#13]] * | \_EsRelation[test][_meta_field{f}#14, emp_no{f}#8, first_name{f}#9, ge..] - * \_EsRelation[language_code][LOOKUP][language_code{f}#19, language_name{f}#20] + * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#19, language_name{f}#20] */ public void testLookupJoinPushDownDisabledForDisjunctionBetweenLeftAndRightField() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); String query = """ FROM test | RENAME languages AS language_code - | LOOKUP JOIN language_code ON language_code + | LOOKUP JOIN languages_lookup ON language_code | WHERE language_name == "English" OR emp_no > 1 """; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index d0c7a1cd61010..9f6ef89008a24 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -285,7 +285,7 @@ TestDataSource makeTestDataSource( String indexName, String mappingFileName, EsqlFunctionRegistry functionRegistry, - IndexResolution lookupResolution, + Map lookupResolution, EnrichResolution enrichResolution, SearchStats stats ) { @@ -2331,7 +2331,7 @@ public void testVerifierOnMissingReferences() { } public void testVerifierOnMissingReferencesWithBinaryPlans() throws Exception { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V5.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); // Do not assert serialization: // This will have a LookupJoinExec, which is not serializable because it doesn't leave the coordinator. diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java index 0fe89b24dfc6a..e4271a0a6ddd5 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java @@ -1316,25 +1316,25 @@ public void testCountStar() { } public void testEnrichOnDefaultFieldWithKeep() { - Set fieldNames = EsqlSession.fieldNames(parser.createStatement(""" + Set fieldNames = fieldNames(""" from employees | enrich languages_policy - | keep emp_no"""), Set.of("language_name")); + | keep emp_no""", Set.of("language_name")); assertThat(fieldNames, equalTo(Set.of("emp_no", "emp_no.*", "language_name", "language_name.*"))); } public void testDissectOverwriteName() { - Set fieldNames = EsqlSession.fieldNames(parser.createStatement(""" + Set fieldNames = fieldNames(""" from employees | dissect first_name "%{first_name} %{more}" - | keep emp_no, first_name, more"""), Set.of()); + | keep emp_no, first_name, more""", Set.of()); assertThat(fieldNames, equalTo(Set.of("emp_no", "emp_no.*", "first_name", "first_name.*"))); } public void testEnrichOnDefaultField() { - Set fieldNames = EsqlSession.fieldNames(parser.createStatement(""" + Set fieldNames = fieldNames(""" from employees - | enrich languages_policy"""), Set.of("language_name")); + | enrich languages_policy""", Set.of("language_name")); assertThat(fieldNames, equalTo(ALL_FIELDS)); } @@ -1345,7 +1345,7 @@ public void testMetrics() { assertThat(e.getMessage(), containsString("line 1:1: mismatched input 'METRICS' expecting {")); return; } - Set fieldNames = EsqlSession.fieldNames(parser.createStatement(query), Set.of()); + Set fieldNames = fieldNames(query, Set.of()); assertThat( fieldNames, equalTo( @@ -1363,8 +1363,218 @@ public void testMetrics() { ); } + public void testLookupJoin() { + assertFieldNames( + "FROM employees | KEEP languages | RENAME languages AS language_code | LOOKUP JOIN languages_lookup ON language_code", + Set.of("languages", "languages.*", "language_code", "language_code.*"), + Set.of("languages_lookup") // Since we have KEEP before the LOOKUP JOIN we need to wildcard the lookup index + ); + } + + public void testLookupJoinKeep() { + assertFieldNames( + """ + FROM employees + | KEEP languages + | RENAME languages AS language_code + | LOOKUP JOIN languages_lookup ON language_code + | KEEP languages, language_code, language_name""", + Set.of("languages", "languages.*", "language_code", "language_code.*", "language_name", "language_name.*"), + Set.of() // Since we have KEEP after the LOOKUP, we can use the global field names instead of wildcarding the lookup index + ); + } + + public void testLookupJoinKeepWildcard() { + assertFieldNames( + """ + FROM employees + | KEEP languages + | RENAME languages AS language_code + | LOOKUP JOIN languages_lookup ON language_code + | KEEP language*""", + Set.of("language*", "languages", "languages.*", "language_code", "language_code.*"), + Set.of() // Since we have KEEP after the LOOKUP, we can use the global field names instead of wildcarding the lookup index + ); + } + + public void testMultiLookupJoin() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | LOOKUP JOIN clientips_lookup ON client_ip + | LOOKUP JOIN message_types_lookup ON message""", + Set.of("*"), // With no KEEP we should keep all fields + Set.of() // since global field names are wildcarded, we don't need to wildcard any indices + ); + } + + public void testMultiLookupJoinKeepBefore() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | KEEP @timestamp, client_ip, event_duration, message + | LOOKUP JOIN clientips_lookup ON client_ip + | LOOKUP JOIN message_types_lookup ON message""", + Set.of("@timestamp", "@timestamp.*", "client_ip", "client_ip.*", "event_duration", "event_duration.*", "message", "message.*"), + Set.of("clientips_lookup", "message_types_lookup") // Since the KEEP is before both JOINS we need to wildcard both indices + ); + } + + public void testMultiLookupJoinKeepBetween() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | LOOKUP JOIN clientips_lookup ON client_ip + | KEEP @timestamp, client_ip, event_duration, message, env + | LOOKUP JOIN message_types_lookup ON message""", + Set.of( + "@timestamp", + "@timestamp.*", + "client_ip", + "client_ip.*", + "event_duration", + "event_duration.*", + "message", + "message.*", + "env", + "env.*" + ), + Set.of("message_types_lookup") // Since the KEEP is before the second JOIN, we need to wildcard the second index + ); + } + + public void testMultiLookupJoinKeepAfter() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | LOOKUP JOIN clientips_lookup ON client_ip + | LOOKUP JOIN message_types_lookup ON message + | KEEP @timestamp, client_ip, event_duration, message, env, type""", + Set.of( + "@timestamp", + "@timestamp.*", + "client_ip", + "client_ip.*", + "event_duration", + "event_duration.*", + "message", + "message.*", + "env", + "env.*", + "type", + "type.*" + ), + Set.of() // Since the KEEP is after both JOINs, we can use the global field names + ); + } + + public void testMultiLookupJoinKeepAfterWildcard() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | LOOKUP JOIN clientips_lookup ON client_ip + | LOOKUP JOIN message_types_lookup ON message + | KEEP *env*, *type*""", + Set.of("*env*", "*type*", "client_ip", "client_ip.*", "message", "message.*"), + Set.of() // Since the KEEP is after both JOINs, we can use the global field names + ); + } + + public void testMultiLookupJoinSameIndex() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | LOOKUP JOIN clientips_lookup ON client_ip + | EVAL client_ip = message + | LOOKUP JOIN clientips_lookup ON client_ip""", + Set.of("*"), // With no KEEP we should keep all fields + Set.of() // since global field names are wildcarded, we don't need to wildcard any indices + ); + } + + public void testMultiLookupJoinSameIndexKeepBefore() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | KEEP @timestamp, client_ip, event_duration, message + | LOOKUP JOIN clientips_lookup ON client_ip + | EVAL client_ip = message + | LOOKUP JOIN clientips_lookup ON client_ip""", + Set.of("@timestamp", "@timestamp.*", "client_ip", "client_ip.*", "event_duration", "event_duration.*", "message", "message.*"), + Set.of("clientips_lookup") // Since there is no KEEP after the last JOIN, we need to wildcard the index + ); + } + + public void testMultiLookupJoinSameIndexKeepBetween() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | LOOKUP JOIN clientips_lookup ON client_ip + | KEEP @timestamp, client_ip, event_duration, message, env + | EVAL client_ip = message + | LOOKUP JOIN clientips_lookup ON client_ip""", + Set.of( + "@timestamp", + "@timestamp.*", + "client_ip", + "client_ip.*", + "event_duration", + "event_duration.*", + "message", + "message.*", + "env", + "env.*" + ), + Set.of("clientips_lookup") // Since there is no KEEP after the last JOIN, we need to wildcard the index + ); + } + + public void testMultiLookupJoinSameIndexKeepAfter() { + assertFieldNames( + """ + FROM sample_data + | EVAL client_ip = client_ip::keyword + | LOOKUP JOIN clientips_lookup ON client_ip + | EVAL client_ip = message + | LOOKUP JOIN clientips_lookup ON client_ip + | KEEP @timestamp, client_ip, event_duration, message, env""", + Set.of( + "@timestamp", + "@timestamp.*", + "client_ip", + "client_ip.*", + "event_duration", + "event_duration.*", + "message", + "message.*", + "env", + "env.*" + ), + Set.of() // Since the KEEP is after both JOINs, we can use the global field names + ); + } + + private Set fieldNames(String query, Set enrichPolicyMatchFields) { + var preAnalysisResult = new EsqlSession.PreAnalysisResult(null); + return EsqlSession.fieldNames(parser.createStatement(query), enrichPolicyMatchFields, preAnalysisResult).fieldNames(); + } + private void assertFieldNames(String query, Set expected) { - Set fieldNames = EsqlSession.fieldNames(parser.createStatement(query), Collections.emptySet()); + Set fieldNames = fieldNames(query, Collections.emptySet()); assertThat(fieldNames, equalTo(expected)); } + + private void assertFieldNames(String query, Set expected, Set wildCardIndices) { + var preAnalysisResult = EsqlSession.fieldNames(parser.createStatement(query), Set.of(), new EsqlSession.PreAnalysisResult(null)); + assertThat("Query-wide field names", preAnalysisResult.fieldNames(), equalTo(expected)); + assertThat("Lookup Indices that expect wildcard lookups", preAnalysisResult.wildcardJoinIndices(), equalTo(wildCardIndices)); + } } From 8deb32fb1beb686eb8a0e853b22aadb2d382f1c1 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Tue, 17 Dec 2024 08:25:44 +1100 Subject: [PATCH 17/44] Mute org.elasticsearch.xpack.ccr.rest.ShardChangesRestIT testShardChangesNoOperation #118800 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index be3805d887bbe..0aea4100a2606 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -299,6 +299,9 @@ tests: - class: org.elasticsearch.reservedstate.service.FileSettingsServiceTests method: testInvalidJSON issue: https://github.com/elastic/elasticsearch/issues/116521 +- class: org.elasticsearch.xpack.ccr.rest.ShardChangesRestIT + method: testShardChangesNoOperation + issue: https://github.com/elastic/elasticsearch/issues/118800 # Examples: # From 5c55c39f9f320aa4defb9f4458ef87d9ac994779 Mon Sep 17 00:00:00 2001 From: Kathleen DeRusso Date: Mon, 16 Dec 2024 17:20:33 -0500 Subject: [PATCH 18/44] Fix semantic text match failure (#118790) * Fix semantic text match failure * Remove shard settings; only check for counts and not specific doc IDs --- .../rest-api-spec/test/inference/45_semantic_text_match.yml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/45_semantic_text_match.yml b/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/45_semantic_text_match.yml index cdbf73d31a272..28093ba49e6cc 100644 --- a/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/45_semantic_text_match.yml +++ b/x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/45_semantic_text_match.yml @@ -210,8 +210,6 @@ setup: query: "inference test" - match: { hits.total.value: 2 } - - match: { hits.hits.0._id: "doc_1" } - - match: { hits.hits.1._id: "doc_2" } # Test querying multiple indices that either use the same inference ID or combine semantic_text with lexical search - do: @@ -246,9 +244,6 @@ setup: query: "inference test" - match: { hits.total.value: 3 } - - match: { hits.hits.0._id: "doc_1" } - - match: { hits.hits.1._id: "doc_3" } - - match: { hits.hits.2._id: "doc_2" } --- "Query a field that has no indexed inference results": From e1d06a8881577c90f57cf0e0a379df78ba22b22e Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Tue, 17 Dec 2024 10:14:38 +1100 Subject: [PATCH 19/44] Mute org.elasticsearch.xpack.security.QueryableReservedRolesIT testDeletingAndCreatingSecurityIndexTriggersSynchronization #118806 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 0aea4100a2606..b43045d24be81 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -302,6 +302,9 @@ tests: - class: org.elasticsearch.xpack.ccr.rest.ShardChangesRestIT method: testShardChangesNoOperation issue: https://github.com/elastic/elasticsearch/issues/118800 +- class: org.elasticsearch.xpack.security.QueryableReservedRolesIT + method: testDeletingAndCreatingSecurityIndexTriggersSynchronization + issue: https://github.com/elastic/elasticsearch/issues/118806 # Examples: # From b44469736c92cfb3dcb21d3aaa177765fd643d5d Mon Sep 17 00:00:00 2001 From: Andrei Stefan Date: Tue, 17 Dec 2024 08:07:56 +0200 Subject: [PATCH 20/44] Remove esql-core ParsingException (#118789) * Remove esql-core ParsingException Move fulltext related classes: querydsl, predicate and tests --- .../xpack/esql/core/ParsingException.java | 56 ------------------- .../core/planner/ExpressionTranslators.java | 14 ----- .../function/EsqlFunctionRegistry.java | 2 +- .../function/fulltext/FullTextWritables.java | 4 +- .../predicate/fulltext/FullTextPredicate.java | 2 +- .../predicate/fulltext/FullTextUtils.java | 8 +-- .../fulltext/MatchQueryPredicate.java | 2 +- .../fulltext/MultiMatchQueryPredicate.java | 2 +- .../xpack/esql/parser/EsqlParser.java | 6 +- .../xpack/esql/parser/ParserUtils.java | 1 - .../xpack/esql/parser/ParsingException.java | 2 +- .../planner/EsqlExpressionTranslators.java | 18 +++++- .../esql}/querydsl/query/MatchQuery.java | 3 +- .../esql}/querydsl/query/MultiMatchQuery.java | 5 +- .../xpack/esql/analysis/ParsingTests.java | 22 ++++---- .../function/EsqlFunctionRegistryTests.java | 2 +- .../AbstractFulltextSerializationTests.java | 3 +- .../fulltext/FullTextUtilsTests.java | 12 ++-- .../MatchQuerySerializationTests.java | 3 +- .../MultiMatchQuerySerializationTests.java | 4 +- .../esql}/querydsl/query/BoolQueryTests.java | 6 +- .../esql}/querydsl/query/MatchQueryTests.java | 4 +- .../querydsl/query/MultiMatchQueryTests.java | 4 +- .../querydsl/query/QueryStringQueryTests.java | 3 +- .../esql/tree/EsqlNodeSubclassTests.java | 2 +- 25 files changed, 66 insertions(+), 124 deletions(-) delete mode 100644 x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/ParsingException.java rename x-pack/plugin/{esql-core/src/main/java/org/elasticsearch/xpack/esql/core => esql/src/main/java/org/elasticsearch/xpack/esql}/expression/predicate/fulltext/FullTextPredicate.java (97%) rename x-pack/plugin/{esql-core/src/main/java/org/elasticsearch/xpack/esql/core => esql/src/main/java/org/elasticsearch/xpack/esql}/expression/predicate/fulltext/FullTextUtils.java (90%) rename x-pack/plugin/{esql-core/src/main/java/org/elasticsearch/xpack/esql/core => esql/src/main/java/org/elasticsearch/xpack/esql}/expression/predicate/fulltext/MatchQueryPredicate.java (96%) rename x-pack/plugin/{esql-core/src/main/java/org/elasticsearch/xpack/esql/core => esql/src/main/java/org/elasticsearch/xpack/esql}/expression/predicate/fulltext/MultiMatchQueryPredicate.java (97%) rename x-pack/plugin/{esql-core/src/main/java/org/elasticsearch/xpack/esql/core => esql/src/main/java/org/elasticsearch/xpack/esql}/querydsl/query/MatchQuery.java (97%) rename x-pack/plugin/{esql-core/src/main/java/org/elasticsearch/xpack/esql/core => esql/src/main/java/org/elasticsearch/xpack/esql}/querydsl/query/MultiMatchQuery.java (95%) rename x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/{operator => }/fulltext/AbstractFulltextSerializationTests.java (88%) rename x-pack/plugin/{esql-core/src/test/java/org/elasticsearch/xpack/esql/core => esql/src/test/java/org/elasticsearch/xpack/esql}/expression/predicate/fulltext/FullTextUtilsTests.java (79%) rename x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/{operator => }/fulltext/MatchQuerySerializationTests.java (89%) rename x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/{operator => }/fulltext/MultiMatchQuerySerializationTests.java (92%) rename x-pack/plugin/{esql-core/src/test/java/org/elasticsearch/xpack/esql/core => esql/src/test/java/org/elasticsearch/xpack/esql}/querydsl/query/BoolQueryTests.java (92%) rename x-pack/plugin/{esql-core/src/test/java/org/elasticsearch/xpack/esql/core => esql/src/test/java/org/elasticsearch/xpack/esql}/querydsl/query/MatchQueryTests.java (96%) rename x-pack/plugin/{esql-core/src/test/java/org/elasticsearch/xpack/esql/core => esql/src/test/java/org/elasticsearch/xpack/esql}/querydsl/query/MultiMatchQueryTests.java (94%) rename x-pack/plugin/{esql-core/src/test/java/org/elasticsearch/xpack/esql/core => esql/src/test/java/org/elasticsearch/xpack/esql}/querydsl/query/QueryStringQueryTests.java (94%) diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/ParsingException.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/ParsingException.java deleted file mode 100644 index bce3f848c9387..0000000000000 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/ParsingException.java +++ /dev/null @@ -1,56 +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.esql.core; - -import org.elasticsearch.xpack.esql.core.tree.Source; - -import static org.elasticsearch.common.logging.LoggerMessageFormat.format; - -public class ParsingException extends QlClientException { - private final int line; - private final int charPositionInLine; - - public ParsingException(String message, Exception cause, int line, int charPositionInLine) { - super(message, cause); - this.line = line; - this.charPositionInLine = charPositionInLine; - } - - public ParsingException(String message, Object... args) { - this(Source.EMPTY, message, args); - } - - public ParsingException(Source source, String message, Object... args) { - super(message, args); - this.line = source.source().getLineNumber(); - this.charPositionInLine = source.source().getColumnNumber(); - } - - public ParsingException(Exception cause, Source source, String message, Object... args) { - super(cause, message, args); - this.line = source.source().getLineNumber(); - this.charPositionInLine = source.source().getColumnNumber(); - } - - public int getLineNumber() { - return line; - } - - public int getColumnNumber() { - return charPositionInLine + 1; - } - - public String getErrorMessage() { - return super.getMessage(); - } - - @Override - public String getMessage() { - return format("line {}:{}: {}", getLineNumber(), getColumnNumber(), getErrorMessage()); - } -} diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/ExpressionTranslators.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/ExpressionTranslators.java index 468d076c1b7ef..e0f4f6b032662 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/ExpressionTranslators.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/planner/ExpressionTranslators.java @@ -11,7 +11,6 @@ import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; import org.elasticsearch.xpack.esql.core.expression.MetadataAttribute; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MultiMatchQueryPredicate; import org.elasticsearch.xpack.esql.core.expression.predicate.logical.And; import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Not; import org.elasticsearch.xpack.esql.core.expression.predicate.logical.Or; @@ -22,7 +21,6 @@ import org.elasticsearch.xpack.esql.core.expression.predicate.regex.WildcardLike; import org.elasticsearch.xpack.esql.core.querydsl.query.BoolQuery; import org.elasticsearch.xpack.esql.core.querydsl.query.ExistsQuery; -import org.elasticsearch.xpack.esql.core.querydsl.query.MultiMatchQuery; import org.elasticsearch.xpack.esql.core.querydsl.query.NotQuery; import org.elasticsearch.xpack.esql.core.querydsl.query.Query; import org.elasticsearch.xpack.esql.core.querydsl.query.RegexQuery; @@ -71,18 +69,6 @@ private static Query translateField(RegexMatch e, String targetFieldName) { } } - public static class MultiMatches extends ExpressionTranslator { - - @Override - protected Query asQuery(MultiMatchQueryPredicate q, TranslatorHandler handler) { - return doTranslate(q, handler); - } - - public static Query doTranslate(MultiMatchQueryPredicate q, TranslatorHandler handler) { - return new MultiMatchQuery(q.source(), q.query(), q.fields(), q); - } - } - public static class BinaryLogic extends ExpressionTranslator< org.elasticsearch.xpack.esql.core.expression.predicate.logical.BinaryLogic> { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java index 1ccc22eb3a6a4..a59ef5bb1575d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistry.java @@ -11,7 +11,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.FeatureFlag; -import org.elasticsearch.xpack.esql.core.ParsingException; import org.elasticsearch.xpack.esql.core.QlIllegalArgumentException; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.function.Function; @@ -147,6 +146,7 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.string.ToUpper; import org.elasticsearch.xpack.esql.expression.function.scalar.string.Trim; import org.elasticsearch.xpack.esql.expression.function.scalar.util.Delay; +import org.elasticsearch.xpack.esql.parser.ParsingException; import org.elasticsearch.xpack.esql.session.Configuration; import java.lang.reflect.Constructor; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java index d6b79d16b74f6..245aca5b7328e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/fulltext/FullTextWritables.java @@ -9,8 +9,8 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.xpack.esql.action.EsqlCapabilities; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MatchQueryPredicate; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MultiMatchQueryPredicate; +import org.elasticsearch.xpack.esql.expression.predicate.fulltext.MatchQueryPredicate; +import org.elasticsearch.xpack.esql.expression.predicate.fulltext.MultiMatchQueryPredicate; import java.util.ArrayList; import java.util.Collections; diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextPredicate.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextPredicate.java similarity index 97% rename from x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextPredicate.java rename to x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextPredicate.java index b23593804f8fe..1dd6f650828c3 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextPredicate.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextPredicate.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.expression.predicate.fulltext; +package org.elasticsearch.xpack.esql.expression.predicate.fulltext; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextUtils.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextUtils.java similarity index 90% rename from x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextUtils.java rename to x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextUtils.java index 6ba2650314d04..32c8e70a0fde6 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextUtils.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextUtils.java @@ -4,13 +4,13 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.expression.predicate.fulltext; +package org.elasticsearch.xpack.esql.expression.predicate.fulltext; import org.elasticsearch.common.Strings; import org.elasticsearch.common.util.Maps; -import org.elasticsearch.xpack.esql.core.ParsingException; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.FullTextPredicate.Operator; import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.expression.predicate.fulltext.FullTextPredicate.Operator; +import org.elasticsearch.xpack.esql.parser.ParsingException; import java.util.LinkedHashMap; import java.util.Locale; @@ -86,7 +86,7 @@ private static String[] splitInTwo(String string, String delimiter) { return split; } - static FullTextPredicate.Operator operator(Map options, String key) { + static Operator operator(Map options, String key) { String value = options.get(key); return value != null ? Operator.valueOf(value.toUpperCase(Locale.ROOT)) : null; } diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/MatchQueryPredicate.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MatchQueryPredicate.java similarity index 96% rename from x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/MatchQueryPredicate.java rename to x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MatchQueryPredicate.java index f2e6088167ba5..66c6d8995b24e 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/MatchQueryPredicate.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MatchQueryPredicate.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.expression.predicate.fulltext; +package org.elasticsearch.xpack.esql.expression.predicate.fulltext; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/MultiMatchQueryPredicate.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MultiMatchQueryPredicate.java similarity index 97% rename from x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/MultiMatchQueryPredicate.java rename to x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MultiMatchQueryPredicate.java index 2d66023a1407d..5d165d9ea01f7 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/MultiMatchQueryPredicate.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MultiMatchQueryPredicate.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.expression.predicate.fulltext; +package org.elasticsearch.xpack.esql.expression.predicate.fulltext; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlParser.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlParser.java index 2e55b4df1e223..9538e3ba495db 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlParser.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/EsqlParser.java @@ -70,11 +70,7 @@ private T invokeParser( BiFunction result ) { if (query.length() > MAX_LENGTH) { - throw new org.elasticsearch.xpack.esql.core.ParsingException( - "ESQL statement is too large [{} characters > {}]", - query.length(), - MAX_LENGTH - ); + throw new ParsingException("ESQL statement is too large [{} characters > {}]", query.length(), MAX_LENGTH); } try { EsqlBaseLexer lexer = new EsqlBaseLexer(CharStreams.fromString(query)); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParserUtils.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParserUtils.java index 89b1ae4e37a68..398c6c5aafbb2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParserUtils.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParserUtils.java @@ -14,7 +14,6 @@ import org.antlr.v4.runtime.tree.ParseTreeVisitor; import org.antlr.v4.runtime.tree.TerminalNode; import org.elasticsearch.common.util.Maps; -import org.elasticsearch.xpack.esql.core.ParsingException; import org.elasticsearch.xpack.esql.core.tree.Location; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.util.Check; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParsingException.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParsingException.java index 484a655fc2988..c25ab92437bfc 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParsingException.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/parser/ParsingException.java @@ -21,7 +21,7 @@ public ParsingException(String message, Exception cause, int line, int charPosit this.charPositionInLine = charPositionInLine + 1; } - ParsingException(String message, Object... args) { + public ParsingException(String message, Object... args) { this(Source.EMPTY, message, args); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsqlExpressionTranslators.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsqlExpressionTranslators.java index 43bbf9a5f4ff1..a1765977ee9c2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsqlExpressionTranslators.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsqlExpressionTranslators.java @@ -25,7 +25,6 @@ import org.elasticsearch.xpack.esql.core.planner.ExpressionTranslators; import org.elasticsearch.xpack.esql.core.planner.TranslatorHandler; import org.elasticsearch.xpack.esql.core.querydsl.query.MatchAll; -import org.elasticsearch.xpack.esql.core.querydsl.query.MatchQuery; import org.elasticsearch.xpack.esql.core.querydsl.query.NotQuery; import org.elasticsearch.xpack.esql.core.querydsl.query.Query; import org.elasticsearch.xpack.esql.core.querydsl.query.QueryStringQuery; @@ -44,6 +43,7 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.ip.CIDRMatch; import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesFunction; import org.elasticsearch.xpack.esql.expression.function.scalar.spatial.SpatialRelatesUtils; +import org.elasticsearch.xpack.esql.expression.predicate.fulltext.MultiMatchQueryPredicate; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.Equals; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThan; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.GreaterThanOrEqual; @@ -53,6 +53,8 @@ import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.LessThanOrEqual; import org.elasticsearch.xpack.esql.expression.predicate.operator.comparison.NotEquals; import org.elasticsearch.xpack.esql.querydsl.query.KqlQuery; +import org.elasticsearch.xpack.esql.querydsl.query.MatchQuery; +import org.elasticsearch.xpack.esql.querydsl.query.MultiMatchQuery; import org.elasticsearch.xpack.esql.querydsl.query.SpatialRelatesQuery; import org.elasticsearch.xpack.versionfield.Version; @@ -92,7 +94,7 @@ public final class EsqlExpressionTranslators { new ExpressionTranslators.IsNotNulls(), new ExpressionTranslators.Nots(), new ExpressionTranslators.Likes(), - new ExpressionTranslators.MultiMatches(), + new MultiMatches(), new MatchFunctionTranslator(), new QueryStringFunctionTranslator(), new KqlFunctionTranslator(), @@ -537,6 +539,18 @@ private static RangeQuery translate(Range r, TranslatorHandler handler) { } } + public static class MultiMatches extends ExpressionTranslator { + + @Override + protected Query asQuery(MultiMatchQueryPredicate q, TranslatorHandler handler) { + return doTranslate(q, handler); + } + + public static Query doTranslate(MultiMatchQueryPredicate q, TranslatorHandler handler) { + return new MultiMatchQuery(q.source(), q.query(), q.fields(), q); + } + } + public static class MatchFunctionTranslator extends ExpressionTranslator { @Override protected Query asQuery(Match match, TranslatorHandler handler) { diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/querydsl/query/MatchQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchQuery.java similarity index 97% rename from x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/querydsl/query/MatchQuery.java rename to x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchQuery.java index e6b6dc20c951a..1614b4f455456 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/querydsl/query/MatchQuery.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MatchQuery.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.querydsl.query; +package org.elasticsearch.xpack.esql.querydsl.query; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.core.Booleans; @@ -12,6 +12,7 @@ import org.elasticsearch.index.query.Operator; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.xpack.esql.core.querydsl.query.Query; import org.elasticsearch.xpack.esql.core.tree.Source; import java.util.Collections; diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/querydsl/query/MultiMatchQuery.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MultiMatchQuery.java similarity index 95% rename from x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/querydsl/query/MultiMatchQuery.java rename to x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MultiMatchQuery.java index 71e3cb9fd494a..84524bad29e08 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/querydsl/query/MultiMatchQuery.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/querydsl/query/MultiMatchQuery.java @@ -4,7 +4,7 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.querydsl.query; +package org.elasticsearch.xpack.esql.querydsl.query; import org.elasticsearch.common.unit.Fuzziness; import org.elasticsearch.core.Booleans; @@ -12,8 +12,9 @@ import org.elasticsearch.index.query.Operator; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MultiMatchQueryPredicate; +import org.elasticsearch.xpack.esql.core.querydsl.query.Query; import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.expression.predicate.fulltext.MultiMatchQueryPredicate; import java.util.Map; import java.util.Objects; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java index 68529e99c6b1b..205c8943d4e3c 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/ParsingTests.java @@ -12,7 +12,6 @@ import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.json.JsonXContent; import org.elasticsearch.xpack.esql.LoadMapping; -import org.elasticsearch.xpack.esql.core.ParsingException; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.expression.function.EsqlFunctionRegistry; @@ -20,6 +19,7 @@ import org.elasticsearch.xpack.esql.index.EsIndex; import org.elasticsearch.xpack.esql.index.IndexResolution; import org.elasticsearch.xpack.esql.parser.EsqlParser; +import org.elasticsearch.xpack.esql.parser.ParsingException; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.esql.plan.logical.Row; import org.elasticsearch.xpack.esql.type.EsqlDataTypeConverter; @@ -49,27 +49,27 @@ public class ParsingTests extends ESTestCase { ); public void testCaseFunctionInvalidInputs() { - assertEquals("1:23: error building [case]: expects at least two arguments", error("row a = 1 | eval x = case()")); - assertEquals("1:23: error building [case]: expects at least two arguments", error("row a = 1 | eval x = case(a)")); - assertEquals("1:23: error building [case]: expects at least two arguments", error("row a = 1 | eval x = case(1)")); + assertEquals("1:22: error building [case]: expects at least two arguments", error("row a = 1 | eval x = case()")); + assertEquals("1:22: error building [case]: expects at least two arguments", error("row a = 1 | eval x = case(a)")); + assertEquals("1:22: error building [case]: expects at least two arguments", error("row a = 1 | eval x = case(1)")); } public void testConcatFunctionInvalidInputs() { - assertEquals("1:23: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat()")); - assertEquals("1:23: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat(a)")); - assertEquals("1:23: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat(1)")); + assertEquals("1:22: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat()")); + assertEquals("1:22: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat(a)")); + assertEquals("1:22: error building [concat]: expects at least two arguments", error("row a = 1 | eval x = concat(1)")); } public void testCoalesceFunctionInvalidInputs() { - assertEquals("1:23: error building [coalesce]: expects at least one argument", error("row a = 1 | eval x = coalesce()")); + assertEquals("1:22: error building [coalesce]: expects at least one argument", error("row a = 1 | eval x = coalesce()")); } public void testGreatestFunctionInvalidInputs() { - assertEquals("1:23: error building [greatest]: expects at least one argument", error("row a = 1 | eval x = greatest()")); + assertEquals("1:22: error building [greatest]: expects at least one argument", error("row a = 1 | eval x = greatest()")); } public void testLeastFunctionInvalidInputs() { - assertEquals("1:23: error building [least]: expects at least one argument", error("row a = 1 | eval x = least()")); + assertEquals("1:22: error building [least]: expects at least one argument", error("row a = 1 | eval x = least()")); } /** @@ -108,7 +108,7 @@ public void testTooBigQuery() { while (query.length() < EsqlParser.MAX_LENGTH) { query.append(", a = CONCAT(a, a)"); } - assertEquals("-1:0: ESQL statement is too large [1000011 characters > 1000000]", error(query.toString())); + assertEquals("-1:-1: ESQL statement is too large [1000011 characters > 1000000]", error(query.toString())); } private String functionName(EsqlFunctionRegistry registry, Expression functionCall) { diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistryTests.java index 801bd8700d014..50cbbdf4a9338 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistryTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/function/EsqlFunctionRegistryTests.java @@ -10,7 +10,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.compute.operator.EvalOperator; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.esql.core.ParsingException; import org.elasticsearch.xpack.esql.core.QlIllegalArgumentException; import org.elasticsearch.xpack.esql.core.expression.Expression; import org.elasticsearch.xpack.esql.core.expression.function.scalar.ScalarFunction; @@ -19,6 +18,7 @@ import org.elasticsearch.xpack.esql.core.tree.SourceTests; import org.elasticsearch.xpack.esql.core.type.DataType; import org.elasticsearch.xpack.esql.expression.function.scalar.EsqlConfigurationFunction; +import org.elasticsearch.xpack.esql.parser.ParsingException; import org.elasticsearch.xpack.esql.session.Configuration; import java.io.IOException; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/AbstractFulltextSerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/AbstractFulltextSerializationTests.java similarity index 88% rename from x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/AbstractFulltextSerializationTests.java rename to x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/AbstractFulltextSerializationTests.java index 370cfaf67fe0f..abd46f4b2b1aa 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/AbstractFulltextSerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/AbstractFulltextSerializationTests.java @@ -5,9 +5,8 @@ * 2.0. */ -package org.elasticsearch.xpack.esql.expression.predicate.operator.fulltext; +package org.elasticsearch.xpack.esql.expression.predicate.fulltext; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.FullTextPredicate; import org.elasticsearch.xpack.esql.expression.AbstractExpressionSerializationTests; import java.util.HashMap; diff --git a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextUtilsTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextUtilsTests.java similarity index 79% rename from x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextUtilsTests.java rename to x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextUtilsTests.java index c6358b4682a79..46bafe5ebae9c 100644 --- a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/expression/predicate/fulltext/FullTextUtilsTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/FullTextUtilsTests.java @@ -4,11 +4,11 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.expression.predicate.fulltext; +package org.elasticsearch.xpack.esql.expression.predicate.fulltext; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.esql.core.ParsingException; import org.elasticsearch.xpack.esql.core.tree.Source; +import org.elasticsearch.xpack.esql.parser.ParsingException; import java.util.Map; @@ -28,15 +28,15 @@ public void testColonDelimited() { public void testColonDelimitedErrorString() { ParsingException e = expectThrows(ParsingException.class, () -> FullTextUtils.parseSettings("k1=v1;k2v2", source)); - assertThat(e.getMessage(), is("line 1:3: Cannot parse entry k2v2 in options k1=v1;k2v2")); + assertThat(e.getMessage(), is("line 1:2: Cannot parse entry k2v2 in options k1=v1;k2v2")); assertThat(e.getLineNumber(), is(1)); - assertThat(e.getColumnNumber(), is(3)); + assertThat(e.getColumnNumber(), is(2)); } public void testColonDelimitedErrorDuplicate() { ParsingException e = expectThrows(ParsingException.class, () -> FullTextUtils.parseSettings("k1=v1;k1=v2", source)); - assertThat(e.getMessage(), is("line 1:3: Duplicate option k1=v2 detected in options k1=v1;k1=v2")); + assertThat(e.getMessage(), is("line 1:2: Duplicate option k1=v2 detected in options k1=v1;k1=v2")); assertThat(e.getLineNumber(), is(1)); - assertThat(e.getColumnNumber(), is(3)); + assertThat(e.getColumnNumber(), is(2)); } } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/MatchQuerySerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MatchQuerySerializationTests.java similarity index 89% rename from x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/MatchQuerySerializationTests.java rename to x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MatchQuerySerializationTests.java index 80a538cf84baa..7781c804a6dfc 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/MatchQuerySerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MatchQuerySerializationTests.java @@ -5,9 +5,8 @@ * 2.0. */ -package org.elasticsearch.xpack.esql.expression.predicate.operator.fulltext; +package org.elasticsearch.xpack.esql.expression.predicate.fulltext; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MatchQueryPredicate; import org.elasticsearch.xpack.esql.expression.AbstractExpressionSerializationTests; import java.io.IOException; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/MultiMatchQuerySerializationTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MultiMatchQuerySerializationTests.java similarity index 92% rename from x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/MultiMatchQuerySerializationTests.java rename to x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MultiMatchQuerySerializationTests.java index d4d0f2edc11b1..17843e24a8663 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/operator/fulltext/MultiMatchQuerySerializationTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/predicate/fulltext/MultiMatchQuerySerializationTests.java @@ -5,9 +5,7 @@ * 2.0. */ -package org.elasticsearch.xpack.esql.expression.predicate.operator.fulltext; - -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MultiMatchQueryPredicate; +package org.elasticsearch.xpack.esql.expression.predicate.fulltext; import java.io.IOException; import java.util.HashMap; diff --git a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/BoolQueryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/BoolQueryTests.java similarity index 92% rename from x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/BoolQueryTests.java rename to x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/BoolQueryTests.java index 1c9d6bc54aebf..1aa5d47ed07ea 100644 --- a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/BoolQueryTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/BoolQueryTests.java @@ -4,9 +4,13 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.querydsl.query; +package org.elasticsearch.xpack.esql.querydsl.query; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.core.querydsl.query.BoolQuery; +import org.elasticsearch.xpack.esql.core.querydsl.query.ExistsQuery; +import org.elasticsearch.xpack.esql.core.querydsl.query.NotQuery; +import org.elasticsearch.xpack.esql.core.querydsl.query.Query; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.tree.SourceTests; import org.elasticsearch.xpack.esql.core.util.StringUtils; diff --git a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/MatchQueryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MatchQueryTests.java similarity index 96% rename from x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/MatchQueryTests.java rename to x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MatchQueryTests.java index 4316bd21ffe94..49d1a9ad19d09 100644 --- a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/MatchQueryTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MatchQueryTests.java @@ -4,17 +4,17 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.querydsl.query; +package org.elasticsearch.xpack.esql.querydsl.query; import org.elasticsearch.index.query.MatchQueryBuilder; import org.elasticsearch.index.query.Operator; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MatchQueryPredicate; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.tree.SourceTests; import org.elasticsearch.xpack.esql.core.type.EsField; import org.elasticsearch.xpack.esql.core.util.StringUtils; +import org.elasticsearch.xpack.esql.expression.predicate.fulltext.MatchQueryPredicate; import java.util.Arrays; import java.util.List; diff --git a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/MultiMatchQueryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MultiMatchQueryTests.java similarity index 94% rename from x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/MultiMatchQueryTests.java rename to x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MultiMatchQueryTests.java index 9ca9765ed0542..93c285f5e3ab0 100644 --- a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/MultiMatchQueryTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/MultiMatchQueryTests.java @@ -4,14 +4,14 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.querydsl.query; +package org.elasticsearch.xpack.esql.querydsl.query; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.index.query.MultiMatchQueryBuilder; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.MultiMatchQueryPredicate; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.util.StringUtils; +import org.elasticsearch.xpack.esql.expression.predicate.fulltext.MultiMatchQueryPredicate; import java.util.HashMap; import java.util.Map; diff --git a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/QueryStringQueryTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/QueryStringQueryTests.java similarity index 94% rename from x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/QueryStringQueryTests.java rename to x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/QueryStringQueryTests.java index 22e7b93e84ce1..3114b852aac70 100644 --- a/x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/querydsl/query/QueryStringQueryTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/querydsl/query/QueryStringQueryTests.java @@ -4,12 +4,13 @@ * 2.0; you may not use this file except in compliance with the Elastic License * 2.0. */ -package org.elasticsearch.xpack.esql.core.querydsl.query; +package org.elasticsearch.xpack.esql.querydsl.query; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.index.query.Operator; import org.elasticsearch.index.query.QueryStringQueryBuilder; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.core.querydsl.query.QueryStringQuery; import org.elasticsearch.xpack.esql.core.tree.Source; import org.elasticsearch.xpack.esql.core.util.StringUtils; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java index c1d94933537f0..f01a125bc3c23 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/tree/EsqlNodeSubclassTests.java @@ -25,7 +25,6 @@ import org.elasticsearch.xpack.esql.core.expression.UnresolvedAttributeTests; import org.elasticsearch.xpack.esql.core.expression.UnresolvedNamedExpression; import org.elasticsearch.xpack.esql.core.expression.function.Function; -import org.elasticsearch.xpack.esql.core.expression.predicate.fulltext.FullTextPredicate; import org.elasticsearch.xpack.esql.core.tree.AbstractNodeTestCase; import org.elasticsearch.xpack.esql.core.tree.Node; import org.elasticsearch.xpack.esql.core.tree.NodeInfo; @@ -40,6 +39,7 @@ import org.elasticsearch.xpack.esql.expression.function.scalar.ip.CIDRMatch; import org.elasticsearch.xpack.esql.expression.function.scalar.math.Pow; import org.elasticsearch.xpack.esql.expression.function.scalar.string.Concat; +import org.elasticsearch.xpack.esql.expression.predicate.fulltext.FullTextPredicate; import org.elasticsearch.xpack.esql.plan.logical.Dissect; import org.elasticsearch.xpack.esql.plan.logical.Grok; import org.elasticsearch.xpack.esql.plan.logical.LogicalPlan; From ce7a0c86fb975f76dfbea5eff1cf51602dbed184 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Tue, 17 Dec 2024 17:34:22 +1100 Subject: [PATCH 21/44] Mute org.elasticsearch.xpack.esql.session.IndexResolverFieldNamesTests org.elasticsearch.xpack.esql.session.IndexResolverFieldNamesTests #118814 --- muted-tests.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index b43045d24be81..da52d585267a5 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -305,6 +305,8 @@ tests: - class: org.elasticsearch.xpack.security.QueryableReservedRolesIT method: testDeletingAndCreatingSecurityIndexTriggersSynchronization issue: https://github.com/elastic/elasticsearch/issues/118806 +- class: org.elasticsearch.xpack.esql.session.IndexResolverFieldNamesTests + issue: https://github.com/elastic/elasticsearch/issues/118814 # Examples: # From d09d57db802ecba45195abfd1b38c12818bbe537 Mon Sep 17 00:00:00 2001 From: Carlos Delgado <6339205+carlosdelest@users.noreply.github.com> Date: Tue, 17 Dec 2024 09:14:50 +0100 Subject: [PATCH 22/44] Fix BwC synonyms tests (#118691) --- muted-tests.yml | 3 -- rest-api-spec/build.gradle | 1 + .../90_synonyms_reloading_for_synset.yml | 33 +++++++++++++++---- 3 files changed, 28 insertions(+), 9 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index da52d585267a5..fe6c77bdf9f93 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -156,9 +156,6 @@ tests: issue: https://github.com/elastic/elasticsearch/issues/117473 - class: org.elasticsearch.repositories.s3.RepositoryS3EcsClientYamlTestSuiteIT issue: https://github.com/elastic/elasticsearch/issues/117525 -- class: org.elasticsearch.backwards.MixedClusterClientYamlTestSuiteIT - method: test {p0=synonyms/90_synonyms_reloading_for_synset/Reload analyzers for specific synonym set} - issue: https://github.com/elastic/elasticsearch/issues/116777 - class: "org.elasticsearch.xpack.esql.qa.multi_node.EsqlSpecIT" method: "test {scoring.*}" issue: https://github.com/elastic/elasticsearch/issues/117641 diff --git a/rest-api-spec/build.gradle b/rest-api-spec/build.gradle index 7347d9c1312dd..bdee32e596c4c 100644 --- a/rest-api-spec/build.gradle +++ b/rest-api-spec/build.gradle @@ -69,4 +69,5 @@ tasks.named("yamlRestCompatTestTransform").configure ({ task -> task.skipTest("search/520_fetch_fields/fetch _seq_no via fields", "error code is changed from 5xx to 400 in 9.0") task.skipTest("search.vectors/41_knn_search_bbq_hnsw/Test knn search", "Scoring has changed in latest versions") task.skipTest("search.vectors/42_knn_search_bbq_flat/Test knn search", "Scoring has changed in latest versions") + task.skipTest("synonyms/90_synonyms_reloading_for_synset/Reload analyzers for specific synonym set", "Can't work until auto-expand replicas is 0-1 for synonyms index") }) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/synonyms/90_synonyms_reloading_for_synset.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/synonyms/90_synonyms_reloading_for_synset.yml index d6c98673253fb..4e6bd83f07955 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/synonyms/90_synonyms_reloading_for_synset.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/synonyms/90_synonyms_reloading_for_synset.yml @@ -1,8 +1,8 @@ ---- -"Reload analyzers for specific synonym set": +setup: - requires: cluster_features: ["gte_v8.10.0"] reason: Reloading analyzers for specific synonym set is introduced in 8.10.0 + # Create synonyms_set1 - do: synonyms.put_synonym: @@ -100,7 +100,12 @@ - '{"index": {"_index": "my_index2", "_id": "2"}}' - '{"my_field": "goodbye"}' - # An update of synonyms_set1 must trigger auto-reloading of analyzers only for synonyms_set1 +--- +"Reload analyzers for specific synonym set": +# These specific tests can't succeed in BwC, as synonyms auto-expand replicas are 0-all. Replicas can't be associated to +# upgraded nodes, and thus we are not able to guarantee that the shards are not failed. +# This test is skipped for BwC until synonyms index has auto-exapnd replicas set to 0-1. + - do: synonyms.put_synonym: id: synonyms_set1 @@ -108,13 +113,12 @@ synonyms_set: - synonyms: "hello, salute" - synonyms: "ciao => goodbye" + - match: { result: "updated" } - gt: { reload_analyzers_details._shards.total: 0 } - gt: { reload_analyzers_details._shards.successful: 0 } - match: { reload_analyzers_details._shards.failed: 0 } - - length: { reload_analyzers_details.reload_details: 1 } # reload details contain only a single index - - match: { reload_analyzers_details.reload_details.0.index: "my_index1" } - - match: { reload_analyzers_details.reload_details.0.reloaded_analyzers.0: "my_analyzer1" } + # Confirm that the index analyzers are reloaded for my_index1 - do: @@ -127,6 +131,23 @@ query: salute - match: { hits.total.value: 1 } +--- +"Check analyzer reloaded and non failed shards for bwc tests": + + - do: + synonyms.put_synonym: + id: synonyms_set1 + body: + synonyms_set: + - synonyms: "hello, salute" + - synonyms: "ciao => goodbye" + - match: { result: "updated" } + - gt: { reload_analyzers_details._shards.total: 0 } + - gt: { reload_analyzers_details._shards.successful: 0 } + - length: { reload_analyzers_details.reload_details: 1 } # reload details contain only a single index + - match: { reload_analyzers_details.reload_details.0.index: "my_index1" } + - match: { reload_analyzers_details.reload_details.0.reloaded_analyzers.0: "my_analyzer1" } + # Confirm that the index analyzers are still the same for my_index2 - do: search: From 87c9d13ed52668fb22e2c5da07744516aa84bf09 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Slobodan=20Adamovi=C4=87?= Date: Tue, 17 Dec 2024 10:49:25 +0100 Subject: [PATCH 23/44] Mark Query Role API as public in seerverless (#118798) This commit makes Query Role API available in Serverless. --- .../xpack/security/rest/action/role/RestQueryRoleAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/role/RestQueryRoleAction.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/role/RestQueryRoleAction.java index 3637159479463..862ff2552b4e3 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/role/RestQueryRoleAction.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/rest/action/role/RestQueryRoleAction.java @@ -34,7 +34,7 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg; -@ServerlessScope(Scope.INTERNAL) +@ServerlessScope(Scope.PUBLIC) public final class RestQueryRoleAction extends NativeRoleBaseRestHandler { @SuppressWarnings("unchecked") From 8793248d6d5dd1135327d938ae7a69e91c0eed00 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 17 Dec 2024 11:22:20 +0100 Subject: [PATCH 24/44] Remove unused code from PercolateQueryBuilder (#118791) --- .../percolator/PercolateQueryBuilder.java | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java index 85af5b120f6fd..c150f01153d35 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/PercolateQueryBuilder.java @@ -43,7 +43,6 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.index.IndexVersion; @@ -80,7 +79,6 @@ import java.util.Collections; import java.util.List; import java.util.Objects; -import java.util.function.BiConsumer; import java.util.function.Supplier; import static org.elasticsearch.search.SearchService.ALLOW_EXPENSIVE_QUERIES; @@ -88,20 +86,12 @@ import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg; public class PercolateQueryBuilder extends AbstractQueryBuilder { - private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(ParseField.class); - static final String DOCUMENT_TYPE_DEPRECATION_MESSAGE = "[types removal] Types are deprecated in [percolate] queries. " - + "The [document_type] should no longer be specified."; - static final String TYPE_DEPRECATION_MESSAGE = "[types removal] Types are deprecated in [percolate] queries. " - + "The [type] of the indexed document should no longer be specified."; - public static final String NAME = "percolate"; static final ParseField DOCUMENT_FIELD = new ParseField("document"); static final ParseField DOCUMENTS_FIELD = new ParseField("documents"); private static final ParseField NAME_FIELD = new ParseField("name"); private static final ParseField QUERY_FIELD = new ParseField("field"); - private static final ParseField DOCUMENT_TYPE_FIELD = new ParseField("document_type"); - private static final ParseField INDEXED_DOCUMENT_FIELD_TYPE = new ParseField("type"); private static final ParseField INDEXED_DOCUMENT_FIELD_INDEX = new ParseField("index"); private static final ParseField INDEXED_DOCUMENT_FIELD_ID = new ParseField("id"); private static final ParseField INDEXED_DOCUMENT_FIELD_ROUTING = new ParseField("routing"); @@ -368,10 +358,6 @@ protected void doXContent(XContentBuilder builder, Params params) throws IOExcep ); } - private static BiConsumer deprecateAndIgnoreType(String key, String message) { - return (target, type) -> deprecationLogger.compatibleCritical(key, message); - } - private static BytesReference parseDocument(XContentParser parser) throws IOException { try (XContentBuilder builder = XContentFactory.jsonBuilder()) { builder.copyCurrentStructure(parser); From 8134c79ce8a11b26ee26d88ad7975c91a127af98 Mon Sep 17 00:00:00 2001 From: Alexander Spies Date: Tue, 17 Dec 2024 13:05:20 +0100 Subject: [PATCH 25/44] ESQL: Skip lookup fields when eliminating missing fields (#118658) We do not have SearchStats for fields from lookup indices. And unfortunately, these are hard to obtain. For now, just do not apply ReplaceMissingFieldWithNull to fields coming from an index used in LOOKUP JOIN. These are identified via their indexmode. --- .../esql/qa/mixed/MixedClusterEsqlSpecIT.java | 4 +- .../xpack/esql/ccq/MultiClusterSpecIT.java | 8 +- .../rest/RequestIndexFilteringTestCase.java | 2 +- .../src/main/resources/lookup-join.csv-spec | 212 ++++++++++++------ .../xpack/esql/action/EsqlCapabilities.java | 2 +- .../local/ReplaceMissingFieldWithNull.java | 15 +- .../elasticsearch/xpack/esql/CsvTests.java | 2 +- .../xpack/esql/analysis/AnalyzerTests.java | 4 +- .../xpack/esql/analysis/VerifierTests.java | 2 +- .../optimizer/LogicalPlanOptimizerTests.java | 12 +- .../optimizer/PhysicalPlanOptimizerTests.java | 2 +- 11 files changed, 180 insertions(+), 85 deletions(-) diff --git a/x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/MixedClusterEsqlSpecIT.java b/x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/MixedClusterEsqlSpecIT.java index 5efe7ffc800a2..004beaafb4009 100644 --- a/x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/MixedClusterEsqlSpecIT.java +++ b/x-pack/plugin/esql/qa/server/mixed-cluster/src/javaRestTest/java/org/elasticsearch/xpack/esql/qa/mixed/MixedClusterEsqlSpecIT.java @@ -21,7 +21,7 @@ import java.util.List; import static org.elasticsearch.xpack.esql.CsvTestUtils.isEnabled; -import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V6; +import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V7; import static org.elasticsearch.xpack.esql.qa.rest.EsqlSpecTestCase.Mode.ASYNC; public class MixedClusterEsqlSpecIT extends EsqlSpecTestCase { @@ -96,7 +96,7 @@ protected boolean supportsInferenceTestService() { @Override protected boolean supportsIndexModeLookup() throws IOException { - return hasCapabilities(List.of(JOIN_LOOKUP_V6.capabilityName())); + return hasCapabilities(List.of(JOIN_LOOKUP_V7.capabilityName())); } @Override diff --git a/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java b/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java index dd75776973c3d..c75a920e16973 100644 --- a/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java +++ b/x-pack/plugin/esql/qa/server/multi-clusters/src/javaRestTest/java/org/elasticsearch/xpack/esql/ccq/MultiClusterSpecIT.java @@ -48,7 +48,7 @@ import static org.elasticsearch.xpack.esql.EsqlTestUtils.classpathResources; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.INLINESTATS; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.INLINESTATS_V2; -import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V6; +import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_LOOKUP_V7; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.JOIN_PLANNING_V1; import static org.elasticsearch.xpack.esql.action.EsqlCapabilities.Cap.METADATA_FIELDS_REMOTE_TEST; import static org.elasticsearch.xpack.esql.qa.rest.EsqlSpecTestCase.Mode.SYNC; @@ -124,7 +124,7 @@ protected void shouldSkipTest(String testName) throws IOException { assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(INLINESTATS.capabilityName())); assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(INLINESTATS_V2.capabilityName())); assumeFalse("INLINESTATS not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_PLANNING_V1.capabilityName())); - assumeFalse("LOOKUP JOIN not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_LOOKUP_V6.capabilityName())); + assumeFalse("LOOKUP JOIN not yet supported in CCS", testCase.requiredCapabilities.contains(JOIN_LOOKUP_V7.capabilityName())); } private TestFeatureService remoteFeaturesService() throws IOException { @@ -283,8 +283,8 @@ protected boolean supportsInferenceTestService() { @Override protected boolean supportsIndexModeLookup() throws IOException { - // CCS does not yet support JOIN_LOOKUP_V6 and clusters falsely report they have this capability - // return hasCapabilities(List.of(JOIN_LOOKUP_V6.capabilityName())); + // CCS does not yet support JOIN_LOOKUP_V7 and clusters falsely report they have this capability + // return hasCapabilities(List.of(JOIN_LOOKUP_V7.capabilityName())); return false; } } diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java index 2aae4c94c33fe..40027249670f6 100644 --- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RequestIndexFilteringTestCase.java @@ -221,7 +221,7 @@ public void testIndicesDontExist() throws IOException { assertThat(e.getMessage(), containsString("index_not_found_exception")); assertThat(e.getMessage(), containsString("no such index [foo]")); - if (EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()) { + if (EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()) { e = expectThrows( ResponseException.class, () -> runEsql(timestampFilter("gte", "2020-01-01").query("FROM test1 | LOOKUP JOIN foo ON id1")) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec index 8b8d24b1bb156..8bcc2c2ff3502 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/lookup-join.csv-spec @@ -8,7 +8,7 @@ ############################################### basicOnTheDataNode -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | EVAL language_code = languages @@ -25,7 +25,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; basicRow -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW language_code = 1 | LOOKUP JOIN languages_lookup ON language_code @@ -36,7 +36,7 @@ language_code:integer | language_name:keyword ; basicOnTheCoordinator -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | SORT emp_no @@ -53,7 +53,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; subsequentEvalOnTheDataNode -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | EVAL language_code = languages @@ -71,7 +71,7 @@ emp_no:integer | language_code:integer | language_name:keyword | language_code_x ; subsequentEvalOnTheCoordinator -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | SORT emp_no @@ -89,7 +89,7 @@ emp_no:integer | language_code:integer | language_name:keyword | language_code_x ; sortEvalBeforeLookup -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | SORT emp_no @@ -106,7 +106,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; nonUniqueLeftKeyOnTheDataNode -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | WHERE emp_no <= 10030 @@ -130,7 +130,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; nonUniqueRightKeyOnTheDataNode -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | EVAL language_code = emp_no % 10 @@ -150,7 +150,7 @@ emp_no:integer | language_code:integer | language_name:keyword | country:k ; nonUniqueRightKeyOnTheCoordinator -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | SORT emp_no @@ -170,7 +170,7 @@ emp_no:integer | language_code:integer | language_name:keyword | country:k ; nonUniqueRightKeyFromRow -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW language_code = 2 | LOOKUP JOIN languages_lookup_non_unique_key ON language_code @@ -186,8 +186,8 @@ language_code:integer | language_name:keyword | country:keyword # Filtering tests with languages_lookup index ############################################### -lookupWithFilterOnLeftSideField -required_capability: join_lookup_v6 +filterOnLeftSide +required_capability: join_lookup_v7 FROM employees | EVAL language_code = languages @@ -203,8 +203,8 @@ emp_no:integer | language_code:integer | language_name:keyword 10093 | 3 | Spanish ; -lookupMessageWithFilterOnRightSideField-Ignore -required_capability: join_lookup_v6 +filterOnRightSide +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -219,8 +219,8 @@ FROM sample_data 2023-10-23T13:51:54.732Z | 172.21.3.15 | 725448 | Connection error | Error ; -lookupWithFieldAndRightSideAfterStats -required_capability: join_lookup_v6 +filterOnRightSideAfterStats +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -232,23 +232,110 @@ count:long | type:keyword 3 | Error ; -lookupWithFieldOnJoinKey-Ignore -required_capability: join_lookup_v6 +filterOnJoinKey +required_capability: join_lookup_v7 FROM employees | EVAL language_code = languages +| WHERE emp_no >= 10091 AND emp_no < 10094 +| LOOKUP JOIN languages_lookup ON language_code +| WHERE language_code == 1 +| KEEP emp_no, language_code, language_name +; + +emp_no:integer | language_code:integer | language_name:keyword +10092 | 1 | English +; + +filterOnJoinKeyAndRightSide +required_capability: join_lookup_v7 + +FROM employees +| WHERE emp_no < 10006 +| EVAL language_code = languages | LOOKUP JOIN languages_lookup ON language_code | WHERE language_code > 1 AND language_name IS NOT NULL | KEEP emp_no, language_code, language_name ; +ignoreOrder:true emp_no:integer | language_code:integer | language_name:keyword 10001 | 2 | French 10003 | 4 | German ; +filterOnRightSideOnTheCoordinator +required_capability: join_lookup_v7 + +FROM employees +| SORT emp_no +| LIMIT 5 +| EVAL language_code = languages +| LOOKUP JOIN languages_lookup ON language_code +| WHERE language_name == "English" +| KEEP emp_no, language_code, language_name +; + +emp_no:integer | language_code:integer | language_name:keyword +10005 | 1 | English +; + +filterOnJoinKeyOnTheCoordinator +required_capability: join_lookup_v7 + +FROM employees +| SORT emp_no +| LIMIT 5 +| EVAL language_code = languages +| LOOKUP JOIN languages_lookup ON language_code +| WHERE language_code == 1 +| KEEP emp_no, language_code, language_name +; + +emp_no:integer | language_code:integer | language_name:keyword +10005 | 1 | English +; + +filterOnJoinKeyAndRightSideOnTheCoordinator +required_capability: join_lookup_v7 + +FROM employees +| SORT emp_no +| LIMIT 5 +| EVAL language_code = languages +| LOOKUP JOIN languages_lookup ON language_code +| WHERE language_code > 1 AND language_name IS NOT NULL +| KEEP emp_no, language_code, language_name +; + +emp_no:integer | language_code:integer | language_name:keyword +10001 | 2 | French +10003 | 4 | German +; + +filterOnTheDataNodeThenFilterOnTheCoordinator +required_capability: join_lookup_v7 + +FROM employees +| EVAL language_code = languages +| WHERE emp_no >= 10091 AND emp_no < 10094 +| LOOKUP JOIN languages_lookup ON language_code +| WHERE language_name == "English" +| KEEP emp_no, language_code, language_name +| SORT emp_no +| WHERE language_code == 1 +; + +emp_no:integer | language_code:integer | language_name:keyword +10092 | 1 | English +; + +########################################################################### +# null and multi-value behavior with languages_lookup_non_unique_key index +########################################################################### + nullJoinKeyOnTheDataNode -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | WHERE emp_no < 10004 @@ -264,9 +351,8 @@ emp_no:integer | language_code:integer | language_name:keyword 10003 | null | null ; - mvJoinKeyOnTheDataNode -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM employees | WHERE 10003 < emp_no AND emp_no < 10008 @@ -284,7 +370,7 @@ emp_no:integer | language_code:integer | language_name:keyword ; mvJoinKeyFromRow -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW language_code = [4, 5, 6, 7] | LOOKUP JOIN languages_lookup_non_unique_key ON language_code @@ -297,7 +383,7 @@ language_code:integer | language_name:keyword | country:keyword ; mvJoinKeyFromRowExpanded -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW language_code = [4, 5, 6, 7, 8] | MV_EXPAND language_code @@ -319,7 +405,7 @@ language_code:integer | language_name:keyword | country:keyword ############################################### lookupIPFromRow -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", right = "right" | LOOKUP JOIN clientips_lookup ON client_ip @@ -330,7 +416,7 @@ left | 172.21.0.5 | right | Development ; lookupIPFromKeepRow -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", right = "right" | KEEP left, client_ip, right @@ -342,7 +428,7 @@ left | 172.21.0.5 | right | Development ; lookupIPFromRowWithShadowing -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", env = "env", right = "right" | LOOKUP JOIN clientips_lookup ON client_ip @@ -353,7 +439,7 @@ left | 172.21.0.5 | right | Development ; lookupIPFromRowWithShadowingKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", env = "env", right = "right" | EVAL client_ip = client_ip::keyword @@ -366,7 +452,7 @@ left | 172.21.0.5 | right | Development ; lookupIPFromRowWithShadowingKeepReordered -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", env = "env", right = "right" | EVAL client_ip = client_ip::keyword @@ -379,7 +465,7 @@ right | Development | 172.21.0.5 ; lookupIPFromIndex -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -398,7 +484,7 @@ ignoreOrder:true ; lookupIPFromIndexKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -418,7 +504,7 @@ ignoreOrder:true ; lookupIPFromIndexKeepKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | KEEP client_ip, event_duration, @timestamp, message @@ -440,7 +526,7 @@ timestamp:date | client_ip:keyword | event_duration:long | msg:keyword ; lookupIPFromIndexStats -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -456,7 +542,7 @@ count:long | env:keyword ; lookupIPFromIndexStatsKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -473,7 +559,7 @@ count:long | env:keyword ; statsAndLookupIPFromIndex -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -494,7 +580,7 @@ count:long | client_ip:keyword | env:keyword ############################################### lookupMessageFromRow -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", message = "Connected to 10.1.0.1", right = "right" | LOOKUP JOIN message_types_lookup ON message @@ -505,7 +591,7 @@ left | Connected to 10.1.0.1 | right | Success ; lookupMessageFromKeepRow -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", message = "Connected to 10.1.0.1", right = "right" | KEEP left, message, right @@ -517,7 +603,7 @@ left | Connected to 10.1.0.1 | right | Success ; lookupMessageFromRowWithShadowing -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", message = "Connected to 10.1.0.1", type = "unknown", right = "right" | LOOKUP JOIN message_types_lookup ON message @@ -528,7 +614,7 @@ left | Connected to 10.1.0.1 | right | Success ; lookupMessageFromRowWithShadowingKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", message = "Connected to 10.1.0.1", type = "unknown", right = "right" | LOOKUP JOIN message_types_lookup ON message @@ -540,7 +626,7 @@ left | Connected to 10.1.0.1 | right | Success ; lookupMessageFromIndex -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -558,7 +644,7 @@ ignoreOrder:true ; lookupMessageFromIndexKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -577,7 +663,7 @@ ignoreOrder:true ; lookupMessageFromIndexKeepKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | KEEP client_ip, event_duration, @timestamp, message @@ -597,7 +683,7 @@ ignoreOrder:true ; lookupMessageFromIndexKeepReordered -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -616,7 +702,7 @@ Success | 172.21.2.162 | 3450233 | Connected to 10.1.0.3 ; lookupMessageFromIndexStats -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -631,7 +717,7 @@ count:long | type:keyword ; lookupMessageFromIndexStatsKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -647,7 +733,7 @@ count:long | type:keyword ; statsAndLookupMessageFromIndex -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | STATS count = count(message) BY message @@ -665,7 +751,7 @@ count:long | type:keyword | message:keyword ; lookupMessageFromIndexTwice -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -687,7 +773,7 @@ ignoreOrder:true ; lookupMessageFromIndexTwiceKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | LOOKUP JOIN message_types_lookup ON message @@ -714,7 +800,7 @@ ignoreOrder:true ############################################### lookupIPAndMessageFromRow -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", right = "right" | LOOKUP JOIN clientips_lookup ON client_ip @@ -726,7 +812,7 @@ left | 172.21.0.5 | Connected to 10.1.0.1 | right | Devel ; lookupIPAndMessageFromRowKeepBefore -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", right = "right" | KEEP left, client_ip, message, right @@ -739,7 +825,7 @@ left | 172.21.0.5 | Connected to 10.1.0.1 | right | Devel ; lookupIPAndMessageFromRowKeepBetween -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", right = "right" | LOOKUP JOIN clientips_lookup ON client_ip @@ -752,7 +838,7 @@ left | 172.21.0.5 | Connected to 10.1.0.1 | right | Devel ; lookupIPAndMessageFromRowKeepAfter -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", right = "right" | LOOKUP JOIN clientips_lookup ON client_ip @@ -765,7 +851,7 @@ left | 172.21.0.5 | Connected to 10.1.0.1 | right | Devel ; lookupIPAndMessageFromRowWithShadowing -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", type = "type", right = "right" | LOOKUP JOIN clientips_lookup ON client_ip @@ -777,7 +863,7 @@ left | 172.21.0.5 | Connected to 10.1.0.1 | right | Devel ; lookupIPAndMessageFromRowWithShadowingKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", right = "right" | EVAL client_ip = client_ip::keyword @@ -791,7 +877,7 @@ left | 172.21.0.5 | Connected to 10.1.0.1 | right | Devel ; lookupIPAndMessageFromRowWithShadowingKeepKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", right = "right" | EVAL client_ip = client_ip::keyword @@ -806,7 +892,7 @@ left | 172.21.0.5 | Connected to 10.1.0.1 | right | Devel ; lookupIPAndMessageFromRowWithShadowingKeepKeepKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", right = "right" | EVAL client_ip = client_ip::keyword @@ -822,7 +908,7 @@ left | 172.21.0.5 | Connected to 10.1.0.1 | right | Devel ; lookupIPAndMessageFromRowWithShadowingKeepReordered -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 ROW left = "left", client_ip = "172.21.0.5", message = "Connected to 10.1.0.1", env = "env", right = "right" | EVAL client_ip = client_ip::keyword @@ -836,7 +922,7 @@ right | Development | Success | 172.21.0.5 ; lookupIPAndMessageFromIndex -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -856,7 +942,7 @@ ignoreOrder:true ; lookupIPAndMessageFromIndexKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -877,7 +963,7 @@ ignoreOrder:true ; lookupIPAndMessageFromIndexStats -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -895,7 +981,7 @@ count:long | env:keyword | type:keyword ; lookupIPAndMessageFromIndexStatsKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -914,7 +1000,7 @@ count:long | env:keyword | type:keyword ; statsAndLookupIPAndMessageFromIndex -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -933,7 +1019,7 @@ count:long | client_ip:keyword | message:keyword | env:keyword | type:keyw ; lookupIPAndMessageFromIndexChainedEvalKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword @@ -955,7 +1041,7 @@ ignoreOrder:true ; lookupIPAndMessageFromIndexChainedRenameKeep -required_capability: join_lookup_v6 +required_capability: join_lookup_v7 FROM sample_data | EVAL client_ip = client_ip::keyword diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 235d0dcbe4164..4fcabb02b2d4f 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -547,7 +547,7 @@ public enum Cap { /** * LOOKUP JOIN */ - JOIN_LOOKUP_V6(Build.current().isSnapshot()), + JOIN_LOOKUP_V7(Build.current().isSnapshot()), /** * Fix for https://github.com/elastic/elasticsearch/issues/117054 diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceMissingFieldWithNull.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceMissingFieldWithNull.java index 096f72f7694e1..f9d86ecf0f61a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceMissingFieldWithNull.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/local/ReplaceMissingFieldWithNull.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.esql.optimizer.rules.logical.local; import org.elasticsearch.common.util.Maps; +import org.elasticsearch.index.IndexMode; import org.elasticsearch.xpack.esql.core.expression.Alias; import org.elasticsearch.xpack.esql.core.expression.AttributeSet; import org.elasticsearch.xpack.esql.core.expression.FieldAttribute; @@ -41,10 +42,17 @@ public class ReplaceMissingFieldWithNull extends ParameterizedRule missingToNull(p, localLogicalOptimizerContext.searchStats())); + AttributeSet lookupFields = new AttributeSet(); + plan.forEachUp(EsRelation.class, esRelation -> { + if (esRelation.indexMode() == IndexMode.LOOKUP) { + lookupFields.addAll(esRelation.output()); + } + }); + + return plan.transformUp(p -> missingToNull(p, localLogicalOptimizerContext.searchStats(), lookupFields)); } - private LogicalPlan missingToNull(LogicalPlan plan, SearchStats stats) { + private LogicalPlan missingToNull(LogicalPlan plan, SearchStats stats, AttributeSet lookupFields) { if (plan instanceof EsRelation || plan instanceof LocalRelation) { return plan; } @@ -95,7 +103,8 @@ else if (plan instanceof Project project) { plan = plan.transformExpressionsOnlyUp( FieldAttribute.class, // Do not use the attribute name, this can deviate from the field name for union types. - f -> stats.exists(f.fieldName()) ? f : Literal.of(f, null) + // Also skip fields from lookup indices because we do not have stats for these. + f -> stats.exists(f.fieldName()) || lookupFields.contains(f) ? f : Literal.of(f, null) ); } diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java index f553c15ef69fa..717ac7b5a62a7 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/CsvTests.java @@ -263,7 +263,7 @@ public final void test() throws Throwable { ); assumeFalse( "lookup join disabled for csv tests", - testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.JOIN_LOOKUP_V6.capabilityName()) + testCase.requiredCapabilities.contains(EsqlCapabilities.Cap.JOIN_LOOKUP_V7.capabilityName()) ); assumeFalse( "can't use TERM function in csv tests", diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java index 4e02119b31744..9c71f20dcde0e 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/AnalyzerTests.java @@ -2139,7 +2139,7 @@ public void testLookupMatchTypeWrong() { } public void testLookupJoinUnknownIndex() { - assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); String errorMessage = "Unknown index [foobar]"; IndexResolution missingLookupIndex = IndexResolution.invalid(errorMessage); @@ -2168,7 +2168,7 @@ public void testLookupJoinUnknownIndex() { } public void testLookupJoinUnknownField() { - assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); String query = "FROM test | LOOKUP JOIN languages_lookup ON last_name"; String errorMessage = "1:45: Unknown column [last_name] in right side of join"; diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java index 58180aafedc0b..182e87d1ab9dd 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/analysis/VerifierTests.java @@ -1964,7 +1964,7 @@ public void testSortByAggregate() { } public void testLookupJoinDataTypeMismatch() { - assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("requires LOOKUP JOIN capability", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); query("FROM test | EVAL language_code = languages | LOOKUP JOIN languages_lookup ON language_code"); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index c4d7b30115c2d..cfb993a7dd73d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -4906,7 +4906,7 @@ public void testPlanSanityCheck() throws Exception { } public void testPlanSanityCheckWithBinaryPlans() throws Exception { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); var plan = optimizedPlan(""" FROM test @@ -5911,7 +5911,7 @@ public void testLookupStats() { * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19] */ public void testLookupJoinPushDownFilterOnJoinKeyWithRename() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); String query = """ FROM test @@ -5954,7 +5954,7 @@ public void testLookupJoinPushDownFilterOnJoinKeyWithRename() { * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19] */ public void testLookupJoinPushDownFilterOnLeftSideField() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); String query = """ FROM test @@ -5998,7 +5998,7 @@ public void testLookupJoinPushDownFilterOnLeftSideField() { * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#18, language_name{f}#19] */ public void testLookupJoinPushDownDisabledForLookupField() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); String query = """ FROM test @@ -6043,7 +6043,7 @@ public void testLookupJoinPushDownDisabledForLookupField() { * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#19, language_name{f}#20] */ public void testLookupJoinPushDownSeparatedForConjunctionBetweenLeftAndRightField() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); String query = """ FROM test @@ -6096,7 +6096,7 @@ public void testLookupJoinPushDownSeparatedForConjunctionBetweenLeftAndRightFiel * \_EsRelation[languages_lookup][LOOKUP][language_code{f}#19, language_name{f}#20] */ public void testLookupJoinPushDownDisabledForDisjunctionBetweenLeftAndRightField() { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); String query = """ FROM test diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java index 9f6ef89008a24..964dd4642d7c2 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/PhysicalPlanOptimizerTests.java @@ -2331,7 +2331,7 @@ public void testVerifierOnMissingReferences() { } public void testVerifierOnMissingReferencesWithBinaryPlans() throws Exception { - assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V6.isEnabled()); + assumeTrue("Requires LOOKUP JOIN", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); // Do not assert serialization: // This will have a LookupJoinExec, which is not serializable because it doesn't leave the coordinator. From 6516a535ab487eabf02b7d65b7cbe928820a181b Mon Sep 17 00:00:00 2001 From: Rene Groeschke Date: Tue, 17 Dec 2024 13:45:27 +0100 Subject: [PATCH 26/44] Add wolfi documentation from 8.16 branch (#118835) port from https://github.com/elastic/elasticsearch/pull/118684 --- docs/Versions.asciidoc | 1 + docs/reference/setup/install/docker.asciidoc | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/docs/Versions.asciidoc b/docs/Versions.asciidoc index bdb0704fcd880..f2e61861bd3a6 100644 --- a/docs/Versions.asciidoc +++ b/docs/Versions.asciidoc @@ -9,6 +9,7 @@ include::{docs-root}/shared/versions/stack/{source_branch}.asciidoc[] :docker-repo: docker.elastic.co/elasticsearch/elasticsearch :docker-image: {docker-repo}:{version} +:docker-wolfi-image: {docker-repo}-wolfi:{version} :kib-docker-repo: docker.elastic.co/kibana/kibana :kib-docker-image: {kib-docker-repo}:{version} :plugin_url: https://artifacts.elastic.co/downloads/elasticsearch-plugins diff --git a/docs/reference/setup/install/docker.asciidoc b/docs/reference/setup/install/docker.asciidoc index 8694d7f5b46c6..86a0e567f6eec 100644 --- a/docs/reference/setup/install/docker.asciidoc +++ b/docs/reference/setup/install/docker.asciidoc @@ -55,6 +55,12 @@ docker pull {docker-image} // REVIEWED[DEC.10.24] -- +Alternatevely, you can use the Wolfi based image. Using Wolfi based images requires Docker version 20.10.10 or superior. +[source,sh,subs="attributes"] +---- +docker pull {docker-wolfi-image} +---- + . Optional: Install https://docs.sigstore.dev/cosign/system_config/installation/[Cosign] for your environment. Then use Cosign to verify the {es} image's signature. From f5712e4875122d9f5c451ac73225725179795929 Mon Sep 17 00:00:00 2001 From: Simon Cooper Date: Tue, 17 Dec 2024 13:18:42 +0000 Subject: [PATCH 27/44] Infrastructure for assuming cluster features in the next major version (#118143) Allow features to be marked as 'assumed', allowing them to be removed in the next major version. --- .../forbidden/es-server-signatures.txt | 4 +- docs/changelog/118143.yaml | 5 + .../cluster/ClusterFeatures.java | 56 ++++- .../coordination/NodeJoinExecutor.java | 81 +++++- .../org/elasticsearch/env/BuildVersion.java | 6 + .../env/DefaultBuildVersion.java | 11 + .../features/FeatureService.java | 23 +- .../elasticsearch/features/NodeFeature.java | 9 +- .../readiness/ReadinessService.java | 4 +- .../DataStreamAutoShardingServiceTests.java | 29 ++- .../coordination/NodeJoinExecutorTests.java | 232 +++++++++++++++++- .../features/FeatureServiceTests.java | 41 ++++ .../HealthNodeTaskExecutorTests.java | 2 +- .../slm/SnapshotLifecycleServiceTests.java | 3 + 14 files changed, 464 insertions(+), 42 deletions(-) create mode 100644 docs/changelog/118143.yaml diff --git a/build-tools-internal/src/main/resources/forbidden/es-server-signatures.txt b/build-tools-internal/src/main/resources/forbidden/es-server-signatures.txt index a9da7995c2b36..53480a4a27b0b 100644 --- a/build-tools-internal/src/main/resources/forbidden/es-server-signatures.txt +++ b/build-tools-internal/src/main/resources/forbidden/es-server-signatures.txt @@ -155,10 +155,8 @@ org.elasticsearch.cluster.ClusterState#compatibilityVersions() @defaultMessage ClusterFeatures#nodeFeatures is for internal use only. Use FeatureService#clusterHasFeature to determine if a feature is present on the cluster. org.elasticsearch.cluster.ClusterFeatures#nodeFeatures() -@defaultMessage ClusterFeatures#allNodeFeatures is for internal use only. Use FeatureService#clusterHasFeature to determine if a feature is present on the cluster. -org.elasticsearch.cluster.ClusterFeatures#allNodeFeatures() @defaultMessage ClusterFeatures#clusterHasFeature is for internal use only. Use FeatureService#clusterHasFeature to determine if a feature is present on the cluster. -org.elasticsearch.cluster.ClusterFeatures#clusterHasFeature(org.elasticsearch.features.NodeFeature) +org.elasticsearch.cluster.ClusterFeatures#clusterHasFeature(org.elasticsearch.cluster.node.DiscoveryNodes, org.elasticsearch.features.NodeFeature) @defaultMessage Do not construct this records outside the source files they are declared in org.elasticsearch.cluster.SnapshotsInProgress$ShardSnapshotStatus#(java.lang.String, org.elasticsearch.cluster.SnapshotsInProgress$ShardState, org.elasticsearch.repositories.ShardGeneration, java.lang.String, org.elasticsearch.repositories.ShardSnapshotResult) diff --git a/docs/changelog/118143.yaml b/docs/changelog/118143.yaml new file mode 100644 index 0000000000000..4dcbf4b4b6c2c --- /dev/null +++ b/docs/changelog/118143.yaml @@ -0,0 +1,5 @@ +pr: 118143 +summary: Infrastructure for assuming cluster features in the next major version +area: "Infra/Core" +type: feature +issues: [] diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterFeatures.java b/server/src/main/java/org/elasticsearch/cluster/ClusterFeatures.java index ad285cbd391cd..5b5a6577082d7 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterFeatures.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterFeatures.java @@ -9,11 +9,12 @@ package org.elasticsearch.cluster; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.common.xcontent.ChunkedToXContentObject; -import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.features.NodeFeature; import org.elasticsearch.xcontent.ToXContent; @@ -79,28 +80,61 @@ public Map> nodeFeatures() { return nodeFeatures; } - /** - * The features in all nodes in the cluster. - *

- * NOTE: This should not be used directly. - * Please use {@link org.elasticsearch.features.FeatureService#clusterHasFeature} instead. - */ - public Set allNodeFeatures() { + private Set allNodeFeatures() { if (allNodeFeatures == null) { allNodeFeatures = Set.copyOf(calculateAllNodeFeatures(nodeFeatures.values())); } return allNodeFeatures; } + /** + * Returns {@code true} if {@code node} can have assumed features. + * @see org.elasticsearch.env.BuildVersion#canRemoveAssumedFeatures + */ + public static boolean featuresCanBeAssumedForNode(DiscoveryNode node) { + return node.getBuildVersion().canRemoveAssumedFeatures(); + } + + /** + * Returns {@code true} if one or more nodes in {@code nodes} can have assumed features. + * @see org.elasticsearch.env.BuildVersion#canRemoveAssumedFeatures + */ + public static boolean featuresCanBeAssumedForNodes(DiscoveryNodes nodes) { + return nodes.getAllNodes().stream().anyMatch(n -> n.getBuildVersion().canRemoveAssumedFeatures()); + } + /** * {@code true} if {@code feature} is present on all nodes in the cluster. *

* NOTE: This should not be used directly. * Please use {@link org.elasticsearch.features.FeatureService#clusterHasFeature} instead. */ - @SuppressForbidden(reason = "directly reading cluster features") - public boolean clusterHasFeature(NodeFeature feature) { - return allNodeFeatures().contains(feature.id()); + public boolean clusterHasFeature(DiscoveryNodes nodes, NodeFeature feature) { + assert nodes.getNodes().keySet().equals(nodeFeatures.keySet()) + : "Cluster features nodes " + nodeFeatures.keySet() + " is different to discovery nodes " + nodes.getNodes().keySet(); + + // basic case + boolean allNodesHaveFeature = allNodeFeatures().contains(feature.id()); + if (allNodesHaveFeature) { + return true; + } + + // if the feature is assumed, check the versions more closely + // it's actually ok if the feature is assumed, and all nodes missing the feature can assume it + // TODO: do we need some kind of transient cache of this calculation? + if (feature.assumedAfterNextCompatibilityBoundary()) { + for (var nf : nodeFeatures.entrySet()) { + if (nf.getValue().contains(feature.id()) == false + && featuresCanBeAssumedForNode(nodes.getNodes().get(nf.getKey())) == false) { + return false; + } + } + + // all nodes missing the feature can assume it - so that's alright then + return true; + } + + return false; } /** diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/NodeJoinExecutor.java b/server/src/main/java/org/elasticsearch/cluster/coordination/NodeJoinExecutor.java index 5235293a54d95..74a8dc7851c89 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/NodeJoinExecutor.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/NodeJoinExecutor.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; import org.elasticsearch.features.FeatureService; +import org.elasticsearch.features.NodeFeature; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.IndexVersions; import org.elasticsearch.persistent.PersistentTasksCustomMetadata; @@ -39,6 +40,7 @@ import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -137,8 +139,8 @@ public ClusterState execute(BatchExecutionContext batchExecutionContex DiscoveryNodes.Builder nodesBuilder = DiscoveryNodes.builder(newState.nodes()); Map compatibilityVersionsMap = new HashMap<>(newState.compatibilityVersions()); - Map> nodeFeatures = new HashMap<>(newState.nodeFeatures()); - Set allNodesFeatures = ClusterFeatures.calculateAllNodeFeatures(nodeFeatures.values()); + Map> nodeFeatures = new HashMap<>(newState.nodeFeatures()); // as present in cluster state + Set effectiveClusterFeatures = calculateEffectiveClusterFeatures(newState.nodes(), nodeFeatures); assert nodesBuilder.isLocalNodeElectedMaster(); @@ -174,14 +176,17 @@ public ClusterState execute(BatchExecutionContext batchExecutionContex } blockForbiddenVersions(compatibilityVersions.transportVersion()); ensureNodesCompatibility(node.getVersion(), minClusterNodeVersion, maxClusterNodeVersion); - enforceNodeFeatureBarrier(node.getId(), allNodesFeatures, features); + Set newNodeEffectiveFeatures = enforceNodeFeatureBarrier(node, effectiveClusterFeatures, features); // we do this validation quite late to prevent race conditions between nodes joining and importing dangling indices // we have to reject nodes that don't support all indices we have in this cluster ensureIndexCompatibility(node.getMinIndexVersion(), node.getMaxIndexVersion(), initialState.getMetadata()); + nodesBuilder.add(node); compatibilityVersionsMap.put(node.getId(), compatibilityVersions); + // store the actual node features here, not including assumed features, as this is persisted in cluster state nodeFeatures.put(node.getId(), features); - allNodesFeatures.retainAll(features); + effectiveClusterFeatures.retainAll(newNodeEffectiveFeatures); + nodesChanged = true; minClusterNodeVersion = Version.min(minClusterNodeVersion, node.getVersion()); maxClusterNodeVersion = Version.max(maxClusterNodeVersion, node.getVersion()); @@ -355,6 +360,35 @@ private static void blockForbiddenVersions(TransportVersion joiningTransportVers } } + /** + * Calculate the cluster's effective features. This includes all features that are assumed on any nodes in the cluster, + * that are also present across the whole cluster as a result. + */ + private Set calculateEffectiveClusterFeatures(DiscoveryNodes nodes, Map> nodeFeatures) { + if (featureService.featuresCanBeAssumedForNodes(nodes)) { + Set assumedFeatures = featureService.getNodeFeatures() + .values() + .stream() + .filter(NodeFeature::assumedAfterNextCompatibilityBoundary) + .map(NodeFeature::id) + .collect(Collectors.toSet()); + + // add all assumed features to the featureset of all nodes of the next major version + nodeFeatures = new HashMap<>(nodeFeatures); + for (var node : nodes.getNodes().entrySet()) { + if (featureService.featuresCanBeAssumedForNode(node.getValue())) { + assert nodeFeatures.containsKey(node.getKey()) : "Node " + node.getKey() + " does not have any features"; + nodeFeatures.computeIfPresent(node.getKey(), (k, v) -> { + var newFeatures = new HashSet<>(v); + return newFeatures.addAll(assumedFeatures) ? newFeatures : v; + }); + } + } + } + + return ClusterFeatures.calculateAllNodeFeatures(nodeFeatures.values()); + } + /** * Ensures that all indices are compatible with the given index version. This will ensure that all indices in the given metadata * will not be created with a newer version of elasticsearch as well as that all indices are newer or equal to the minimum index @@ -461,13 +495,44 @@ public static void ensureVersionBarrier(Version joiningNodeVersion, Version minC } } - private void enforceNodeFeatureBarrier(String nodeId, Set existingNodesFeatures, Set newNodeFeatures) { + /** + * Enforces the feature join barrier - a joining node should have all features already present in all existing nodes in the cluster + * + * @return The set of features that this node has (including assumed features) + */ + private Set enforceNodeFeatureBarrier(DiscoveryNode node, Set effectiveClusterFeatures, Set newNodeFeatures) { // prevent join if it does not have one or more features that all other nodes have - Set missingFeatures = new HashSet<>(existingNodesFeatures); + Set missingFeatures = new HashSet<>(effectiveClusterFeatures); missingFeatures.removeAll(newNodeFeatures); - if (missingFeatures.isEmpty() == false) { - throw new IllegalStateException("Node " + nodeId + " is missing required features " + missingFeatures); + if (missingFeatures.isEmpty()) { + // nothing missing - all ok + return newNodeFeatures; + } + + if (featureService.featuresCanBeAssumedForNode(node)) { + // it might still be ok for this node to join if this node can have assumed features, + // and all the missing features are assumed + // we can get the NodeFeature object direct from this node's registered features + // as all existing nodes in the cluster have the features present in existingNodesFeatures, including this one + newNodeFeatures = new HashSet<>(newNodeFeatures); + for (Iterator it = missingFeatures.iterator(); it.hasNext();) { + String feature = it.next(); + NodeFeature nf = featureService.getNodeFeatures().get(feature); + if (nf.assumedAfterNextCompatibilityBoundary()) { + // its ok for this feature to be missing from this node + it.remove(); + // and it should be assumed to still be in the cluster + newNodeFeatures.add(feature); + } + // even if we don't remove it, still continue, so the exception message below is accurate + } + } + + if (missingFeatures.isEmpty()) { + return newNodeFeatures; + } else { + throw new IllegalStateException("Node " + node.getId() + " is missing required features " + missingFeatures); } } diff --git a/server/src/main/java/org/elasticsearch/env/BuildVersion.java b/server/src/main/java/org/elasticsearch/env/BuildVersion.java index 7a6b27eab2330..5c3602283fef3 100644 --- a/server/src/main/java/org/elasticsearch/env/BuildVersion.java +++ b/server/src/main/java/org/elasticsearch/env/BuildVersion.java @@ -37,6 +37,12 @@ */ public abstract class BuildVersion implements ToXContentFragment, Writeable { + /** + * Checks if this version can operate properly in a cluster without features + * that are assumed in the currently running Elasticsearch. + */ + public abstract boolean canRemoveAssumedFeatures(); + /** * Check whether this version is on or after a minimum threshold. * diff --git a/server/src/main/java/org/elasticsearch/env/DefaultBuildVersion.java b/server/src/main/java/org/elasticsearch/env/DefaultBuildVersion.java index a7e1a4fee341d..70aa3f6639a4d 100644 --- a/server/src/main/java/org/elasticsearch/env/DefaultBuildVersion.java +++ b/server/src/main/java/org/elasticsearch/env/DefaultBuildVersion.java @@ -47,6 +47,17 @@ final class DefaultBuildVersion extends BuildVersion { this(in.readVInt()); } + @Override + public boolean canRemoveAssumedFeatures() { + /* + * We can remove assumed features if the node version is the next major version. + * This is because the next major version can only form a cluster with the + * latest minor version of the previous major, so any features introduced before that point + * (that are marked as assumed in the running code version) are automatically met by that version. + */ + return version.major == Version.CURRENT.major + 1; + } + @Override public boolean onOrAfterMinimumCompatible() { return Version.CURRENT.minimumCompatibilityVersion().onOrBefore(version); diff --git a/server/src/main/java/org/elasticsearch/features/FeatureService.java b/server/src/main/java/org/elasticsearch/features/FeatureService.java index 9a0ac7cafc183..c04fbae05ee2c 100644 --- a/server/src/main/java/org/elasticsearch/features/FeatureService.java +++ b/server/src/main/java/org/elasticsearch/features/FeatureService.java @@ -9,7 +9,10 @@ package org.elasticsearch.features; +import org.elasticsearch.cluster.ClusterFeatures; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.logging.LogManager; import org.elasticsearch.logging.Logger; @@ -38,9 +41,7 @@ public class FeatureService { * as the local node's supported feature set */ public FeatureService(List specs) { - - var featureData = FeatureData.createFromSpecifications(specs); - nodeFeatures = featureData.getNodeFeatures(); + this.nodeFeatures = FeatureData.createFromSpecifications(specs).getNodeFeatures(); logger.info("Registered local node features {}", nodeFeatures.keySet().stream().sorted().toList()); } @@ -53,11 +54,25 @@ public Map getNodeFeatures() { return nodeFeatures; } + /** + * Returns {@code true} if {@code node} can have assumed features. + */ + public boolean featuresCanBeAssumedForNode(DiscoveryNode node) { + return ClusterFeatures.featuresCanBeAssumedForNode(node); + } + + /** + * Returns {@code true} if one or more nodes in {@code nodes} can have assumed features. + */ + public boolean featuresCanBeAssumedForNodes(DiscoveryNodes nodes) { + return ClusterFeatures.featuresCanBeAssumedForNodes(nodes); + } + /** * Returns {@code true} if all nodes in {@code state} support feature {@code feature}. */ @SuppressForbidden(reason = "We need basic feature information from cluster state") public boolean clusterHasFeature(ClusterState state, NodeFeature feature) { - return state.clusterFeatures().clusterHasFeature(feature); + return state.clusterFeatures().clusterHasFeature(state.nodes(), feature); } } diff --git a/server/src/main/java/org/elasticsearch/features/NodeFeature.java b/server/src/main/java/org/elasticsearch/features/NodeFeature.java index 957308e805562..961b386d62802 100644 --- a/server/src/main/java/org/elasticsearch/features/NodeFeature.java +++ b/server/src/main/java/org/elasticsearch/features/NodeFeature.java @@ -15,10 +15,17 @@ * A feature published by a node. * * @param id The feature id. Must be unique in the node. + * @param assumedAfterNextCompatibilityBoundary + * {@code true} if this feature is removed at the next compatibility boundary (ie next major version), + * and so should be assumed to be true for all nodes after that boundary. */ -public record NodeFeature(String id) { +public record NodeFeature(String id, boolean assumedAfterNextCompatibilityBoundary) { public NodeFeature { Objects.requireNonNull(id); } + + public NodeFeature(String id) { + this(id, false); + } } diff --git a/server/src/main/java/org/elasticsearch/readiness/ReadinessService.java b/server/src/main/java/org/elasticsearch/readiness/ReadinessService.java index 15b9eacfa2118..de56ead9b5aba 100644 --- a/server/src/main/java/org/elasticsearch/readiness/ReadinessService.java +++ b/server/src/main/java/org/elasticsearch/readiness/ReadinessService.java @@ -294,8 +294,8 @@ protected boolean areFileSettingsApplied(ClusterState clusterState) { } @SuppressForbidden(reason = "need to check file settings support on exact cluster state") - private static boolean supportsFileSettings(ClusterState clusterState) { - return clusterState.clusterFeatures().clusterHasFeature(FileSettingsFeatures.FILE_SETTINGS_SUPPORTED); + private boolean supportsFileSettings(ClusterState clusterState) { + return clusterState.clusterFeatures().clusterHasFeature(clusterState.nodes(), FileSettingsFeatures.FILE_SETTINGS_SUPPORTED); } private void setReady(boolean ready) { diff --git a/server/src/test/java/org/elasticsearch/action/datastreams/autosharding/DataStreamAutoShardingServiceTests.java b/server/src/test/java/org/elasticsearch/action/datastreams/autosharding/DataStreamAutoShardingServiceTests.java index 2c6e273bb6e23..ba0f04d174f43 100644 --- a/server/src/test/java/org/elasticsearch/action/datastreams/autosharding/DataStreamAutoShardingServiceTests.java +++ b/server/src/test/java/org/elasticsearch/action/datastreams/autosharding/DataStreamAutoShardingServiceTests.java @@ -19,6 +19,8 @@ import org.elasticsearch.cluster.metadata.IndexMetadataStats; import org.elasticsearch.cluster.metadata.IndexWriteLoad; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.node.DiscoveryNodeUtils; +import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -110,6 +112,7 @@ public void testCalculateValidations() { ); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", @@ -143,8 +146,9 @@ public Set getFeatures() { // cluster doesn't have feature ClusterState stateNoFeature = ClusterState.builder(ClusterName.DEFAULT).metadata(Metadata.builder()).build(); + Settings settings = Settings.builder().put(DataStreamAutoShardingService.DATA_STREAMS_AUTO_SHARDING_ENABLED, true).build(); DataStreamAutoShardingService noFeatureService = new DataStreamAutoShardingService( - Settings.builder().put(DataStreamAutoShardingService.DATA_STREAMS_AUTO_SHARDING_ENABLED, true).build(), + settings, clusterService, new FeatureService(List.of()), () -> now @@ -155,15 +159,16 @@ public Set getFeatures() { } { + Settings settings = Settings.builder() + .put(DataStreamAutoShardingService.DATA_STREAMS_AUTO_SHARDING_ENABLED, true) + .putList( + DataStreamAutoShardingService.DATA_STREAMS_AUTO_SHARDING_EXCLUDES_SETTING.getKey(), + List.of("foo", dataStreamName + "*") + ) + .build(); // patterns are configured to exclude the current data stream DataStreamAutoShardingService noFeatureService = new DataStreamAutoShardingService( - Settings.builder() - .put(DataStreamAutoShardingService.DATA_STREAMS_AUTO_SHARDING_ENABLED, true) - .putList( - DataStreamAutoShardingService.DATA_STREAMS_AUTO_SHARDING_EXCLUDES_SETTING.getKey(), - List.of("foo", dataStreamName + "*") - ) - .build(), + settings, clusterService, new FeatureService(List.of()), () -> now @@ -199,6 +204,7 @@ public void testCalculateIncreaseShardingRecommendations() { DataStream dataStream = dataStreamSupplier.apply(null); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", @@ -237,6 +243,7 @@ public void testCalculateIncreaseShardingRecommendations() { ); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", @@ -275,6 +282,7 @@ public void testCalculateIncreaseShardingRecommendations() { ); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", @@ -313,6 +321,7 @@ public void testCalculateDecreaseShardingRecommendations() { DataStream dataStream = dataStreamSupplier.apply(null); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", @@ -353,6 +362,7 @@ public void testCalculateDecreaseShardingRecommendations() { DataStream dataStream = dataStreamSupplier.apply(null); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", @@ -401,6 +411,7 @@ public void testCalculateDecreaseShardingRecommendations() { ); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", @@ -447,6 +458,7 @@ public void testCalculateDecreaseShardingRecommendations() { ); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", @@ -487,6 +499,7 @@ public void testCalculateDecreaseShardingRecommendations() { DataStream dataStream = dataStreamSupplier.apply(null); builder.put(dataStream); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("n1")).add(DiscoveryNodeUtils.create("n2"))) .nodeFeatures( Map.of( "n1", diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinExecutorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinExecutorTests.java index 27775270a83eb..492a142492e18 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinExecutorTests.java @@ -33,6 +33,7 @@ import org.elasticsearch.common.ReferenceDocs; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.SuppressForbidden; import org.elasticsearch.features.FeatureService; import org.elasticsearch.features.FeatureSpecification; import org.elasticsearch.features.NodeFeature; @@ -46,11 +47,13 @@ import org.elasticsearch.threadpool.ThreadPool; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Stream; import static org.elasticsearch.cluster.metadata.DesiredNodesTestCase.assertDesiredNodesStatusIsCorrect; @@ -227,6 +230,227 @@ public Set getFeatures() { ); } + @SuppressForbidden(reason = "we need to actually check what is in cluster state") + private static Map> getRecordedNodeFeatures(ClusterState state) { + return state.clusterFeatures().nodeFeatures(); + } + + private static Version nextMajor() { + return Version.fromId((Version.CURRENT.major + 1) * 1_000_000 + 99); + } + + public void testCanJoinClusterWithAssumedFeatures() throws Exception { + AllocationService allocationService = createAllocationService(); + RerouteService rerouteService = (reason, priority, listener) -> listener.onResponse(null); + FeatureService featureService = new FeatureService(List.of(new FeatureSpecification() { + @Override + public Set getFeatures() { + return Set.of(new NodeFeature("f1"), new NodeFeature("af1", true), new NodeFeature("af2", true)); + } + })); + + NodeJoinExecutor executor = new NodeJoinExecutor(allocationService, rerouteService, featureService); + + DiscoveryNode masterNode = DiscoveryNodeUtils.create(UUIDs.base64UUID()); + DiscoveryNode otherNode = DiscoveryNodeUtils.create(UUIDs.base64UUID()); + Map> features = new HashMap<>(); + features.put(masterNode.getId(), Set.of("f1", "af1", "af2")); + features.put(otherNode.getId(), Set.of("f1", "af1", "af2")); + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(masterNode).localNodeId(masterNode.getId()).masterNodeId(masterNode.getId()).add(otherNode)) + .nodeFeatures(features) + .build(); + + // it is valid for major+1 versions to join clusters assumed features still present + // this can happen in the process of marking, then removing, assumed features + // they should still be recorded appropriately + DiscoveryNode newNode = DiscoveryNodeUtils.builder(UUIDs.base64UUID()) + .version(nextMajor(), IndexVersions.MINIMUM_COMPATIBLE, IndexVersion.current()) + .build(); + clusterState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( + clusterState, + executor, + List.of( + JoinTask.singleNode( + newNode, + CompatibilityVersionsUtils.staticCurrent(), + Set.of("f1", "af2"), + TEST_REASON, + NO_FAILURE_LISTENER, + 0L + ) + ) + ); + features.put(newNode.getId(), Set.of("f1", "af2")); + + // extra final check that the recorded cluster features are as they should be + assertThat(getRecordedNodeFeatures(clusterState), equalTo(features)); + } + + public void testJoinClusterWithAssumedFeaturesDoesntAllowNonAssumed() throws Exception { + AllocationService allocationService = createAllocationService(); + RerouteService rerouteService = (reason, priority, listener) -> listener.onResponse(null); + FeatureService featureService = new FeatureService(List.of(new FeatureSpecification() { + @Override + public Set getFeatures() { + return Set.of(new NodeFeature("f1"), new NodeFeature("af1", true)); + } + })); + + NodeJoinExecutor executor = new NodeJoinExecutor(allocationService, rerouteService, featureService); + + DiscoveryNode masterNode = DiscoveryNodeUtils.create(UUIDs.base64UUID()); + DiscoveryNode otherNode = DiscoveryNodeUtils.create(UUIDs.base64UUID()); + Map> features = new HashMap<>(); + features.put(masterNode.getId(), Set.of("f1", "af1")); + features.put(otherNode.getId(), Set.of("f1", "af1")); + + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(masterNode).localNodeId(masterNode.getId()).masterNodeId(masterNode.getId()).add(otherNode)) + .nodeFeatures(features) + .build(); + + DiscoveryNode newNodeNextMajor = DiscoveryNodeUtils.builder(UUIDs.base64UUID()) + .version(nextMajor(), IndexVersions.MINIMUM_COMPATIBLE, IndexVersion.current()) + .build(); + clusterState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( + clusterState, + executor, + List.of( + JoinTask.singleNode( + newNodeNextMajor, + CompatibilityVersionsUtils.staticCurrent(), + Set.of("f1"), + TEST_REASON, + NO_FAILURE_LISTENER, + 0L + ) + ) + ); + features.put(newNodeNextMajor.getId(), Set.of("f1")); + + // even though a next major has joined without af1, this doesnt allow the current major to join with af1 missing features + DiscoveryNode newNodeCurMajor = DiscoveryNodeUtils.create(UUIDs.base64UUID()); + AtomicReference ex = new AtomicReference<>(); + clusterState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( + clusterState, + executor, + List.of( + JoinTask.singleNode( + newNodeCurMajor, + CompatibilityVersionsUtils.staticCurrent(), + Set.of("f1"), + TEST_REASON, + ActionTestUtils.assertNoSuccessListener(ex::set), + 0L + ) + ) + ); + assertThat(ex.get().getMessage(), containsString("missing required features [af1]")); + + // a next major can't join missing non-assumed features + DiscoveryNode newNodeNextMajorMissing = DiscoveryNodeUtils.builder(UUIDs.base64UUID()) + .version(nextMajor(), IndexVersions.MINIMUM_COMPATIBLE, IndexVersion.current()) + .build(); + ex.set(null); + clusterState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( + clusterState, + executor, + List.of( + JoinTask.singleNode( + newNodeNextMajorMissing, + CompatibilityVersionsUtils.staticCurrent(), + Set.of(), + TEST_REASON, + ActionTestUtils.assertNoSuccessListener(ex::set), + 0L + ) + ) + ); + assertThat(ex.get().getMessage(), containsString("missing required features [f1]")); + + // extra final check that the recorded cluster features are as they should be, and newNodeNextMajor hasn't gained af1 + assertThat(getRecordedNodeFeatures(clusterState), equalTo(features)); + } + + /* + * Same as above but the current major missing features is processed in the same execution + */ + public void testJoinClusterWithAssumedFeaturesDoesntAllowNonAssumedSameExecute() throws Exception { + AllocationService allocationService = createAllocationService(); + RerouteService rerouteService = (reason, priority, listener) -> listener.onResponse(null); + FeatureService featureService = new FeatureService(List.of(new FeatureSpecification() { + @Override + public Set getFeatures() { + return Set.of(new NodeFeature("f1"), new NodeFeature("af1", true)); + } + })); + + NodeJoinExecutor executor = new NodeJoinExecutor(allocationService, rerouteService, featureService); + + DiscoveryNode masterNode = DiscoveryNodeUtils.create(UUIDs.base64UUID()); + DiscoveryNode otherNode = DiscoveryNodeUtils.create(UUIDs.base64UUID()); + Map> features = new HashMap<>(); + features.put(masterNode.getId(), Set.of("f1", "af1")); + features.put(otherNode.getId(), Set.of("f1", "af1")); + + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(masterNode).localNodeId(masterNode.getId()).masterNodeId(masterNode.getId()).add(otherNode)) + .nodeFeatures(features) + .build(); + + DiscoveryNode newNodeNextMajor = DiscoveryNodeUtils.builder(UUIDs.base64UUID()) + .version(nextMajor(), IndexVersions.MINIMUM_COMPATIBLE, IndexVersion.current()) + .build(); + DiscoveryNode newNodeCurMajor = DiscoveryNodeUtils.create(UUIDs.base64UUID()); + DiscoveryNode newNodeNextMajorMissing = DiscoveryNodeUtils.builder(UUIDs.base64UUID()) + .version(nextMajor(), IndexVersions.MINIMUM_COMPATIBLE, IndexVersion.current()) + .build(); + // even though a next major could join, this doesnt allow the current major to join with missing features + // nor a next major missing non-assumed features + AtomicReference thisMajorEx = new AtomicReference<>(); + AtomicReference nextMajorEx = new AtomicReference<>(); + List tasks = List.of( + JoinTask.singleNode( + newNodeNextMajor, + CompatibilityVersionsUtils.staticCurrent(), + Set.of("f1"), + TEST_REASON, + NO_FAILURE_LISTENER, + 0L + ), + JoinTask.singleNode( + newNodeCurMajor, + CompatibilityVersionsUtils.staticCurrent(), + Set.of("f1"), + TEST_REASON, + ActionTestUtils.assertNoSuccessListener(thisMajorEx::set), + 0L + ), + JoinTask.singleNode( + newNodeNextMajorMissing, + CompatibilityVersionsUtils.staticCurrent(), + Set.of(), + TEST_REASON, + ActionTestUtils.assertNoSuccessListener(nextMajorEx::set), + 0L + ) + ); + if (randomBoolean()) { + // sometimes combine them together into a single task for completeness + tasks = List.of(new JoinTask(tasks.stream().flatMap(t -> t.nodeJoinTasks().stream()).toList(), false, 0L, null)); + } + + clusterState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful(clusterState, executor, tasks); + features.put(newNodeNextMajor.getId(), Set.of("f1")); + + assertThat(thisMajorEx.get().getMessage(), containsString("missing required features [af1]")); + assertThat(nextMajorEx.get().getMessage(), containsString("missing required features [f1]")); + + // extra check that the recorded cluster features are as they should be, and newNodeNextMajor hasn't gained af1 + assertThat(getRecordedNodeFeatures(clusterState), equalTo(features)); + } + public void testSuccess() { Settings.builder().build(); Metadata.Builder metaBuilder = Metadata.builder(); @@ -921,8 +1145,8 @@ public void testSetsNodeFeaturesWhenRejoining() throws Exception { .nodeFeatures(Map.of(masterNode.getId(), Set.of("f1", "f2"), rejoinNode.getId(), Set.of())) .build(); - assertThat(clusterState.clusterFeatures().clusterHasFeature(new NodeFeature("f1")), is(false)); - assertThat(clusterState.clusterFeatures().clusterHasFeature(new NodeFeature("f2")), is(false)); + assertThat(clusterState.clusterFeatures().clusterHasFeature(clusterState.nodes(), new NodeFeature("f1")), is(false)); + assertThat(clusterState.clusterFeatures().clusterHasFeature(clusterState.nodes(), new NodeFeature("f2")), is(false)); final var resultingState = ClusterStateTaskExecutorUtils.executeAndAssertSuccessful( clusterState, @@ -939,8 +1163,8 @@ public void testSetsNodeFeaturesWhenRejoining() throws Exception { ) ); - assertThat(resultingState.clusterFeatures().clusterHasFeature(new NodeFeature("f1")), is(true)); - assertThat(resultingState.clusterFeatures().clusterHasFeature(new NodeFeature("f2")), is(true)); + assertThat(resultingState.clusterFeatures().clusterHasFeature(resultingState.nodes(), new NodeFeature("f1")), is(true)); + assertThat(resultingState.clusterFeatures().clusterHasFeature(resultingState.nodes(), new NodeFeature("f2")), is(true)); } private DesiredNodeWithStatus createActualizedDesiredNode() { diff --git a/server/src/test/java/org/elasticsearch/features/FeatureServiceTests.java b/server/src/test/java/org/elasticsearch/features/FeatureServiceTests.java index 874a6a96313e4..a64303f376b20 100644 --- a/server/src/test/java/org/elasticsearch/features/FeatureServiceTests.java +++ b/server/src/test/java/org/elasticsearch/features/FeatureServiceTests.java @@ -9,8 +9,14 @@ package org.elasticsearch.features; +import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNodeUtils; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.node.VersionInformation; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.IndexVersions; import org.elasticsearch.test.ESTestCase; import java.util.List; @@ -69,6 +75,12 @@ public void testStateHasFeatures() { ); ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes( + DiscoveryNodes.builder() + .add(DiscoveryNodeUtils.create("node1")) + .add(DiscoveryNodeUtils.create("node2")) + .add(DiscoveryNodeUtils.create("node3")) + ) .nodeFeatures( Map.of("node1", Set.of("f1", "f2", "nf1"), "node2", Set.of("f1", "f2", "nf2"), "node3", Set.of("f1", "f2", "nf1")) ) @@ -81,4 +93,33 @@ public void testStateHasFeatures() { assertFalse(service.clusterHasFeature(state, new NodeFeature("nf2"))); assertFalse(service.clusterHasFeature(state, new NodeFeature("nf3"))); } + + private static Version nextMajor() { + return Version.fromId((Version.CURRENT.major + 1) * 1_000_000 + 99); + } + + public void testStateHasAssumedFeatures() { + List specs = List.of( + new TestFeatureSpecification(Set.of(new NodeFeature("f1"), new NodeFeature("f2"), new NodeFeature("af1", true))) + ); + + ClusterState state = ClusterState.builder(ClusterName.DEFAULT) + .nodes( + DiscoveryNodes.builder() + .add(DiscoveryNodeUtils.create("node1")) + .add(DiscoveryNodeUtils.create("node2")) + .add( + DiscoveryNodeUtils.builder("node3") + .version(new VersionInformation(nextMajor(), IndexVersions.MINIMUM_COMPATIBLE, IndexVersion.current())) + .build() + ) + ) + .nodeFeatures(Map.of("node1", Set.of("f1", "af1"), "node2", Set.of("f1", "f2", "af1"), "node3", Set.of("f1", "f2"))) + .build(); + + FeatureService service = new FeatureService(specs); + assertTrue(service.clusterHasFeature(state, new NodeFeature("f1"))); + assertFalse(service.clusterHasFeature(state, new NodeFeature("f2"))); + assertTrue(service.clusterHasFeature(state, new NodeFeature("af1", true))); + } } diff --git a/server/src/test/java/org/elasticsearch/health/node/selection/HealthNodeTaskExecutorTests.java b/server/src/test/java/org/elasticsearch/health/node/selection/HealthNodeTaskExecutorTests.java index 97f44f7480a72..92bfabf6f1972 100644 --- a/server/src/test/java/org/elasticsearch/health/node/selection/HealthNodeTaskExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/health/node/selection/HealthNodeTaskExecutorTests.java @@ -77,8 +77,8 @@ public void setUp() throws Exception { clusterService = createClusterService(threadPool); localNodeId = clusterService.localNode().getId(); persistentTasksService = mock(PersistentTasksService.class); - featureService = new FeatureService(List.of(new HealthFeatures())); settings = Settings.builder().build(); + featureService = new FeatureService(List.of(new HealthFeatures())); clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); } diff --git a/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java b/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java index 36887681f5575..9955fe4cf0f95 100644 --- a/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java +++ b/x-pack/plugin/slm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleServiceTests.java @@ -529,6 +529,7 @@ public void testValidateIntervalScheduleSupport() { var featureService = new FeatureService(List.of(new SnapshotLifecycleFeatures())); { ClusterState state = ClusterState.builder(new ClusterName("cluster")) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("a")).add(DiscoveryNodeUtils.create("b"))) .nodeFeatures(Map.of("a", Set.of(), "b", Set.of(SnapshotLifecycleService.INTERVAL_SCHEDULE.id()))) .build(); @@ -540,6 +541,7 @@ public void testValidateIntervalScheduleSupport() { } { ClusterState state = ClusterState.builder(new ClusterName("cluster")) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("a"))) .nodeFeatures(Map.of("a", Set.of(SnapshotLifecycleService.INTERVAL_SCHEDULE.id()))) .build(); try { @@ -550,6 +552,7 @@ public void testValidateIntervalScheduleSupport() { } { ClusterState state = ClusterState.builder(new ClusterName("cluster")) + .nodes(DiscoveryNodes.builder().add(DiscoveryNodeUtils.create("a")).add(DiscoveryNodeUtils.create("b"))) .nodeFeatures(Map.of("a", Set.of(), "b", Set.of(SnapshotLifecycleService.INTERVAL_SCHEDULE.id()))) .build(); try { From 312c21a3240339477c70fb512b5643b23952d572 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Wed, 18 Dec 2024 00:58:11 +1100 Subject: [PATCH 28/44] Mute org.elasticsearch.index.engine.RecoverySourcePruneMergePolicyTests testPruneSome #118728 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index fe6c77bdf9f93..cdf3007ee0027 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -304,6 +304,9 @@ tests: issue: https://github.com/elastic/elasticsearch/issues/118806 - class: org.elasticsearch.xpack.esql.session.IndexResolverFieldNamesTests issue: https://github.com/elastic/elasticsearch/issues/118814 +- class: org.elasticsearch.index.engine.RecoverySourcePruneMergePolicyTests + method: testPruneSome + issue: https://github.com/elastic/elasticsearch/issues/118728 # Examples: # From e0763c25ae9600611fe93b8d4133b5106ff280fd Mon Sep 17 00:00:00 2001 From: Andrei Stefan Date: Tue, 17 Dec 2024 17:16:40 +0200 Subject: [PATCH 29/44] Mark the lookup join tests in IndexResolverFieldNamesTests as snapshot-only (#118815) --- muted-tests.yml | 2 -- .../esql/session/IndexResolverFieldNamesTests.java | 13 +++++++++++++ .../qa/server/src/main/resources/docs/docs.csv-spec | 2 +- 3 files changed, 14 insertions(+), 3 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index cdf3007ee0027..42845fda82180 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -302,8 +302,6 @@ tests: - class: org.elasticsearch.xpack.security.QueryableReservedRolesIT method: testDeletingAndCreatingSecurityIndexTriggersSynchronization issue: https://github.com/elastic/elasticsearch/issues/118806 -- class: org.elasticsearch.xpack.esql.session.IndexResolverFieldNamesTests - issue: https://github.com/elastic/elasticsearch/issues/118814 - class: org.elasticsearch.index.engine.RecoverySourcePruneMergePolicyTests method: testPruneSome issue: https://github.com/elastic/elasticsearch/issues/118728 diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java index e4271a0a6ddd5..31ec4663738f7 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/session/IndexResolverFieldNamesTests.java @@ -9,6 +9,7 @@ import org.elasticsearch.Build; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.esql.action.EsqlCapabilities; import org.elasticsearch.xpack.esql.parser.EsqlParser; import org.elasticsearch.xpack.esql.parser.ParsingException; @@ -1364,6 +1365,7 @@ public void testMetrics() { } public void testLookupJoin() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( "FROM employees | KEEP languages | RENAME languages AS language_code | LOOKUP JOIN languages_lookup ON language_code", Set.of("languages", "languages.*", "language_code", "language_code.*"), @@ -1372,6 +1374,7 @@ public void testLookupJoin() { } public void testLookupJoinKeep() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM employees @@ -1385,6 +1388,7 @@ public void testLookupJoinKeep() { } public void testLookupJoinKeepWildcard() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM employees @@ -1398,6 +1402,7 @@ public void testLookupJoinKeepWildcard() { } public void testMultiLookupJoin() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data @@ -1410,6 +1415,7 @@ public void testMultiLookupJoin() { } public void testMultiLookupJoinKeepBefore() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data @@ -1423,6 +1429,7 @@ public void testMultiLookupJoinKeepBefore() { } public void testMultiLookupJoinKeepBetween() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data @@ -1447,6 +1454,7 @@ public void testMultiLookupJoinKeepBetween() { } public void testMultiLookupJoinKeepAfter() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data @@ -1473,6 +1481,7 @@ public void testMultiLookupJoinKeepAfter() { } public void testMultiLookupJoinKeepAfterWildcard() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data @@ -1486,6 +1495,7 @@ public void testMultiLookupJoinKeepAfterWildcard() { } public void testMultiLookupJoinSameIndex() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data @@ -1499,6 +1509,7 @@ public void testMultiLookupJoinSameIndex() { } public void testMultiLookupJoinSameIndexKeepBefore() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data @@ -1513,6 +1524,7 @@ public void testMultiLookupJoinSameIndexKeepBefore() { } public void testMultiLookupJoinSameIndexKeepBetween() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data @@ -1538,6 +1550,7 @@ public void testMultiLookupJoinSameIndexKeepBetween() { } public void testMultiLookupJoinSameIndexKeepAfter() { + assumeTrue("LOOKUP JOIN available as snapshot only", EsqlCapabilities.Cap.JOIN_LOOKUP_V7.isEnabled()); assertFieldNames( """ FROM sample_data diff --git a/x-pack/plugin/sql/qa/server/src/main/resources/docs/docs.csv-spec b/x-pack/plugin/sql/qa/server/src/main/resources/docs/docs.csv-spec index 60e81be43cc96..2fa82c05cc1aa 100644 --- a/x-pack/plugin/sql/qa/server/src/main/resources/docs/docs.csv-spec +++ b/x-pack/plugin/sql/qa/server/src/main/resources/docs/docs.csv-spec @@ -3353,7 +3353,7 @@ Alejandro Amabile Anoosh Basil -Bojan +Brendon // end::filterToday ; From 6d943e9e53576700b8417164d0e4c899c5f84e52 Mon Sep 17 00:00:00 2001 From: Valeriy Khakhutskyy <1292899+valeriy42@users.noreply.github.com> Date: Tue, 17 Dec 2024 16:33:27 +0100 Subject: [PATCH 30/44] [ML] Ignore failures from renormalizing buckets in read-only index (#118674) In anomaly detection, score renormalization will update the anomaly score in the result indices. However, if the index in the old format was marked as read-only, the score update will fail. Since this failure is expected, this PR suppresses the error logging in this specific case. --- docs/changelog/118674.yaml | 5 ++++ .../JobRenormalizedResultsPersister.java | 26 ++++++++++++++++++- 2 files changed, 30 insertions(+), 1 deletion(-) create mode 100644 docs/changelog/118674.yaml diff --git a/docs/changelog/118674.yaml b/docs/changelog/118674.yaml new file mode 100644 index 0000000000000..eeb90a3b38f66 --- /dev/null +++ b/docs/changelog/118674.yaml @@ -0,0 +1,5 @@ +pr: 118674 +summary: Ignore failures from renormalizing buckets in read-only index +area: Machine Learning +type: enhancement +issues: [] diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobRenormalizedResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobRenormalizedResultsPersister.java index 3c0d2aca4deda..3c82841f1b99e 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobRenormalizedResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobRenormalizedResultsPersister.java @@ -8,10 +8,12 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentBuilder; @@ -102,7 +104,29 @@ public void executeRequest() { try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { BulkResponse addRecordsResponse = client.bulk(bulkRequest).actionGet(); if (addRecordsResponse.hasFailures()) { - logger.error("[{}] Bulk index of results has errors: {}", jobId, addRecordsResponse.buildFailureMessage()); + // Implementation note: Ignore the failures from writing to the read-only index, as it comes + // from changing the index format version. + boolean hasNonReadOnlyFailures = false; + for (BulkItemResponse response : addRecordsResponse.getItems()) { + if (response.isFailed() == false) { + continue; + } + if (response.getFailureMessage().contains(IndexMetadata.INDEX_READ_ONLY_BLOCK.description())) { + // We expect this to happen when the old index is made read-only and being reindexed + logger.debug( + "[{}] Ignoring failure to write renormalized results to a read-only index [{}]: {}", + jobId, + response.getFailure().getIndex(), + response.getFailureMessage() + ); + } else { + hasNonReadOnlyFailures = true; + break; + } + } + if (hasNonReadOnlyFailures) { + logger.error("[{}] Bulk index of results has errors: {}", jobId, addRecordsResponse.buildFailureMessage()); + } } } From f64c05ac32ee23930cf913c7939c9eeac12f00ce Mon Sep 17 00:00:00 2001 From: Adam Szaraniec Date: Tue, 17 Dec 2024 20:09:22 +0400 Subject: [PATCH 31/44] Update alias.asciidoc (#118553) Add section about removing index --- docs/reference/alias.asciidoc | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/docs/reference/alias.asciidoc b/docs/reference/alias.asciidoc index 9d784f530d63c..f676644c4ec48 100644 --- a/docs/reference/alias.asciidoc +++ b/docs/reference/alias.asciidoc @@ -407,3 +407,24 @@ POST _aliases } ---- // TEST[s/^/PUT my-index-2099.05.06-000001\n/] + +[discrete] +[[remove-index]] +=== Remove an index + +To remove an index, use the aliases API's `remove_index` action. + +[source,console] +---- +POST _aliases +{ + "actions": [ + { + "remove_index": { + "index": "my-index-2099.05.06-000001" + } + } + ] +} +---- +// TEST[s/^/PUT my-index-2099.05.06-000001\n/] From 1d2840ece1b8a5fcc1e6836d70bb1187160f2cd2 Mon Sep 17 00:00:00 2001 From: Luigi Dell'Aquila Date: Tue, 17 Dec 2024 17:13:52 +0100 Subject: [PATCH 32/44] EQL: add support for partial search results (#116388) Allow queries to succeed if some shards are failing --- docs/changelog/116388.yaml | 5 + docs/reference/eql/eql-search-api.asciidoc | 47 ++ .../rest-api-spec/api/eql.search.json | 10 + .../org/elasticsearch/TransportVersions.java | 1 + .../test/eql/BaseEqlSpecTestCase.java | 84 +- .../elasticsearch/test/eql/DataLoader.java | 6 + .../test/eql/EqlDateNanosSpecTestCase.java | 36 +- .../test/eql/EqlExtraSpecTestCase.java | 36 +- .../eql/EqlMissingEventsSpecTestCase.java | 36 +- .../eql/EqlSampleMultipleEntriesTestCase.java | 36 +- .../test/eql/EqlSampleTestCase.java | 43 +- .../org/elasticsearch/test/eql/EqlSpec.java | 51 +- .../eql/EqlSpecFailingShardsTestCase.java | 83 ++ .../elasticsearch/test/eql/EqlSpecLoader.java | 7 + .../test/eql/EqlSpecTestCase.java | 43 +- .../data/endgame-shard-failures.data | 14 + .../data/endgame-shard-failures.mapping | 105 +++ .../main/resources/test_failing_shards.toml | 173 ++++ .../xpack/eql/qa/mixed_node/EqlSearchIT.java | 11 +- .../xpack/eql/EqlDateNanosIT.java | 25 +- .../elasticsearch/xpack/eql/EqlExtraIT.java | 25 +- .../elasticsearch/xpack/eql/EqlSampleIT.java | 25 +- .../xpack/eql/EqlSampleMultipleEntriesIT.java | 18 +- .../elasticsearch/xpack/eql/EqlSpecIT.java | 25 +- .../xpack/eql/EqlDateNanosIT.java | 24 +- .../elasticsearch/xpack/eql/EqlExtraIT.java | 24 +- .../xpack/eql/EqlMissingEventsIT.java | 24 +- .../elasticsearch/xpack/eql/EqlSampleIT.java | 24 +- .../xpack/eql/EqlSampleMultipleEntriesIT.java | 17 +- .../xpack/eql/EqlSpecFailingShardsIT.java | 53 ++ .../elasticsearch/xpack/eql/EqlSpecIT.java | 24 +- .../rest-api-spec/test/eql/10_basic.yml | 143 ++++ .../xpack/eql/action/CCSPartialResultsIT.java | 613 ++++++++++++++ .../eql/action/PartialSearchResultsIT.java | 780 ++++++++++++++++++ .../xpack/eql/action/EqlSearchRequest.java | 47 +- .../xpack/eql/action/EqlSearchResponse.java | 42 +- .../xpack/eql/action/EqlSearchTask.java | 4 +- .../execution/assembler/ExecutionManager.java | 7 +- .../execution/payload/AbstractPayload.java | 10 +- .../eql/execution/payload/EventPayload.java | 2 +- .../eql/execution/sample/SampleIterator.java | 29 +- .../eql/execution/sample/SamplePayload.java | 11 +- .../execution/search/BasicQueryClient.java | 18 +- .../execution/search/PITAwareQueryClient.java | 25 +- .../eql/execution/search/RuntimeUtils.java | 31 +- .../execution/sequence/SequencePayload.java | 11 +- .../execution/sequence/TumblingWindow.java | 43 +- .../xpack/eql/plugin/EqlPlugin.java | 16 +- .../xpack/eql/plugin/RestEqlSearchAction.java | 6 + .../eql/plugin/TransportEqlSearchAction.java | 36 +- .../xpack/eql/session/EmptyPayload.java | 13 +- .../xpack/eql/session/EqlConfiguration.java | 14 + .../xpack/eql/session/Payload.java | 3 + .../xpack/eql/session/Results.java | 19 +- .../xpack/eql/util/SearchHitUtils.java | 12 + .../elasticsearch/xpack/eql/EqlTestUtils.java | 4 + .../eql/action/EqlSearchRequestTests.java | 8 + .../eql/action/EqlSearchResponseTests.java | 14 +- .../eql/action/LocalStateEQLXPackPlugin.java | 21 +- .../assembler/ImplicitTiebreakerTests.java | 10 +- .../assembler/SequenceSpecTests.java | 10 +- .../execution/sample/CircuitBreakerTests.java | 5 +- .../search/PITAwareQueryClientTests.java | 12 +- .../sequence/CircuitBreakerTests.java | 32 +- .../execution/sequence/PITFailureTests.java | 12 +- 65 files changed, 3068 insertions(+), 130 deletions(-) create mode 100644 docs/changelog/116388.yaml create mode 100644 x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecFailingShardsTestCase.java create mode 100644 x-pack/plugin/eql/qa/common/src/main/resources/data/endgame-shard-failures.data create mode 100644 x-pack/plugin/eql/qa/common/src/main/resources/data/endgame-shard-failures.mapping create mode 100644 x-pack/plugin/eql/qa/common/src/main/resources/test_failing_shards.toml create mode 100644 x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecFailingShardsIT.java create mode 100644 x-pack/plugin/eql/src/internalClusterTest/java/org/elasticsearch/xpack/eql/action/CCSPartialResultsIT.java create mode 100644 x-pack/plugin/eql/src/internalClusterTest/java/org/elasticsearch/xpack/eql/action/PartialSearchResultsIT.java diff --git a/docs/changelog/116388.yaml b/docs/changelog/116388.yaml new file mode 100644 index 0000000000000..59cdafc9ec337 --- /dev/null +++ b/docs/changelog/116388.yaml @@ -0,0 +1,5 @@ +pr: 116388 +summary: Add support for partial shard results +area: EQL +type: enhancement +issues: [] diff --git a/docs/reference/eql/eql-search-api.asciidoc b/docs/reference/eql/eql-search-api.asciidoc index d7f10f4627f6c..0fd490609277f 100644 --- a/docs/reference/eql/eql-search-api.asciidoc +++ b/docs/reference/eql/eql-search-api.asciidoc @@ -88,6 +88,53 @@ request that targets only `bar*` still returns an error. + Defaults to `true`. +`allow_partial_search_results`:: +(Optional, Boolean) + +If `false`, the request returns an error if one or more shards involved in the query are unavailable. ++ +If `true`, the query is executed only on the available shards, ignoring shard request timeouts and +<>. ++ +Defaults to `false`. ++ +To override the default for this field, set the +`xpack.eql.default_allow_partial_results` cluster setting to `true`. + + +[IMPORTANT] +==== +You can also specify this value using the `allow_partial_search_results` request body parameter. +If both parameters are specified, only the query parameter is used. +==== + + +`allow_partial_sequence_results`:: +(Optional, Boolean) + + +Used together with `allow_partial_search_results=true`, controls the behavior of sequence queries specifically +(if `allow_partial_search_results=false`, this setting has no effect). +If `true` and if some shards are unavailable, the sequences are calculated on available shards only. ++ +If `false` and if some shards are unavailable, the query only returns information about the shard failures, +but no further results. ++ +Defaults to `false`. ++ +Consider that sequences calculated with `allow_partial_search_results=true` can return incorrect results +(eg. if a <> clause matches records in unavailable shards) ++ +To override the default for this field, set the +`xpack.eql.default_allow_partial_sequence_results` cluster setting to `true`. + + +[IMPORTANT] +==== +You can also specify this value using the `allow_partial_sequence_results` request body parameter. +If both parameters are specified, only the query parameter is used. +==== + `ccs_minimize_roundtrips`:: (Optional, Boolean) If `true`, network round-trips between the local and the remote cluster are minimized when running cross-cluster search (CCS) requests. diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/eql.search.json b/rest-api-spec/src/main/resources/rest-api-spec/api/eql.search.json index c854c44d9d761..0f9af508f4c16 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/eql.search.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/eql.search.json @@ -41,6 +41,16 @@ "type": "time", "description": "Update the time interval in which the results (partial or final) for this search will be available", "default": "5d" + }, + "allow_partial_search_results": { + "type":"boolean", + "description":"Control whether the query should keep running in case of shard failures, and return partial results", + "default":false + }, + "allow_partial_sequence_results": { + "type":"boolean", + "description":"Control whether a sequence query should return partial results or no results at all in case of shard failures. This option has effect only if [allow_partial_search_results] is true.", + "default":false } }, "body":{ diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index f5e581a81a37c..371af961720cc 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -138,6 +138,7 @@ static TransportVersion def(int id) { public static final TransportVersion KNN_QUERY_RESCORE_OVERSAMPLE = def(8_806_00_0); public static final TransportVersion SEMANTIC_QUERY_LENIENT = def(8_807_00_0); public static final TransportVersion ESQL_QUERY_BUILDER_IN_SEARCH_FUNCTIONS = def(8_808_00_0); + public static final TransportVersion EQL_ALLOW_PARTIAL_SEARCH_RESULTS = def(8_809_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/BaseEqlSpecTestCase.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/BaseEqlSpecTestCase.java index 90244d9b2c019..3557114e2f4c7 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/BaseEqlSpecTestCase.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/BaseEqlSpecTestCase.java @@ -33,6 +33,9 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; + public abstract class BaseEqlSpecTestCase extends RemoteClusterAwareEqlRestTestCase { protected static final String PARAM_FORMATTING = "%2$s"; @@ -52,6 +55,9 @@ public abstract class BaseEqlSpecTestCase extends RemoteClusterAwareEqlRestTestC */ private final int size; private final int maxSamplesPerKey; + private final Boolean allowPartialSearchResults; + private final Boolean allowPartialSequenceResults; + private final Boolean expectShardFailures; @Before public void setup() throws Exception { @@ -104,7 +110,16 @@ protected static List asArray(List specs) { } results.add( - new Object[] { spec.query(), name, spec.expectedEventIds(), spec.joinKeys(), spec.size(), spec.maxSamplesPerKey() } + new Object[] { + spec.query(), + name, + spec.expectedEventIds(), + spec.joinKeys(), + spec.size(), + spec.maxSamplesPerKey(), + spec.allowPartialSearchResults(), + spec.allowPartialSequenceResults(), + spec.expectShardFailures() } ); } @@ -118,7 +133,10 @@ protected static List asArray(List specs) { List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { this.index = index; @@ -128,6 +146,9 @@ protected static List asArray(List specs) { this.joinKeys = joinKeys; this.size = size == null ? -1 : size; this.maxSamplesPerKey = maxSamplesPerKey == null ? -1 : maxSamplesPerKey; + this.allowPartialSearchResults = allowPartialSearchResults; + this.allowPartialSequenceResults = allowPartialSequenceResults; + this.expectShardFailures = expectShardFailures; } public void test() throws Exception { @@ -137,6 +158,7 @@ public void test() throws Exception { private void assertResponse(ObjectPath response) throws Exception { List> events = response.evaluate("hits.events"); List> sequences = response.evaluate("hits.sequences"); + Object shardFailures = response.evaluate("shard_failures"); if (events != null) { assertEvents(events); @@ -145,6 +167,7 @@ private void assertResponse(ObjectPath response) throws Exception { } else { fail("No events or sequences found"); } + assertShardFailures(shardFailures); } protected ObjectPath runQuery(String index, String query) throws Exception { @@ -163,6 +186,32 @@ protected ObjectPath runQuery(String index, String query) throws Exception { if (maxSamplesPerKey > 0) { builder.field("max_samples_per_key", maxSamplesPerKey); } + boolean allowPartialResultsInBody = randomBoolean(); + if (allowPartialSearchResults != null) { + if (allowPartialResultsInBody) { + builder.field("allow_partial_search_results", String.valueOf(allowPartialSearchResults)); + if (allowPartialSequenceResults != null) { + builder.field("allow_partial_sequence_results", String.valueOf(allowPartialSequenceResults)); + } + } else { + // these will be overwritten by the path params, that have higher priority than the query (JSON body) params + if (allowPartialSearchResults != null) { + builder.field("allow_partial_search_results", randomBoolean()); + } + if (allowPartialSequenceResults != null) { + builder.field("allow_partial_sequence_results", randomBoolean()); + } + } + } else { + // Tests that don't specify a setting for these parameters should always pass. + // These params should be irrelevant. + if (randomBoolean()) { + builder.field("allow_partial_search_results", randomBoolean()); + } + if (randomBoolean()) { + builder.field("allow_partial_sequence_results", randomBoolean()); + } + } builder.endObject(); Request request = new Request("POST", "/" + index + "/_eql/search"); @@ -170,6 +219,23 @@ protected ObjectPath runQuery(String index, String query) throws Exception { if (ccsMinimizeRoundtrips != null) { request.addParameter("ccs_minimize_roundtrips", ccsMinimizeRoundtrips.toString()); } + if (allowPartialSearchResults != null) { + if (allowPartialResultsInBody == false) { + request.addParameter("allow_partial_search_results", String.valueOf(allowPartialSearchResults)); + if (allowPartialSequenceResults != null) { + request.addParameter("allow_partial_sequence_results", String.valueOf(allowPartialSequenceResults)); + } + } + } else { + // Tests that don't specify a setting for these parameters should always pass. + // These params should be irrelevant. + if (randomBoolean()) { + request.addParameter("allow_partial_search_results", String.valueOf(randomBoolean())); + } + if (randomBoolean()) { + request.addParameter("allow_partial_sequence_results", String.valueOf(randomBoolean())); + } + } int timeout = Math.toIntExact(timeout().millis()); RequestConfig config = RequestConfig.copy(RequestConfig.DEFAULT) .setConnectionRequestTimeout(timeout) @@ -182,6 +248,20 @@ protected ObjectPath runQuery(String index, String query) throws Exception { return ObjectPath.createFromResponse(client().performRequest(request)); } + private void assertShardFailures(Object shardFailures) { + if (expectShardFailures != null) { + if (expectShardFailures) { + assertNotNull(shardFailures); + List list = (List) shardFailures; + assertThat(list.size(), is(greaterThan(0))); + } else { + assertNull(shardFailures); + } + } else { + assertNull(shardFailures); + } + } + private void assertEvents(List> events) { assertNotNull(events); logger.debug("Events {}", new Object() { diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/DataLoader.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/DataLoader.java index 1d51af574c810..4618bd8f4ff3d 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/DataLoader.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/DataLoader.java @@ -52,6 +52,7 @@ */ public class DataLoader { public static final String TEST_INDEX = "endgame-140"; + public static final String TEST_SHARD_FAILURES_INDEX = "endgame-shard-failures"; public static final String TEST_EXTRA_INDEX = "extra"; public static final String TEST_NANOS_INDEX = "endgame-140-nanos"; public static final String TEST_SAMPLE = "sample1,sample2,sample3"; @@ -103,6 +104,11 @@ public static void loadDatasetIntoEs(RestClient client, CheckedBiFunction eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - this(TEST_NANOS_INDEX, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + this( + TEST_NANOS_INDEX, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } // constructor for multi-cluster tests @@ -40,9 +54,23 @@ public EqlDateNanosSpecTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - super(index, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + super( + index, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } @Override diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlExtraSpecTestCase.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlExtraSpecTestCase.java index 292fe6c895cee..cc858ded25f37 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlExtraSpecTestCase.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlExtraSpecTestCase.java @@ -27,9 +27,23 @@ public EqlExtraSpecTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - this(TEST_EXTRA_INDEX, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + this( + TEST_EXTRA_INDEX, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } // constructor for multi-cluster tests @@ -40,9 +54,23 @@ public EqlExtraSpecTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - super(index, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + super( + index, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } @Override diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlMissingEventsSpecTestCase.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlMissingEventsSpecTestCase.java index cdda9e9e068f5..f62c2b29101db 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlMissingEventsSpecTestCase.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlMissingEventsSpecTestCase.java @@ -27,9 +27,23 @@ public EqlMissingEventsSpecTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - this(TEST_MISSING_EVENTS_INDEX, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + this( + TEST_MISSING_EVENTS_INDEX, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } // constructor for multi-cluster tests @@ -40,9 +54,23 @@ public EqlMissingEventsSpecTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - super(index, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + super( + index, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } @Override diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSampleMultipleEntriesTestCase.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSampleMultipleEntriesTestCase.java index 6471e264a92fa..a38ccacb42f5f 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSampleMultipleEntriesTestCase.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSampleMultipleEntriesTestCase.java @@ -21,9 +21,23 @@ public EqlSampleMultipleEntriesTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - this(TEST_SAMPLE_MULTI, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + this( + TEST_SAMPLE_MULTI, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } public EqlSampleMultipleEntriesTestCase( @@ -33,9 +47,23 @@ public EqlSampleMultipleEntriesTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - super(index, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + super( + index, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } @ParametersFactory(shuffle = false, argumentFormatting = PARAM_FORMATTING) diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSampleTestCase.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSampleTestCase.java index dfae73b3602a7..4748bd0e3307b 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSampleTestCase.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSampleTestCase.java @@ -15,8 +15,29 @@ public abstract class EqlSampleTestCase extends BaseEqlSpecTestCase { - public EqlSampleTestCase(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - this(TEST_SAMPLE, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlSampleTestCase( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + this( + TEST_SAMPLE, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } public EqlSampleTestCase( @@ -26,9 +47,23 @@ public EqlSampleTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - super(index, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + super( + index, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } @ParametersFactory(shuffle = false, argumentFormatting = PARAM_FORMATTING) diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpec.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpec.java index db7ee05ff2239..4dd617bac0abd 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpec.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpec.java @@ -30,6 +30,9 @@ public class EqlSpec { private Integer size; private Integer maxSamplesPerKey; + private Boolean allowPartialSearchResults; + private Boolean allowPartialSequenceResults; + private Boolean expectShardFailures; public String name() { return name; @@ -103,6 +106,30 @@ public void maxSamplesPerKey(Integer maxSamplesPerKey) { this.maxSamplesPerKey = maxSamplesPerKey; } + public Boolean allowPartialSearchResults() { + return allowPartialSearchResults; + } + + public void allowPartialSearchResults(Boolean allowPartialSearchResults) { + this.allowPartialSearchResults = allowPartialSearchResults; + } + + public Boolean allowPartialSequenceResults() { + return allowPartialSequenceResults; + } + + public void allowPartialSequenceResults(Boolean allowPartialSequenceResults) { + this.allowPartialSequenceResults = allowPartialSequenceResults; + } + + public Boolean expectShardFailures() { + return expectShardFailures; + } + + public void expectShardFailures(Boolean expectShardFailures) { + this.expectShardFailures = expectShardFailures; + } + @Override public String toString() { String str = ""; @@ -132,7 +159,15 @@ public String toString() { if (maxSamplesPerKey != null) { str = appendWithComma(str, "max_samples_per_key", "" + maxSamplesPerKey); } - + if (allowPartialSearchResults != null) { + str = appendWithComma(str, "allow_partial_search_results", String.valueOf(allowPartialSearchResults)); + } + if (allowPartialSequenceResults != null) { + str = appendWithComma(str, "allow_partial_sequence_results", String.valueOf(allowPartialSequenceResults)); + } + if (expectShardFailures != null) { + str = appendWithComma(str, "expect_shard_failures", String.valueOf(expectShardFailures)); + } return str; } @@ -150,12 +185,22 @@ public boolean equals(Object other) { return Objects.equals(this.query(), that.query()) && Objects.equals(size, that.size) - && Objects.equals(maxSamplesPerKey, that.maxSamplesPerKey); + && Objects.equals(maxSamplesPerKey, that.maxSamplesPerKey) + && Objects.equals(allowPartialSearchResults, that.allowPartialSearchResults) + && Objects.equals(allowPartialSequenceResults, that.allowPartialSequenceResults) + && Objects.equals(expectShardFailures, that.expectShardFailures); } @Override public int hashCode() { - return Objects.hash(this.query, size, maxSamplesPerKey); + return Objects.hash( + this.query, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } private static String appendWithComma(String str, String name, String append) { diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecFailingShardsTestCase.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecFailingShardsTestCase.java new file mode 100644 index 0000000000000..c490a2f703dcc --- /dev/null +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecFailingShardsTestCase.java @@ -0,0 +1,83 @@ +/* + * 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.test.eql; + +import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; + +import java.util.List; + +import static org.elasticsearch.test.eql.DataLoader.TEST_INDEX; +import static org.elasticsearch.test.eql.DataLoader.TEST_SHARD_FAILURES_INDEX; + +public abstract class EqlSpecFailingShardsTestCase extends BaseEqlSpecTestCase { + + @ParametersFactory(shuffle = false, argumentFormatting = PARAM_FORMATTING) + public static List readTestSpecs() throws Exception { + + // Load EQL validation specs + return asArray(EqlSpecLoader.load("/test_failing_shards.toml")); + } + + @Override + protected String tiebreaker() { + return "serial_event_id"; + } + + // constructor for "local" rest tests + public EqlSpecFailingShardsTestCase( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + this( + TEST_INDEX + "," + TEST_SHARD_FAILURES_INDEX, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); + } + + // constructor for multi-cluster tests + public EqlSpecFailingShardsTestCase( + String index, + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearch, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + index, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearch, + allowPartialSequenceResults, + expectShardFailures + ); + } +} diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecLoader.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecLoader.java index a1f555563e29c..f86107cf3bac5 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecLoader.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecLoader.java @@ -76,6 +76,10 @@ private static Integer getInteger(TomlTable table, String key) { return null; } + private static Boolean getBoolean(TomlTable table, String key) { + return table.getBoolean(key); + } + private static List readFromStream(InputStream is, Set uniqueTestNames) throws Exception { List testSpecs = new ArrayList<>(); @@ -90,6 +94,9 @@ private static List readFromStream(InputStream is, Set uniqueTe spec.note(getTrimmedString(table, "note")); spec.description(getTrimmedString(table, "description")); spec.size(getInteger(table, "size")); + spec.allowPartialSearchResults(getBoolean(table, "allow_partial_search_results")); + spec.allowPartialSequenceResults(getBoolean(table, "allow_partial_sequence_results")); + spec.expectShardFailures(getBoolean(table, "expect_shard_failures")); List arr = table.getList("tags"); if (arr != null) { diff --git a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecTestCase.java b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecTestCase.java index 7113924f79029..62a3ea72fe51f 100644 --- a/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecTestCase.java +++ b/x-pack/plugin/eql/qa/common/src/main/java/org/elasticsearch/test/eql/EqlSpecTestCase.java @@ -28,8 +28,29 @@ protected String tiebreaker() { } // constructor for "local" rest tests - public EqlSpecTestCase(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - this(TEST_INDEX, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlSpecTestCase( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearch, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + this( + TEST_INDEX, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearch, + allowPartialSequenceResults, + expectShardFailures + ); } // constructor for multi-cluster tests @@ -40,8 +61,22 @@ public EqlSpecTestCase( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearch, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - super(index, query, name, eventIds, joinKeys, size, maxSamplesPerKey); + super( + index, + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearch, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/common/src/main/resources/data/endgame-shard-failures.data b/x-pack/plugin/eql/qa/common/src/main/resources/data/endgame-shard-failures.data new file mode 100644 index 0000000000000..18a1d05656d09 --- /dev/null +++ b/x-pack/plugin/eql/qa/common/src/main/resources/data/endgame-shard-failures.data @@ -0,0 +1,14 @@ +[ + { + "event_subtype_full": "already_running", + "event_type": "process", + "event_type_full": "process_event", + "opcode": 3, + "pid": 0, + "process_name": "System Idle Process", + "serial_event_id": 10000, + "subtype": "create", + "timestamp": 117444736000000000, + "unique_pid": 1 + } +] diff --git a/x-pack/plugin/eql/qa/common/src/main/resources/data/endgame-shard-failures.mapping b/x-pack/plugin/eql/qa/common/src/main/resources/data/endgame-shard-failures.mapping new file mode 100644 index 0000000000000..3b5039f4098af --- /dev/null +++ b/x-pack/plugin/eql/qa/common/src/main/resources/data/endgame-shard-failures.mapping @@ -0,0 +1,105 @@ +# Text patterns like "[runtime_random_keyword_type]" will get replaced at runtime with a random string type. +# See DataLoader class for pattern replacements. +{ + "runtime":{ + "broken":{ + "type": "long", + "script": { + "lang": "painless", + "source": "emit(doc['non_existing'].value.dayOfWeekEnum.getDisplayName(TextStyle.FULL, Locale.ENGLISH))" + } + } + }, + "properties" : { + "command_line" : { + "type" : "[runtime_random_keyword_type]" + }, + "event_type" : { + "type" : "[runtime_random_keyword_type]" + }, + "event" : { + "properties" : { + "category" : { + "type" : "alias", + "path" : "event_type" + }, + "sequence" : { + "type" : "alias", + "path" : "serial_event_id" + } + } + }, + "md5" : { + "type" : "[runtime_random_keyword_type]" + }, + "parent_process_name": { + "type" : "[runtime_random_keyword_type]" + }, + "parent_process_path": { + "type" : "[runtime_random_keyword_type]" + }, + "pid" : { + "type" : "long" + }, + "ppid" : { + "type" : "long" + }, + "process_name": { + "type" : "[runtime_random_keyword_type]" + }, + "process_path": { + "type" : "[runtime_random_keyword_type]" + }, + "subtype" : { + "type" : "[runtime_random_keyword_type]" + }, + "timestamp" : { + "type" : "date" + }, + "@timestamp" : { + "type" : "date" + }, + "user" : { + "type" : "[runtime_random_keyword_type]" + }, + "user_name" : { + "type" : "[runtime_random_keyword_type]" + }, + "user_domain": { + "type" : "[runtime_random_keyword_type]" + }, + "hostname" : { + "type" : "text", + "fields" : { + "[runtime_random_keyword_type]" : { + "type" : "[runtime_random_keyword_type]", + "ignore_above" : 256 + } + } + }, + "opcode" : { + "type" : "long" + }, + "file_name" : { + "type" : "text", + "fields" : { + "[runtime_random_keyword_type]" : { + "type" : "[runtime_random_keyword_type]", + "ignore_above" : 256 + } + } + }, + "file_path" : { + "type" : "[runtime_random_keyword_type]" + }, + "serial_event_id" : { + "type" : "long" + }, + "source_address" : { + "type" : "ip" + }, + "exit_code" : { + "type" : "long" + } + } +} diff --git a/x-pack/plugin/eql/qa/common/src/main/resources/test_failing_shards.toml b/x-pack/plugin/eql/qa/common/src/main/resources/test_failing_shards.toml new file mode 100644 index 0000000000000..a551c66fd48bd --- /dev/null +++ b/x-pack/plugin/eql/qa/common/src/main/resources/test_failing_shards.toml @@ -0,0 +1,173 @@ +# this query doesn't touch the "broken" field, so it should not fail +[[queries]] +name = "eventQueryNoShardFailures" +query = 'process where serial_event_id == 1' +allow_partial_search_results = true +expected_event_ids = [1] +expect_shard_failures = false + + +[[queries]] +name = "eventQueryShardFailures" +query = 'process where serial_event_id == 1 or broken == 1' +allow_partial_search_results = true +expected_event_ids = [1] +expect_shard_failures = true + + +[[queries]] +name = "eventQueryShardFailuresOptionalField" +query = 'process where serial_event_id == 1 and ?optional_field_default_null == null or broken == 1' +allow_partial_search_results = true +expected_event_ids = [1] +expect_shard_failures = true + + +[[queries]] +name = "eventQueryShardFailuresOptionalFieldMatching" +query = 'process where serial_event_id == 2 and ?subtype == "create" or broken == 1' +allow_partial_search_results = true +expected_event_ids = [2] +expect_shard_failures = true + + +# this query doesn't touch the "broken" field, so it should not fail +[[queries]] +name = "sequenceQueryNoShardFailures" +query = ''' +sequence + [process where serial_event_id == 1] + [process where serial_event_id == 2] +''' +expected_event_ids = [1, 2] +expect_shard_failures = false + + +# this query doesn't touch the "broken" field, so it should not fail +[[queries]] +name = "sequenceQueryNoShardFailuresAllowFalse" +query = ''' +sequence + [process where serial_event_id == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = false +expected_event_ids = [1, 2] +expect_shard_failures = false + + +# this query doesn't touch the "broken" field, so it should not fail +[[queries]] +name = "sequenceQueryNoShardFailuresAllowTrue" +query = ''' +sequence + [process where serial_event_id == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +expected_event_ids = [1, 2] +expect_shard_failures = false + + +[[queries]] +name = "sequenceQueryMissingShards" +query = ''' +sequence + [process where serial_event_id == 1 or broken == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +expected_event_ids = [] +expect_shard_failures = true + + +[[queries]] +name = "sequenceQueryMissingShardsPartialResults" +query = ''' +sequence + [process where serial_event_id == 1 or broken == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +allow_partial_sequence_results = true +expected_event_ids = [1, 2] +expect_shard_failures = true + + +[[queries]] +name = "sequenceQueryMissingShardsPartialResultsOptional" +query = ''' +sequence + [process where ?serial_event_id == 1 or broken == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +allow_partial_sequence_results = true +expected_event_ids = [1, 2] +expect_shard_failures = true + + +[[queries]] +name = "sequenceQueryMissingShardsPartialResultsOptional2" +query = ''' +sequence with maxspan=100000d + [process where serial_event_id == 1 and ?subtype == "create" or broken == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +allow_partial_sequence_results = true +expected_event_ids = [1, 2] +expect_shard_failures = true + + +[[queries]] +name = "sequenceQueryMissingShardsPartialResultsOptionalMissing" +query = ''' +sequence with maxspan=100000d + [process where serial_event_id == 1 and ?subtype == "create"] + ![process where broken == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +allow_partial_sequence_results = true +expected_event_ids = [1, -1, 2] +expect_shard_failures = true + + +[[queries]] +name = "sequenceQueryMissingShardsPartialResultsOptionalMissing2" +query = ''' +sequence with maxspan=100000d + [process where serial_event_id == 1 and ?subtype == "create" or broken == 1] + ![process where broken == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +allow_partial_sequence_results = true +expected_event_ids = [1, -1, 2] +expect_shard_failures = true + + +[[queries]] +name = "sampleQueryMissingShardsPartialResults" +query = ''' +sample by event_subtype_full + [process where serial_event_id == 1 or broken == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +expected_event_ids = [1, 2] +expect_shard_failures = true + + +[[queries]] +name = "sampleQueryMissingShardsPartialResultsOptional" +query = ''' +sample by event_subtype_full + [process where serial_event_id == 1 and ?subtype == "create" or broken == 1] + [process where serial_event_id == 2] +''' +allow_partial_search_results = true +expected_event_ids = [1, 2] +expect_shard_failures = true + diff --git a/x-pack/plugin/eql/qa/mixed-node/src/javaRestTest/java/org/elasticsearch/xpack/eql/qa/mixed_node/EqlSearchIT.java b/x-pack/plugin/eql/qa/mixed-node/src/javaRestTest/java/org/elasticsearch/xpack/eql/qa/mixed_node/EqlSearchIT.java index 2a29572374fa8..60c7fb1c7ad25 100644 --- a/x-pack/plugin/eql/qa/mixed-node/src/javaRestTest/java/org/elasticsearch/xpack/eql/qa/mixed_node/EqlSearchIT.java +++ b/x-pack/plugin/eql/qa/mixed-node/src/javaRestTest/java/org/elasticsearch/xpack/eql/qa/mixed_node/EqlSearchIT.java @@ -407,7 +407,16 @@ private void assertMultiValueFunctionQuery( for (int id : ids) { eventIds.add(String.valueOf(id)); } - request.setJsonEntity("{\"query\":\"" + query + "\"}"); + + StringBuilder payload = new StringBuilder("{\"query\":\"" + query + "\""); + if (randomBoolean()) { + payload.append(", \"allow_partial_search_results\": true"); + } + if (randomBoolean()) { + payload.append(", \"allow_partial_sequence_results\": true"); + } + payload.append("}"); + request.setJsonEntity(payload.toString()); assertResponse(query, eventIds, runEql(client, request)); testedFunctions.add(functionName); } diff --git a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlDateNanosIT.java b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlDateNanosIT.java index c20968871472f..5d6824232d80f 100644 --- a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlDateNanosIT.java +++ b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlDateNanosIT.java @@ -37,7 +37,28 @@ protected String getRemoteCluster() { return REMOTE_CLUSTER.getHttpAddresses(); } - public EqlDateNanosIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(remoteClusterIndex(TEST_NANOS_INDEX), query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlDateNanosIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + remoteClusterIndex(TEST_NANOS_INDEX), + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlExtraIT.java b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlExtraIT.java index 774c19d02adf0..79b095434814b 100644 --- a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlExtraIT.java +++ b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlExtraIT.java @@ -37,7 +37,28 @@ protected String getRemoteCluster() { return REMOTE_CLUSTER.getHttpAddresses(); } - public EqlExtraIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(remoteClusterIndex(TEST_EXTRA_INDEX), query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlExtraIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + remoteClusterIndex(TEST_EXTRA_INDEX), + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleIT.java b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleIT.java index 1502c250bd058..7673eec32ec55 100644 --- a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleIT.java +++ b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleIT.java @@ -37,7 +37,28 @@ protected String getRemoteCluster() { return REMOTE_CLUSTER.getHttpAddresses(); } - public EqlSampleIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(remoteClusterPattern(TEST_SAMPLE), query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlSampleIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + remoteClusterPattern(TEST_SAMPLE), + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleMultipleEntriesIT.java b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleMultipleEntriesIT.java index 795fe4e103a31..ac6f7fe508c99 100644 --- a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleMultipleEntriesIT.java +++ b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleMultipleEntriesIT.java @@ -43,8 +43,22 @@ public EqlSampleMultipleEntriesIT( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - super(remoteClusterPattern(TEST_SAMPLE_MULTI), query, name, eventIds, joinKeys, size, maxSamplesPerKey); + super( + remoteClusterPattern(TEST_SAMPLE_MULTI), + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecIT.java b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecIT.java index 2cddecb644a1a..db0c03e8fdb6f 100644 --- a/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecIT.java +++ b/x-pack/plugin/eql/qa/multi-cluster-with-security/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecIT.java @@ -37,7 +37,28 @@ protected String getRemoteCluster() { return REMOTE_CLUSTER.getHttpAddresses(); } - public EqlSpecIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(remoteClusterIndex(TEST_INDEX), query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlSpecIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + remoteClusterIndex(TEST_INDEX), + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlDateNanosIT.java b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlDateNanosIT.java index 1df10fde7fde5..5e1fa224de58d 100644 --- a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlDateNanosIT.java +++ b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlDateNanosIT.java @@ -27,7 +27,27 @@ protected String getTestRestCluster() { return cluster.getHttpAddresses(); } - public EqlDateNanosIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlDateNanosIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlExtraIT.java b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlExtraIT.java index 8af8fcac087b5..cb92eddeb0410 100644 --- a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlExtraIT.java +++ b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlExtraIT.java @@ -27,7 +27,27 @@ protected String getTestRestCluster() { return cluster.getHttpAddresses(); } - public EqlExtraIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlExtraIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlMissingEventsIT.java b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlMissingEventsIT.java index 05557fb4883b3..4f1faf3322e7f 100644 --- a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlMissingEventsIT.java +++ b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlMissingEventsIT.java @@ -27,8 +27,28 @@ protected String getTestRestCluster() { return cluster.getHttpAddresses(); } - public EqlMissingEventsIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlMissingEventsIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleIT.java b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleIT.java index dc2c653fad89e..c0bce3ffc9e4f 100644 --- a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleIT.java +++ b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleIT.java @@ -27,8 +27,28 @@ protected String getTestRestCluster() { return cluster.getHttpAddresses(); } - public EqlSampleIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlSampleIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleMultipleEntriesIT.java b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleMultipleEntriesIT.java index af1ade9120bbd..f50ee36095ae0 100644 --- a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleMultipleEntriesIT.java +++ b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSampleMultipleEntriesIT.java @@ -33,9 +33,22 @@ public EqlSampleMultipleEntriesIT( List eventIds, String[] joinKeys, Integer size, - Integer maxSamplesPerKey + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures ) { - super(query, name, eventIds, joinKeys, size, maxSamplesPerKey); + super( + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecFailingShardsIT.java b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecFailingShardsIT.java new file mode 100644 index 0000000000000..cf05811a77857 --- /dev/null +++ b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecFailingShardsIT.java @@ -0,0 +1,53 @@ +/* + * 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.eql; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; + +import org.elasticsearch.test.TestClustersThreadFilter; +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.eql.EqlSpecFailingShardsTestCase; +import org.junit.ClassRule; + +import java.util.List; + +@ThreadLeakFilters(filters = TestClustersThreadFilter.class) +public class EqlSpecFailingShardsIT extends EqlSpecFailingShardsTestCase { + + @ClassRule + public static final ElasticsearchCluster cluster = EqlTestCluster.CLUSTER; + + @Override + protected String getTestRestCluster() { + return cluster.getHttpAddresses(); + } + + public EqlSpecFailingShardsIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); + } +} diff --git a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecIT.java b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecIT.java index 7aac0ae336c8a..0aad5cc1b73da 100644 --- a/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecIT.java +++ b/x-pack/plugin/eql/qa/rest/src/javaRestTest/java/org/elasticsearch/xpack/eql/EqlSpecIT.java @@ -27,7 +27,27 @@ protected String getTestRestCluster() { return cluster.getHttpAddresses(); } - public EqlSpecIT(String query, String name, List eventIds, String[] joinKeys, Integer size, Integer maxSamplesPerKey) { - super(query, name, eventIds, joinKeys, size, maxSamplesPerKey); + public EqlSpecIT( + String query, + String name, + List eventIds, + String[] joinKeys, + Integer size, + Integer maxSamplesPerKey, + Boolean allowPartialSearchResults, + Boolean allowPartialSequenceResults, + Boolean expectShardFailures + ) { + super( + query, + name, + eventIds, + joinKeys, + size, + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults, + expectShardFailures + ); } } diff --git a/x-pack/plugin/eql/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/eql/10_basic.yml b/x-pack/plugin/eql/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/eql/10_basic.yml index e49264d76d5e9..c7974f3b584b4 100644 --- a/x-pack/plugin/eql/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/eql/10_basic.yml +++ b/x-pack/plugin/eql/qa/rest/src/yamlRestTest/resources/rest-api-spec/test/eql/10_basic.yml @@ -83,6 +83,34 @@ setup: id: 123 valid: true + - do: + indices.create: + index: eql_test_rebel + body: + mappings: + properties: + some_keyword: + type: keyword + runtime: + day_of_week: + type: keyword + script: + source: "throw new IllegalArgumentException(\"rebel shards\")" + - do: + bulk: + refresh: true + body: + - index: + _index: eql_test_rebel + _id: "1" + - event: + - category: process + "@timestamp": 2020-02-03T12:34:56Z + user: SYSTEM + id: 123 + valid: false + some_keyword: longer than normal + --- # Testing round-trip and the basic shape of the response "Execute some EQL.": @@ -478,3 +506,118 @@ setup: query: 'sequence with maxspan=10d [network where user == "ADMIN"] ![network where used == "SYSTEM"]' - match: { error.root_cause.0.type: "verification_exception" } - match: { error.root_cause.0.reason: "Found 1 problem\nline 1:75: Unknown column [used], did you mean [user]?" } + + +--- +"Execute query shard failures and with allow_partial_search_results": + - do: + eql.search: + index: eql_test* + body: + query: 'process where user == "SYSTEM" and day_of_week == "Monday"' + fields: [{"field":"@timestamp","format":"epoch_millis"},"id","valid","day_of_week"] + allow_partial_search_results: true + + - match: {timed_out: false} + - match: {hits.total.value: 1} + - match: {hits.total.relation: "eq"} + - match: {hits.events.0._source.user: "SYSTEM"} + - match: {hits.events.0._id: "1"} + - match: {hits.events.0.fields.@timestamp: ["1580733296000"]} + - match: {hits.events.0.fields.id: [123]} + - match: {hits.events.0.fields.valid: [false]} + - match: {hits.events.0.fields.day_of_week: ["Monday"]} + - match: {shard_failures.0.index: "eql_test_rebel"} + + +--- +"Execute query shard failures and with allow_partial_search_results as request param": + - do: + eql.search: + index: eql_test* + allow_partial_search_results: true + body: + query: 'process where user == "SYSTEM" and day_of_week == "Monday"' + fields: [{"field":"@timestamp","format":"epoch_millis"},"id","valid","day_of_week"] + + - match: {timed_out: false} + - match: {hits.total.value: 1} + - match: {hits.total.relation: "eq"} + - match: {hits.events.0._source.user: "SYSTEM"} + - match: {hits.events.0._id: "1"} + - match: {hits.events.0.fields.@timestamp: ["1580733296000"]} + - match: {hits.events.0.fields.id: [123]} + - match: {hits.events.0.fields.valid: [false]} + - match: {hits.events.0.fields.day_of_week: ["Monday"]} + - match: {shard_failures.0.index: "eql_test_rebel"} + + +--- +"Execute sequence with shard failures and allow_partial_search_results=true": + - do: + eql.search: + index: eql_test* + body: + query: 'sequence [process where user == "SYSTEM" and day_of_week == "Monday"] [process where user == "SYSTEM" and day_of_week == "Tuesday"]' + fields: [{"field":"@timestamp","format":"epoch_millis"},"id","valid","day_of_week"] + allow_partial_search_results: true + + - match: {timed_out: false} + - match: {hits.total.value: 0} + - match: {shard_failures.0.index: "eql_test_rebel"} + + +--- +"Execute sequence with shard failures, allow_partial_search_results=true and allow_partial_sequence_results=true": + - do: + eql.search: + index: eql_test* + body: + query: 'sequence [process where user == "SYSTEM" and day_of_week == "Monday"] [process where user == "SYSTEM" and day_of_week == "Tuesday"]' + fields: [{"field":"@timestamp","format":"epoch_millis"},"id","valid","day_of_week"] + allow_partial_search_results: true + allow_partial_sequence_results: true + + - match: {timed_out: false} + - match: {hits.total.value: 1} + - match: {hits.total.relation: "eq"} + - match: {hits.sequences.0.events.0._source.user: "SYSTEM"} + - match: {hits.sequences.0.events.0._id: "1"} + - match: {hits.sequences.0.events.0.fields.@timestamp: ["1580733296000"]} + - match: {hits.sequences.0.events.0.fields.id: [123]} + - match: {hits.sequences.0.events.0.fields.valid: [false]} + - match: {hits.sequences.0.events.0.fields.day_of_week: ["Monday"]} + - match: {hits.sequences.0.events.1._id: "2"} + - match: {hits.sequences.0.events.1.fields.@timestamp: ["1580819696000"]} + - match: {hits.sequences.0.events.1.fields.id: [123]} + - match: {hits.sequences.0.events.1.fields.valid: [true]} + - match: {hits.sequences.0.events.1.fields.day_of_week: ["Tuesday"]} + - match: {shard_failures.0.index: "eql_test_rebel"} + + +--- +"Execute sequence with shard failures, allow_partial_search_results=true and allow_partial_sequence_results=true as query params": + - do: + eql.search: + index: eql_test* + allow_partial_search_results: true + allow_partial_sequence_results: true + body: + query: 'sequence [process where user == "SYSTEM" and day_of_week == "Monday"] [process where user == "SYSTEM" and day_of_week == "Tuesday"]' + fields: [{"field":"@timestamp","format":"epoch_millis"},"id","valid","day_of_week"] + + - match: {timed_out: false} + - match: {hits.total.value: 1} + - match: {hits.total.relation: "eq"} + - match: {hits.sequences.0.events.0._source.user: "SYSTEM"} + - match: {hits.sequences.0.events.0._id: "1"} + - match: {hits.sequences.0.events.0.fields.@timestamp: ["1580733296000"]} + - match: {hits.sequences.0.events.0.fields.id: [123]} + - match: {hits.sequences.0.events.0.fields.valid: [false]} + - match: {hits.sequences.0.events.0.fields.day_of_week: ["Monday"]} + - match: {hits.sequences.0.events.1._id: "2"} + - match: {hits.sequences.0.events.1.fields.@timestamp: ["1580819696000"]} + - match: {hits.sequences.0.events.1.fields.id: [123]} + - match: {hits.sequences.0.events.1.fields.valid: [true]} + - match: {hits.sequences.0.events.1.fields.day_of_week: ["Tuesday"]} + - match: {shard_failures.0.index: "eql_test_rebel"} diff --git a/x-pack/plugin/eql/src/internalClusterTest/java/org/elasticsearch/xpack/eql/action/CCSPartialResultsIT.java b/x-pack/plugin/eql/src/internalClusterTest/java/org/elasticsearch/xpack/eql/action/CCSPartialResultsIT.java new file mode 100644 index 0000000000000..da6bb6180428b --- /dev/null +++ b/x-pack/plugin/eql/src/internalClusterTest/java/org/elasticsearch/xpack/eql/action/CCSPartialResultsIT.java @@ -0,0 +1,613 @@ +/* + * 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.eql.action; + +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsAction; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.AbstractMultiClustersTestCase; +import org.elasticsearch.xpack.eql.plugin.EqlPlugin; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutionException; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class CCSPartialResultsIT extends AbstractMultiClustersTestCase { + + static String REMOTE_CLUSTER = "cluster_a"; + + protected Collection> nodePlugins(String cluster) { + return Collections.singletonList(LocalStateEQLXPackPlugin.class); + } + + protected final Client localClient() { + return client(LOCAL_CLUSTER); + } + + @Override + protected List remoteClusterAlias() { + return List.of(REMOTE_CLUSTER); + } + + @Override + protected boolean reuseClusters() { + return false; + } + + /** + * + * @return remote node name + */ + private String createSchema() { + final Client remoteClient = client(REMOTE_CLUSTER); + final String remoteNode = cluster(REMOTE_CLUSTER).startDataOnlyNode(); + final String remoteNode2 = cluster(REMOTE_CLUSTER).startDataOnlyNode(); + + assertAcked( + remoteClient.admin() + .indices() + .prepareCreate("test-1-remote") + .setSettings( + Settings.builder() + .put("index.routing.allocation.require._name", remoteNode) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build() + ) + .setMapping("@timestamp", "type=date"), + TimeValue.timeValueSeconds(60) + ); + + assertAcked( + remoteClient.admin() + .indices() + .prepareCreate("test-2-remote") + .setSettings( + Settings.builder() + .put("index.routing.allocation.require._name", remoteNode2) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build() + ) + .setMapping("@timestamp", "type=date"), + TimeValue.timeValueSeconds(60) + ); + + for (int i = 0; i < 5; i++) { + int val = i * 2; + remoteClient.prepareIndex("test-1-remote") + .setId(Integer.toString(i)) + .setSource("@timestamp", 100000 + val, "event.category", "process", "key", "same", "value", val) + .get(); + } + for (int i = 0; i < 5; i++) { + int val = i * 2 + 1; + remoteClient.prepareIndex("test-2-remote") + .setId(Integer.toString(i)) + .setSource("@timestamp", 100000 + val, "event.category", "process", "key", "same", "value", val) + .get(); + } + + remoteClient.admin().indices().prepareRefresh().get(); + return remoteNode; + } + + // ------------------------------------------------------------------------ + // queries with full cluster (no missing shards) + // ------------------------------------------------------------------------ + + public void testNoFailures() throws ExecutionException, InterruptedException, IOException { + createSchema(); + + // event query + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("process where true") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + EqlSearchResponse response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().events().size(), equalTo(10)); + for (int i = 0; i < 10; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + i)); + } + assertThat(response.shardFailures().length, is(0)); + + // sequence query on both shards + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 1] [process where value == 2]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + EqlSearchResponse.Sequence sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 1")); + assertThat(sequence.events().get(1).toString(), containsString("\"value\" : 2")); + assertThat(response.shardFailures().length, is(0)); + + // sequence query on the available shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 1] [process where value == 3]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 1")); + assertThat(sequence.events().get(1).toString(), containsString("\"value\" : 3")); + assertThat(response.shardFailures().length, is(0)); + + // sequence query on the unavailable shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 0] [process where value == 2]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 0")); + assertThat(sequence.events().get(1).toString(), containsString("\"value\" : 2")); + assertThat(response.shardFailures().length, is(0)); + + // sequence query with missing event on unavailable shard + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(0)); + + // sample query on both shards + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 2] [process where value == 1]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + EqlSearchResponse.Sequence sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 2")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(0)); + + // sample query on the available shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 3] [process where value == 1]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(0)); + + // sample query on the unavailable shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 2] [process where value == 0]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 2")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 0")); + assertThat(response.shardFailures().length, is(0)); + + } + + // ------------------------------------------------------------------------ + // same queries, with missing shards and allow_partial_search_results=true + // and allow_partial_sequence_result=true + // ------------------------------------------------------------------------ + + public void testAllowPartialSearchAndSequence_event() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + // event query + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("process where true") + .allowPartialSearchResults(true); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().events().size(), equalTo(5)); + for (int i = 0; i < 5; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + (i * 2 + 1))); + } + assertThat(response.shardFailures().length, is(1)); + } + + public void testAllowPartialSearchAndSequence_sequence() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + // sequence query on both shards + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 1] [process where value == 2]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the available shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 1] [process where value == 3]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 1")); + assertThat(sequence.events().get(1).toString(), containsString("\"value\" : 3")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the unavailable shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 0] [process where value == 2]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query with missing event on unavailable shard. THIS IS A FALSE POSITIVE + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 1")); + assertThat(sequence.events().get(2).toString(), containsString("\"value\" : 3")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + public void testAllowPartialSearchAndSequence_sample() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + // sample query on both shards + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 2] [process where value == 1]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the available shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 3] [process where value == 1]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the unavailable shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 2] [process where value == 0]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + // ------------------------------------------------------------------------ + // same queries, with missing shards and allow_partial_search_results=true + // and default allow_partial_sequence_results (ie. false) + // ------------------------------------------------------------------------ + + public void testAllowPartialSearch_event() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + // event query + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("process where true") + .allowPartialSearchResults(true); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().events().size(), equalTo(5)); + for (int i = 0; i < 5; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + (i * 2 + 1))); + } + assertThat(response.shardFailures().length, is(1)); + + } + + public void testAllowPartialSearch_sequence() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + // sequence query on both shards + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 1] [process where value == 2]") + .allowPartialSearchResults(true); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the available shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 1] [process where value == 3]") + .allowPartialSearchResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the unavailable shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 0] [process where value == 2]") + .allowPartialSearchResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query with missing event on unavailable shard. THIS IS A FALSE POSITIVE + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]") + .allowPartialSearchResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + public void testAllowPartialSearch_sample() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + // sample query on both shards + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 2] [process where value == 1]") + .allowPartialSearchResults(true); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the available shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 3] [process where value == 1]") + .allowPartialSearchResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the unavailable shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 2] [process where value == 0]") + .allowPartialSearchResults(true); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + // ------------------------------------------------------------------------ + // same queries, with missing shards and with default xpack.eql.default_allow_partial_results=true + // ------------------------------------------------------------------------ + + public void testClusterSetting_event() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + cluster(REMOTE_CLUSTER).client() + .execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().put(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey(), true) + ) + ) + .get(); + + // event query + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*").query("process where true"); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().events().size(), equalTo(5)); + for (int i = 0; i < 5; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + (i * 2 + 1))); + } + assertThat(response.shardFailures().length, is(1)); + + localClient().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().putNull(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey()) + ) + ).get(); + } + + public void testClusterSetting_sequence() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + cluster(REMOTE_CLUSTER).client() + .execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().put(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey(), true) + ) + ) + .get(); + // sequence query on both shards + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 1] [process where value == 2]"); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the available shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 1] [process where value == 3]"); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the unavailable shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence [process where value == 0] [process where value == 2]"); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query with missing event on unavailable shard. THIS IS A FALSE POSITIVE + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]"); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + localClient().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().putNull(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey()) + ) + ).get(); + } + + public void testClusterSetting_sample() throws ExecutionException, InterruptedException, IOException { + var remoteNode = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + cluster(REMOTE_CLUSTER).stopNode(remoteNode); + + cluster(REMOTE_CLUSTER).client() + .execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().put(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey(), true) + ) + ) + .get(); + + // sample query on both shards + var request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 2] [process where value == 1]"); + var response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the available shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 3] [process where value == 1]"); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the unavailable shard only + request = new EqlSearchRequest().indices(REMOTE_CLUSTER + ":test-*") + .query("sample by key [process where value == 2] [process where value == 0]"); + response = localClient().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1-remote")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + localClient().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().putNull(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey()) + ) + ).get(); + } +} diff --git a/x-pack/plugin/eql/src/internalClusterTest/java/org/elasticsearch/xpack/eql/action/PartialSearchResultsIT.java b/x-pack/plugin/eql/src/internalClusterTest/java/org/elasticsearch/xpack/eql/action/PartialSearchResultsIT.java new file mode 100644 index 0000000000000..9048d11f4eddf --- /dev/null +++ b/x-pack/plugin/eql/src/internalClusterTest/java/org/elasticsearch/xpack/eql/action/PartialSearchResultsIT.java @@ -0,0 +1,780 @@ +/* + * 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.eql.action; + +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsAction; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.action.search.SearchPhaseExecutionException; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.xpack.core.async.GetAsyncResultRequest; +import org.elasticsearch.xpack.eql.plugin.EqlAsyncGetResultAction; +import org.elasticsearch.xpack.eql.plugin.EqlPlugin; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.ExecutionException; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; + +public class PartialSearchResultsIT extends AbstractEqlIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return CollectionUtils.appendToCopy(super.nodePlugins(), MockTransportService.TestPlugin.class); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(SearchService.KEEPALIVE_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(randomIntBetween(100, 500))) + .build(); + } + + /** + * + * @return node name where the first index is + */ + private String createSchema() { + internalCluster().ensureAtLeastNumDataNodes(2); + final List dataNodes = internalCluster().clusterService() + .state() + .nodes() + .getDataNodes() + .values() + .stream() + .map(DiscoveryNode::getName) + .toList(); + final String assignedNodeForIndex1 = randomFrom(dataNodes); + + assertAcked( + indicesAdmin().prepareCreate("test-1") + .setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.routing.allocation.include._name", assignedNodeForIndex1) + .build() + ) + .setMapping("@timestamp", "type=date") + ); + assertAcked( + indicesAdmin().prepareCreate("test-2") + .setSettings( + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.routing.allocation.exclude._name", assignedNodeForIndex1) + .build() + ) + .setMapping("@timestamp", "type=date") + ); + + for (int i = 0; i < 5; i++) { + int val = i * 2; + prepareIndex("test-1").setId(Integer.toString(i)) + .setSource("@timestamp", 100000 + val, "event.category", "process", "key", "same", "value", val) + .get(); + } + for (int i = 0; i < 5; i++) { + int val = i * 2 + 1; + prepareIndex("test-2").setId(Integer.toString(i)) + .setSource("@timestamp", 100000 + val, "event.category", "process", "key", "same", "value", val) + .get(); + } + refresh(); + return assignedNodeForIndex1; + } + + public void testNoFailures() throws Exception { + createSchema(); + + // ------------------------------------------------------------------------ + // queries with full cluster (no missing shards) + // ------------------------------------------------------------------------ + + // event query + var request = new EqlSearchRequest().indices("test-*") + .query("process where true") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + EqlSearchResponse response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().events().size(), equalTo(10)); + for (int i = 0; i < 10; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + i)); + } + assertThat(response.shardFailures().length, is(0)); + + // sequence query on both shards + request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 1] [process where value == 2]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + EqlSearchResponse.Sequence sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 1")); + assertThat(sequence.events().get(1).toString(), containsString("\"value\" : 2")); + assertThat(response.shardFailures().length, is(0)); + + // sequence query on the available shard only + request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 1] [process where value == 3]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 1")); + assertThat(sequence.events().get(1).toString(), containsString("\"value\" : 3")); + assertThat(response.shardFailures().length, is(0)); + + // sequence query on the unavailable shard only + request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 0] [process where value == 2]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 0")); + assertThat(sequence.events().get(1).toString(), containsString("\"value\" : 2")); + assertThat(response.shardFailures().length, is(0)); + + // sequence query with missing event on unavailable shard + request = new EqlSearchRequest().indices("test-*") + .query("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(0)); + + // sample query on both shards + request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 2] [process where value == 1]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + EqlSearchResponse.Sequence sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 2")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(0)); + + // sample query on the available shard only + request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 3] [process where value == 1]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(0)); + + // sample query on the unavailable shard only + request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 2] [process where value == 0]") + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 2")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 0")); + assertThat(response.shardFailures().length, is(0)); + + } + + // ------------------------------------------------------------------------ + // same queries, with missing shards. Let them fail + // allow_partial_sequence_results has no effect if allow_partial_sequence_results is not set to true. + // ------------------------------------------------------------------------ + + public void testFailures_event() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // event query + shouldFail("process where true"); + + } + + public void testFailures_sequence() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // sequence query on both shards + shouldFail("sequence [process where value == 1] [process where value == 2]"); + + // sequence query on the available shard only + shouldFail("sequence [process where value == 1] [process where value == 3]"); + + // sequence query on the unavailable shard only + shouldFail("sequence [process where value == 0] [process where value == 2]"); + + // sequence query with missing event on unavailable shard. + shouldFail("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]"); + } + + public void testFailures_sample() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // sample query on both shards + shouldFail("sample by key [process where value == 2] [process where value == 1]"); + + // sample query on the available shard only + shouldFail("sample by key [process where value == 3] [process where value == 1]"); + + // sample query on the unavailable shard only + shouldFail("sample by key [process where value == 2] [process where value == 0]"); + + } + + // ------------------------------------------------------------------------ + // same queries, with missing shards and allow_partial_search_results=true + // and allow_partial_sequence_result=true + // ------------------------------------------------------------------------ + + public void testAllowPartialSearchAndSequenceResults_event() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // event query + var request = new EqlSearchRequest().indices("test-*").query("process where true").allowPartialSearchResults(true); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().events().size(), equalTo(5)); + for (int i = 0; i < 5; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + (i * 2 + 1))); + } + assertThat(response.shardFailures().length, is(1)); + + } + + public void testAllowPartialSearchAndSequenceResults_sequence() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // sequence query on both shards + var request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 1] [process where value == 2]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the available shard only + request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 1] [process where value == 3]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 1")); + assertThat(sequence.events().get(1).toString(), containsString("\"value\" : 3")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the unavailable shard only + request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 0] [process where value == 2]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query with missing event on unavailable shard. THIS IS A FALSE POSITIVE + request = new EqlSearchRequest().indices("test-*") + .query("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + sequence = response.hits().sequences().get(0); + assertThat(sequence.events().get(0).toString(), containsString("\"value\" : 1")); + assertThat(sequence.events().get(2).toString(), containsString("\"value\" : 3")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + public void testAllowPartialSearchAndSequenceResults_sample() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // sample query on both shards + var request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 2] [process where value == 1]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the available shard only + request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 3] [process where value == 1]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the unavailable shard only + request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 2] [process where value == 0]") + .allowPartialSearchResults(true) + .allowPartialSequenceResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + // ------------------------------------------------------------------------ + // same queries, with missing shards and allow_partial_search_results=true + // and default allow_partial_sequence_results (ie. false) + // ------------------------------------------------------------------------ + + public void testAllowPartialSearchResults_event() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // event query + var request = new EqlSearchRequest().indices("test-*").query("process where true").allowPartialSearchResults(true); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().events().size(), equalTo(5)); + for (int i = 0; i < 5; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + (i * 2 + 1))); + } + assertThat(response.shardFailures().length, is(1)); + + } + + public void testAllowPartialSearchResults_sequence() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // sequence query on both shards + var request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 1] [process where value == 2]") + .allowPartialSearchResults(true); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the available shard only + request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 1] [process where value == 3]") + .allowPartialSearchResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the unavailable shard only + request = new EqlSearchRequest().indices("test-*") + .query("sequence [process where value == 0] [process where value == 2]") + .allowPartialSearchResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query with missing event on unavailable shard. THIS IS A FALSE POSITIVE + request = new EqlSearchRequest().indices("test-*") + .query("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]") + .allowPartialSearchResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + public void testAllowPartialSearchResults_sample() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // sample query on both shards + var request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 2] [process where value == 1]") + .allowPartialSearchResults(true); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the available shard only + request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 3] [process where value == 1]") + .allowPartialSearchResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the unavailable shard only + request = new EqlSearchRequest().indices("test-*") + .query("sample by key [process where value == 2] [process where value == 0]") + .allowPartialSearchResults(true); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + // ------------------------------------------------------------------------ + // same queries, this time async, with missing shards and allow_partial_search_results=true + // and default allow_partial_sequence_results (ie. false) + // ------------------------------------------------------------------------ + + public void testAsyncAllowPartialSearchResults_event() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // event query + var response = runAsync("process where true", true); + assertThat(response.hits().events().size(), equalTo(5)); + for (int i = 0; i < 5; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + (i * 2 + 1))); + } + assertThat(response.shardFailures().length, is(1)); + + } + + public void testAsyncAllowPartialSearchResults_sequence() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + // sequence query on both shards + var response = runAsync("sequence [process where value == 1] [process where value == 2]", true); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the available shard only + response = runAsync("sequence [process where value == 1] [process where value == 3]", true); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the unavailable shard only + response = runAsync("sequence [process where value == 0] [process where value == 2]", true); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query with missing event on unavailable shard. THIS IS A FALSE POSITIVE + response = runAsync( + "sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]", + true + ); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + public void testAsyncAllowPartialSearchResults_sample() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + // sample query on both shards + var response = runAsync("sample by key [process where value == 2] [process where value == 1]", true); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the available shard only + response = runAsync("sample by key [process where value == 3] [process where value == 1]", true); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the unavailable shard only + response = runAsync("sample by key [process where value == 2] [process where value == 0]", true); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + } + + // ------------------------------------------------------------------------ + // same queries, with missing shards and with default xpack.eql.default_allow_partial_results=true + // ------------------------------------------------------------------------ + + public void testClusterSetting_event() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + client().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().put(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey(), true) + ) + ).get(); + + // event query + var request = new EqlSearchRequest().indices("test-*").query("process where true"); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().events().size(), equalTo(5)); + for (int i = 0; i < 5; i++) { + assertThat(response.hits().events().get(i).toString(), containsString("\"value\" : " + (i * 2 + 1))); + } + assertThat(response.shardFailures().length, is(1)); + + client().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().putNull(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey()) + ) + ).get(); + } + + public void testClusterSetting_sequence() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + client().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().put(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey(), true) + ) + ).get(); + // sequence query on both shards + var request = new EqlSearchRequest().indices("test-*").query("sequence [process where value == 1] [process where value == 2]"); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the available shard only + request = new EqlSearchRequest().indices("test-*").query("sequence [process where value == 1] [process where value == 3]"); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query on the unavailable shard only + request = new EqlSearchRequest().indices("test-*").query("sequence [process where value == 0] [process where value == 2]"); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sequence query with missing event on unavailable shard. THIS IS A FALSE POSITIVE + request = new EqlSearchRequest().indices("test-*") + .query("sequence with maxspan=10s [process where value == 1] ![process where value == 2] [process where value == 3]"); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + client().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().putNull(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey()) + ) + ).get(); + } + + public void testClusterSetting_sample() throws Exception { + final String assignedNodeForIndex1 = createSchema(); + // ------------------------------------------------------------------------ + // stop one of the nodes, make one of the shards unavailable + // ------------------------------------------------------------------------ + + internalCluster().stopNode(assignedNodeForIndex1); + + client().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().put(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey(), true) + ) + ).get(); + + // sample query on both shards + var request = new EqlSearchRequest().indices("test-*").query("sample by key [process where value == 2] [process where value == 1]"); + var response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the available shard only + request = new EqlSearchRequest().indices("test-*").query("sample by key [process where value == 3] [process where value == 1]"); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(1)); + var sample = response.hits().sequences().get(0); + assertThat(sample.events().get(0).toString(), containsString("\"value\" : 3")); + assertThat(sample.events().get(1).toString(), containsString("\"value\" : 1")); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + // sample query on the unavailable shard only + request = new EqlSearchRequest().indices("test-*").query("sample by key [process where value == 2] [process where value == 0]"); + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + assertThat(response.hits().sequences().size(), equalTo(0)); + assertThat(response.shardFailures().length, is(1)); + assertThat(response.shardFailures()[0].index(), is("test-1")); + assertThat(response.shardFailures()[0].reason(), containsString("NoShardAvailableActionException")); + + client().execute( + ClusterUpdateSettingsAction.INSTANCE, + new ClusterUpdateSettingsRequest(TimeValue.THIRTY_SECONDS, TimeValue.THIRTY_SECONDS).persistentSettings( + Settings.builder().putNull(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getKey()) + ) + ).get(); + } + + private static EqlSearchResponse runAsync(String query, Boolean allowPartialSearchResults) throws InterruptedException, + ExecutionException { + EqlSearchRequest request; + EqlSearchResponse response; + request = new EqlSearchRequest().indices("test-*").query(query).waitForCompletionTimeout(TimeValue.ZERO); + if (allowPartialSearchResults != null) { + request = request.allowPartialSearchResults(allowPartialSearchResults); + } + response = client().execute(EqlSearchAction.INSTANCE, request).get(); + while (response.isRunning()) { + GetAsyncResultRequest getResultsRequest = new GetAsyncResultRequest(response.id()).setKeepAlive(TimeValue.timeValueMinutes(10)) + .setWaitForCompletionTimeout(TimeValue.timeValueMillis(10)); + response = client().execute(EqlAsyncGetResultAction.INSTANCE, getResultsRequest).get(); + } + return response; + } + + private static void shouldFail(String query) throws InterruptedException { + EqlSearchRequest request = new EqlSearchRequest().indices("test-*").query(query); + if (randomBoolean()) { + request = request.allowPartialSearchResults(false); + } + if (randomBoolean()) { + request = request.allowPartialSequenceResults(randomBoolean()); + } + try { + client().execute(EqlSearchAction.INSTANCE, request).get(); + fail(); + } catch (ExecutionException e) { + assertThat(e.getCause(), instanceOf(SearchPhaseExecutionException.class)); + } + } +} diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchRequest.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchRequest.java index 0aeddd525e317..5804e11b72ff5 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchRequest.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchRequest.java @@ -63,6 +63,8 @@ public class EqlSearchRequest extends ActionRequest implements IndicesRequest.Re private List fetchFields; private Map runtimeMappings = emptyMap(); private int maxSamplesPerKey = RequestDefaults.MAX_SAMPLES_PER_KEY; + private Boolean allowPartialSearchResults; + private Boolean allowPartialSequenceResults; // Async settings private TimeValue waitForCompletionTimeout = null; @@ -83,6 +85,8 @@ public class EqlSearchRequest extends ActionRequest implements IndicesRequest.Re static final String KEY_FETCH_FIELDS = "fields"; static final String KEY_RUNTIME_MAPPINGS = "runtime_mappings"; static final String KEY_MAX_SAMPLES_PER_KEY = "max_samples_per_key"; + static final String KEY_ALLOW_PARTIAL_SEARCH_RESULTS = "allow_partial_search_results"; + static final String KEY_ALLOW_PARTIAL_SEQUENCE_RESULTS = "allow_partial_sequence_results"; static final ParseField FILTER = new ParseField(KEY_FILTER); static final ParseField TIMESTAMP_FIELD = new ParseField(KEY_TIMESTAMP_FIELD); @@ -97,6 +101,8 @@ public class EqlSearchRequest extends ActionRequest implements IndicesRequest.Re static final ParseField RESULT_POSITION = new ParseField(KEY_RESULT_POSITION); static final ParseField FETCH_FIELDS_FIELD = SearchSourceBuilder.FETCH_FIELDS_FIELD; static final ParseField MAX_SAMPLES_PER_KEY = new ParseField(KEY_MAX_SAMPLES_PER_KEY); + static final ParseField ALLOW_PARTIAL_SEARCH_RESULTS = new ParseField(KEY_ALLOW_PARTIAL_SEARCH_RESULTS); + static final ParseField ALLOW_PARTIAL_SEQUENCE_RESULTS = new ParseField(KEY_ALLOW_PARTIAL_SEQUENCE_RESULTS); private static final ObjectParser PARSER = objectParser(EqlSearchRequest::new); @@ -135,6 +141,13 @@ public EqlSearchRequest(StreamInput in) throws IOException { if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_7_0)) { maxSamplesPerKey = in.readInt(); } + if (in.getTransportVersion().onOrAfter(TransportVersions.EQL_ALLOW_PARTIAL_SEARCH_RESULTS)) { + allowPartialSearchResults = in.readOptionalBoolean(); + allowPartialSequenceResults = in.readOptionalBoolean(); + } else { + allowPartialSearchResults = false; + allowPartialSequenceResults = false; + } } @Override @@ -245,6 +258,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(KEY_RUNTIME_MAPPINGS, runtimeMappings); } builder.field(KEY_MAX_SAMPLES_PER_KEY, maxSamplesPerKey); + builder.field(KEY_ALLOW_PARTIAL_SEARCH_RESULTS, allowPartialSearchResults); + builder.field(KEY_ALLOW_PARTIAL_SEQUENCE_RESULTS, allowPartialSequenceResults); return builder; } @@ -279,6 +294,8 @@ protected static ObjectParser objectParser parser.declareField(EqlSearchRequest::fetchFields, EqlSearchRequest::parseFetchFields, FETCH_FIELDS_FIELD, ValueType.VALUE_ARRAY); parser.declareObject(EqlSearchRequest::runtimeMappings, (p, c) -> p.map(), SearchSourceBuilder.RUNTIME_MAPPINGS_FIELD); parser.declareInt(EqlSearchRequest::maxSamplesPerKey, MAX_SAMPLES_PER_KEY); + parser.declareBoolean(EqlSearchRequest::allowPartialSearchResults, ALLOW_PARTIAL_SEARCH_RESULTS); + parser.declareBoolean(EqlSearchRequest::allowPartialSequenceResults, ALLOW_PARTIAL_SEQUENCE_RESULTS); return parser; } @@ -427,6 +444,24 @@ public EqlSearchRequest maxSamplesPerKey(int maxSamplesPerKey) { return this; } + public Boolean allowPartialSearchResults() { + return allowPartialSearchResults; + } + + public EqlSearchRequest allowPartialSearchResults(Boolean val) { + this.allowPartialSearchResults = val; + return this; + } + + public Boolean allowPartialSequenceResults() { + return allowPartialSequenceResults; + } + + public EqlSearchRequest allowPartialSequenceResults(Boolean val) { + this.allowPartialSequenceResults = val; + return this; + } + private static List parseFetchFields(XContentParser parser) throws IOException { List result = new ArrayList<>(); Token token = parser.currentToken(); @@ -470,6 +505,10 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getTransportVersion().onOrAfter(TransportVersions.V_8_7_0)) { out.writeInt(maxSamplesPerKey); } + if (out.getTransportVersion().onOrAfter(TransportVersions.EQL_ALLOW_PARTIAL_SEARCH_RESULTS)) { + out.writeOptionalBoolean(allowPartialSearchResults); + out.writeOptionalBoolean(allowPartialSequenceResults); + } } @Override @@ -496,7 +535,9 @@ public boolean equals(Object o) { && Objects.equals(resultPosition, that.resultPosition) && Objects.equals(fetchFields, that.fetchFields) && Objects.equals(runtimeMappings, that.runtimeMappings) - && Objects.equals(maxSamplesPerKey, that.maxSamplesPerKey); + && Objects.equals(maxSamplesPerKey, that.maxSamplesPerKey) + && Objects.equals(allowPartialSearchResults, that.allowPartialSearchResults) + && Objects.equals(allowPartialSequenceResults, that.allowPartialSequenceResults); } @Override @@ -517,7 +558,9 @@ public int hashCode() { resultPosition, fetchFields, runtimeMappings, - maxSamplesPerKey + maxSamplesPerKey, + allowPartialSearchResults, + allowPartialSequenceResults ); } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchResponse.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchResponse.java index 2b7b8b074fa71..a4d93b7659970 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchResponse.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchResponse.java @@ -7,8 +7,11 @@ package org.elasticsearch.xpack.eql.action; import org.apache.lucene.search.TotalHits; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -17,6 +20,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParserUtils; import org.elasticsearch.core.Nullable; @@ -36,6 +40,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -54,6 +59,7 @@ public class EqlSearchResponse extends ActionResponse implements ToXContentObjec private final String asyncExecutionId; private final boolean isRunning; private final boolean isPartial; + private final ShardSearchFailure[] shardFailures; private static final class Fields { static final String TOOK = "took"; @@ -62,6 +68,7 @@ private static final class Fields { static final String ID = "id"; static final String IS_RUNNING = "is_running"; static final String IS_PARTIAL = "is_partial"; + static final String SHARD_FAILURES = "shard_failures"; } private static final ParseField TOOK = new ParseField(Fields.TOOK); @@ -70,8 +77,10 @@ private static final class Fields { private static final ParseField ID = new ParseField(Fields.ID); private static final ParseField IS_RUNNING = new ParseField(Fields.IS_RUNNING); private static final ParseField IS_PARTIAL = new ParseField(Fields.IS_PARTIAL); + private static final ParseField SHARD_FAILURES = new ParseField(Fields.SHARD_FAILURES); private static final InstantiatingObjectParser PARSER; + static { InstantiatingObjectParser.Builder parser = InstantiatingObjectParser.builder( "eql/search_response", @@ -84,11 +93,12 @@ private static final class Fields { parser.declareString(optionalConstructorArg(), ID); parser.declareBoolean(constructorArg(), IS_RUNNING); parser.declareBoolean(constructorArg(), IS_PARTIAL); + parser.declareObjectArray(optionalConstructorArg(), (p, c) -> ShardSearchFailure.EMPTY_ARRAY, SHARD_FAILURES); PARSER = parser.build(); } - public EqlSearchResponse(Hits hits, long tookInMillis, boolean isTimeout) { - this(hits, tookInMillis, isTimeout, null, false, false); + public EqlSearchResponse(Hits hits, long tookInMillis, boolean isTimeout, ShardSearchFailure[] shardFailures) { + this(hits, tookInMillis, isTimeout, null, false, false, shardFailures); } public EqlSearchResponse( @@ -97,7 +107,8 @@ public EqlSearchResponse( boolean isTimeout, String asyncExecutionId, boolean isRunning, - boolean isPartial + boolean isPartial, + ShardSearchFailure[] shardFailures ) { super(); this.hits = hits == null ? Hits.EMPTY : hits; @@ -106,6 +117,7 @@ public EqlSearchResponse( this.asyncExecutionId = asyncExecutionId; this.isRunning = isRunning; this.isPartial = isPartial; + this.shardFailures = shardFailures; } public EqlSearchResponse(StreamInput in) throws IOException { @@ -116,6 +128,11 @@ public EqlSearchResponse(StreamInput in) throws IOException { asyncExecutionId = in.readOptionalString(); isPartial = in.readBoolean(); isRunning = in.readBoolean(); + if (in.getTransportVersion().onOrAfter(TransportVersions.EQL_ALLOW_PARTIAL_SEARCH_RESULTS)) { + shardFailures = in.readArray(ShardSearchFailure::readShardSearchFailure, ShardSearchFailure[]::new); + } else { + shardFailures = ShardSearchFailure.EMPTY_ARRAY; + } } public static EqlSearchResponse fromXContent(XContentParser parser) { @@ -130,6 +147,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(asyncExecutionId); out.writeBoolean(isPartial); out.writeBoolean(isRunning); + if (out.getTransportVersion().onOrAfter(TransportVersions.EQL_ALLOW_PARTIAL_SEARCH_RESULTS)) { + out.writeArray(shardFailures); + } } @Override @@ -147,6 +167,13 @@ private XContentBuilder innerToXContent(XContentBuilder builder, Params params) builder.field(IS_RUNNING.getPreferredName(), isRunning); builder.field(TOOK.getPreferredName(), tookInMillis); builder.field(TIMED_OUT.getPreferredName(), isTimeout); + if (CollectionUtils.isEmpty(shardFailures) == false) { + builder.startArray(SHARD_FAILURES.getPreferredName()); + for (ShardOperationFailedException shardFailure : ExceptionsHelper.groupBy(shardFailures)) { + shardFailure.toXContent(builder, params); + } + builder.endArray(); + } hits.toXContent(builder, params); return builder; } @@ -178,6 +205,10 @@ public boolean isPartial() { return isPartial; } + public ShardSearchFailure[] shardFailures() { + return shardFailures; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -190,12 +221,13 @@ public boolean equals(Object o) { return Objects.equals(hits, that.hits) && Objects.equals(tookInMillis, that.tookInMillis) && Objects.equals(isTimeout, that.isTimeout) - && Objects.equals(asyncExecutionId, that.asyncExecutionId); + && Objects.equals(asyncExecutionId, that.asyncExecutionId) + && Arrays.equals(shardFailures, that.shardFailures); } @Override public int hashCode() { - return Objects.hash(hits, tookInMillis, isTimeout, asyncExecutionId); + return Objects.hash(hits, tookInMillis, isTimeout, asyncExecutionId, Arrays.hashCode(shardFailures)); } @Override diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchTask.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchTask.java index 2a1bc3b7adb67..0fc8e8c88d7d9 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchTask.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/action/EqlSearchTask.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.eql.action; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.core.TimeValue; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.xpack.core.async.AsyncExecutionId; @@ -39,7 +40,8 @@ public EqlSearchResponse getCurrentResult() { false, getExecutionId().getEncoded(), true, - true + true, + ShardSearchFailure.EMPTY_ARRAY ); } } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/assembler/ExecutionManager.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/assembler/ExecutionManager.java index b26c815c1a2b5..672d6b87a8dbb 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/assembler/ExecutionManager.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/assembler/ExecutionManager.java @@ -167,7 +167,9 @@ public Executable assemble( criteria.subList(0, completionStage), criteria.get(completionStage), matcher, - listOfKeys + listOfKeys, + cfg.allowPartialSearchResults(), + cfg.allowPartialSequenceResults() ); return w; @@ -235,7 +237,8 @@ public Executable assemble(List> listOfKeys, List cfg.fetchSize(), limit, session.circuitBreaker(), - cfg.maxSamplesPerKey() + cfg.maxSamplesPerKey(), + cfg.allowPartialSearchResults() ); } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/payload/AbstractPayload.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/payload/AbstractPayload.java index 823cd04d25f45..9fecf958b9714 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/payload/AbstractPayload.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/payload/AbstractPayload.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.eql.execution.payload; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.eql.session.Payload; @@ -14,10 +15,12 @@ public abstract class AbstractPayload implements Payload { private final boolean timedOut; private final TimeValue timeTook; + private ShardSearchFailure[] shardFailures; - protected AbstractPayload(boolean timedOut, TimeValue timeTook) { + protected AbstractPayload(boolean timedOut, TimeValue timeTook, ShardSearchFailure[] shardFailures) { this.timedOut = timedOut; this.timeTook = timeTook; + this.shardFailures = shardFailures; } @Override @@ -29,4 +32,9 @@ public boolean timedOut() { public TimeValue timeTook() { return timeTook; } + + @Override + public ShardSearchFailure[] shardFailures() { + return shardFailures; + } } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/payload/EventPayload.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/payload/EventPayload.java index a7845ca62dccc..6471bc0814f70 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/payload/EventPayload.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/payload/EventPayload.java @@ -20,7 +20,7 @@ public class EventPayload extends AbstractPayload { private final List values; public EventPayload(SearchResponse response) { - super(response.isTimedOut(), response.getTook()); + super(response.isTimedOut(), response.getTook(), response.getShardFailures()); SearchHits hits = response.getHits(); values = new ArrayList<>(hits.getHits().length); diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sample/SampleIterator.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sample/SampleIterator.java index 89f1c4d1eb041..b9b7cfd6b615a 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sample/SampleIterator.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sample/SampleIterator.java @@ -14,6 +14,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.core.TimeValue; import org.elasticsearch.search.SearchHit; @@ -35,6 +36,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -44,6 +46,7 @@ import static org.elasticsearch.common.Strings.EMPTY_ARRAY; import static org.elasticsearch.xpack.eql.execution.assembler.SampleQueryRequest.COMPOSITE_AGG_NAME; import static org.elasticsearch.xpack.eql.execution.search.RuntimeUtils.prepareRequest; +import static org.elasticsearch.xpack.eql.util.SearchHitUtils.addShardFailures; public class SampleIterator implements Executable { @@ -58,6 +61,7 @@ public class SampleIterator implements Executable { private final Limit limit; private final int maxSamplesPerKey; private long startTime; + private Map shardFailures = new HashMap<>(); // ---------- CIRCUIT BREAKER ----------- @@ -84,13 +88,16 @@ public class SampleIterator implements Executable { */ private long previousTotalPageSize = 0; + private boolean allowPartialSearchResults; + public SampleIterator( QueryClient client, List criteria, int fetchSize, Limit limit, CircuitBreaker circuitBreaker, - int maxSamplesPerKey + int maxSamplesPerKey, + boolean allowPartialSearchResults ) { this.client = client; this.criteria = criteria; @@ -100,6 +107,7 @@ public SampleIterator( this.limit = limit; this.circuitBreaker = circuitBreaker; this.maxSamplesPerKey = maxSamplesPerKey; + this.allowPartialSearchResults = allowPartialSearchResults; } @Override @@ -147,6 +155,7 @@ private void advance(ActionListener listener) { private void queryForCompositeAggPage(ActionListener listener, final SampleQueryRequest request) { client.query(request, listener.delegateFailureAndWrap((delegate, r) -> { + addShardFailures(shardFailures, r); // either the fields values or the fields themselves are missing // or the filter applied on the eql query matches no documents if (r.hasAggregations() == false) { @@ -209,13 +218,16 @@ private void finalStep(ActionListener listener) { for (SampleCriterion criterion : criteria) { SampleQueryRequest r = criterion.finalQuery(); r.singleKeyPair(compositeKeyValues, maxCriteria, maxSamplesPerKey); - searches.add(prepareRequest(r.searchSource(), false, EMPTY_ARRAY)); + searches.add(prepareRequest(r.searchSource(), false, allowPartialSearchResults, EMPTY_ARRAY)); } sampleKeys.add(new SequenceKey(compositeKeyValues.toArray())); } int initialSize = samples.size(); client.multiQuery(searches, listener.delegateFailureAndWrap((delegate, r) -> { + for (MultiSearchResponse.Item item : r) { + addShardFailures(shardFailures, item.getResponse()); + } List> sample = new ArrayList<>(maxCriteria); MultiSearchResponse.Item[] response = r.getResponses(); int docGroupsCounter = 1; @@ -280,14 +292,23 @@ private void payload(ActionListener listener) { log.trace("Sending payload for [{}] samples", samples.size()); if (samples.isEmpty()) { - listener.onResponse(new EmptyPayload(Type.SAMPLE, timeTook())); + listener.onResponse(new EmptyPayload(Type.SAMPLE, timeTook(), shardFailures.values().toArray(new ShardSearchFailure[0]))); return; } // get results through search (to keep using PIT) client.fetchHits( hits(samples), - ActionListeners.map(listener, listOfHits -> new SamplePayload(samples, listOfHits, false, timeTook())) + ActionListeners.map( + listener, + listOfHits -> new SamplePayload( + samples, + listOfHits, + false, + timeTook(), + shardFailures.values().toArray(new ShardSearchFailure[0]) + ) + ) ); } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sample/SamplePayload.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sample/SamplePayload.java index 121f4c208273b..aee084dd88734 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sample/SamplePayload.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sample/SamplePayload.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.eql.execution.sample; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.core.TimeValue; import org.elasticsearch.search.SearchHit; import org.elasticsearch.xpack.eql.action.EqlSearchResponse.Event; @@ -19,8 +20,14 @@ class SamplePayload extends AbstractPayload { private final List values; - SamplePayload(List samples, List> docs, boolean timedOut, TimeValue timeTook) { - super(timedOut, timeTook); + SamplePayload( + List samples, + List> docs, + boolean timedOut, + TimeValue timeTook, + ShardSearchFailure[] shardFailures + ) { + super(timedOut, timeTook, shardFailures); values = new ArrayList<>(samples.size()); for (int i = 0; i < samples.size(); i++) { diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/BasicQueryClient.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/BasicQueryClient.java index 6cbe5298b5950..18623c17dcffb 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/BasicQueryClient.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/BasicQueryClient.java @@ -46,12 +46,14 @@ public class BasicQueryClient implements QueryClient { final Client client; final String[] indices; final List fetchFields; + private final boolean allowPartialSearchResults; public BasicQueryClient(EqlSession eqlSession) { this.cfg = eqlSession.configuration(); this.client = eqlSession.client(); this.indices = cfg.indices(); this.fetchFields = cfg.fetchFields(); + this.allowPartialSearchResults = cfg.allowPartialSearchResults(); } @Override @@ -60,11 +62,11 @@ public void query(QueryRequest request, ActionListener listener) // set query timeout searchSource.timeout(cfg.requestTimeout()); - SearchRequest search = prepareRequest(searchSource, false, indices); - search(search, searchLogListener(listener, log)); + SearchRequest search = prepareRequest(searchSource, false, allowPartialSearchResults, indices); + search(search, allowPartialSearchResults, searchLogListener(listener, log, allowPartialSearchResults)); } - protected void search(SearchRequest search, ActionListener listener) { + protected void search(SearchRequest search, boolean allowPartialSearchResults, ActionListener listener) { if (cfg.isCancelled()) { listener.onFailure(new TaskCancelledException("cancelled")); return; @@ -77,7 +79,7 @@ protected void search(SearchRequest search, ActionListener liste client.search(search, listener); } - protected void search(MultiSearchRequest search, ActionListener listener) { + protected void search(MultiSearchRequest search, boolean allowPartialSearchResults, ActionListener listener) { if (cfg.isCancelled()) { listener.onFailure(new TaskCancelledException("cancelled")); return; @@ -91,7 +93,7 @@ protected void search(MultiSearchRequest search, ActionListener> refs, ActionListener { + search(multiSearchBuilder.request(), allowPartialSearchResults, listener.delegateFailureAndWrap((delegate, r) -> { for (MultiSearchResponse.Item item : r.getResponses()) { // check for failures if (item.isFailure()) { @@ -187,6 +189,6 @@ public void multiQuery(List searches, ActionListener listener) { + protected void search(SearchRequest search, boolean allowPartialSearchResults, ActionListener listener) { // no pitId, ask for one if (pitId == null) { - openPIT(listener, () -> searchWithPIT(search, listener)); + openPIT(listener, () -> searchWithPIT(search, listener, allowPartialSearchResults), allowPartialSearchResults); } else { - searchWithPIT(search, listener); + searchWithPIT(search, listener, allowPartialSearchResults); } } - private void searchWithPIT(SearchRequest request, ActionListener listener) { + private void searchWithPIT(SearchRequest request, ActionListener listener, boolean allowPartialSearchResults) { makeRequestPITCompatible(request); // get the pid on each response - super.search(request, pitListener(SearchResponse::pointInTimeId, listener)); + super.search(request, allowPartialSearchResults, pitListener(SearchResponse::pointInTimeId, listener)); } @Override - protected void search(MultiSearchRequest search, ActionListener listener) { + protected void search(MultiSearchRequest search, boolean allowPartialSearchResults, ActionListener listener) { // no pitId, ask for one if (pitId == null) { - openPIT(listener, () -> searchWithPIT(search, listener)); + openPIT(listener, () -> searchWithPIT(search, allowPartialSearchResults, listener), allowPartialSearchResults); } else { - searchWithPIT(search, listener); + searchWithPIT(search, allowPartialSearchResults, listener); } } - private void searchWithPIT(MultiSearchRequest search, ActionListener listener) { + private void searchWithPIT(MultiSearchRequest search, boolean allowPartialSearchResults, ActionListener listener) { for (SearchRequest request : search.requests()) { makeRequestPITCompatible(request); } // get the pid on each request - super.search(search, pitListener(r -> { + super.search(search, allowPartialSearchResults, pitListener(r -> { // get pid for (MultiSearchResponse.Item item : r.getResponses()) { // pick the first non-failing response @@ -135,9 +135,10 @@ private ActionListener pitListener( ); } - private void openPIT(ActionListener listener, Runnable runnable) { + private void openPIT(ActionListener listener, Runnable runnable, boolean allowPartialSearchResults) { OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndexResolver.FIELD_CAPS_INDICES_OPTIONS) - .keepAlive(keepAlive); + .keepAlive(keepAlive) + .allowPartialSearchResults(allowPartialSearchResults); request.indexFilter(filter); client.execute(TransportOpenPointInTimeAction.TYPE, request, listener.delegateFailureAndWrap((l, r) -> { pitId = r.getPointInTimeId(); diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/RuntimeUtils.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/RuntimeUtils.java index 40f7f7139efa1..92af8c562f840 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/RuntimeUtils.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/RuntimeUtils.java @@ -56,10 +56,14 @@ public final class RuntimeUtils { private RuntimeUtils() {} - public static ActionListener searchLogListener(ActionListener listener, Logger log) { + public static ActionListener searchLogListener( + ActionListener listener, + Logger log, + boolean allowPartialResults + ) { return listener.delegateFailureAndWrap((delegate, response) -> { ShardSearchFailure[] failures = response.getShardFailures(); - if (CollectionUtils.isEmpty(failures) == false) { + if (CollectionUtils.isEmpty(failures) == false && allowPartialResults == false) { delegate.onFailure(new EqlIllegalArgumentException(failures[0].reason(), failures[0].getCause())); return; } @@ -70,16 +74,22 @@ public static ActionListener searchLogListener(ActionListener multiSearchLogListener(ActionListener listener, Logger log) { + public static ActionListener multiSearchLogListener( + ActionListener listener, + boolean allowPartialSearchResults, + Logger log + ) { return listener.delegateFailureAndWrap((delegate, items) -> { for (MultiSearchResponse.Item item : items) { Exception failure = item.getFailure(); SearchResponse response = item.getResponse(); if (failure == null) { - ShardSearchFailure[] failures = response.getShardFailures(); - if (CollectionUtils.isEmpty(failures) == false) { - failure = new EqlIllegalArgumentException(failures[0].reason(), failures[0].getCause()); + if (allowPartialSearchResults == false) { + ShardSearchFailure[] failures = response.getShardFailures(); + if (CollectionUtils.isEmpty(failures) == false) { + failure = new EqlIllegalArgumentException(failures[0].reason(), failures[0].getCause()); + } } } if (failure != null) { @@ -170,11 +180,16 @@ public static HitExtractor createExtractor(FieldExtraction ref, EqlConfiguration throw new EqlIllegalArgumentException("Unexpected value reference {}", ref.getClass()); } - public static SearchRequest prepareRequest(SearchSourceBuilder source, boolean includeFrozen, String... indices) { + public static SearchRequest prepareRequest( + SearchSourceBuilder source, + boolean includeFrozen, + boolean allowPartialSearchResults, + String... indices + ) { SearchRequest searchRequest = new SearchRequest(); searchRequest.indices(indices); searchRequest.source(source); - searchRequest.allowPartialSearchResults(false); + searchRequest.allowPartialSearchResults(allowPartialSearchResults); searchRequest.indicesOptions( includeFrozen ? IndexResolver.FIELD_CAPS_FROZEN_INDICES_OPTIONS : IndexResolver.FIELD_CAPS_INDICES_OPTIONS ); diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sequence/SequencePayload.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sequence/SequencePayload.java index 45083babddbb4..b4a8edc79b3ad 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sequence/SequencePayload.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sequence/SequencePayload.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.eql.execution.sequence; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.core.TimeValue; import org.elasticsearch.search.SearchHit; import org.elasticsearch.xpack.eql.action.EqlSearchResponse.Event; @@ -19,8 +20,14 @@ class SequencePayload extends AbstractPayload { private final List values; - SequencePayload(List sequences, List> docs, boolean timedOut, TimeValue timeTook) { - super(timedOut, timeTook); + SequencePayload( + List sequences, + List> docs, + boolean timedOut, + TimeValue timeTook, + ShardSearchFailure[] shardFailures + ) { + super(timedOut, timeTook, shardFailures); values = new ArrayList<>(sequences.size()); for (int i = 0; i < sequences.size(); i++) { diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sequence/TumblingWindow.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sequence/TumblingWindow.java index eabf6df518ad4..fac8788db0f95 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sequence/TumblingWindow.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/sequence/TumblingWindow.java @@ -13,6 +13,7 @@ import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.common.Strings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.Tuple; @@ -41,6 +42,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -51,6 +53,7 @@ import static org.elasticsearch.action.ActionListener.runAfter; import static org.elasticsearch.xpack.eql.execution.ExecutionUtils.copySource; import static org.elasticsearch.xpack.eql.execution.search.RuntimeUtils.combineFilters; +import static org.elasticsearch.xpack.eql.util.SearchHitUtils.addShardFailures; import static org.elasticsearch.xpack.eql.util.SearchHitUtils.qualifiedIndex; /** @@ -103,6 +106,9 @@ protected boolean removeEldestEntry(Map.Entry eldest) { private final boolean hasKeys; private final List> listOfKeys; + private final boolean allowPartialSearchResults; + private final boolean allowPartialSequenceResults; + private Map shardFailures = new HashMap<>(); // flag used for DESC sequences to indicate whether // the window needs to restart (since the DESC query still has results) @@ -127,7 +133,10 @@ public TumblingWindow( List criteria, SequenceCriterion until, SequenceMatcher matcher, - List> listOfKeys + List> listOfKeys, + boolean allowPartialSearchResults, + boolean allowPartialSequenceResults + ) { this.client = client; @@ -141,6 +150,8 @@ public TumblingWindow( this.hasKeys = baseRequest.keySize() > 0; this.restartWindowFromTailQuery = baseRequest.descending(); this.listOfKeys = listOfKeys; + this.allowPartialSearchResults = allowPartialSearchResults; + this.allowPartialSequenceResults = allowPartialSequenceResults; } @Override @@ -158,6 +169,9 @@ public void execute(ActionListener listener) { * Move the window while preserving the same base. */ private void tumbleWindow(int currentStage, ActionListener listener) { + if (allowPartialSequenceResults == false && shardFailures.isEmpty() == false) { + doPayload(listener); + } if (currentStage > matcher.firstPositiveStage && matcher.hasCandidates() == false) { if (restartWindowFromTailQuery) { currentStage = matcher.firstPositiveStage; @@ -224,6 +238,9 @@ public void checkMissingEvents(Runnable next, ActionListener listener) private void doCheckMissingEvents(List batchToCheck, MultiSearchResponse p, ActionListener listener, Runnable next) { MultiSearchResponse.Item[] responses = p.getResponses(); + for (MultiSearchResponse.Item response : responses) { + addShardFailures(shardFailures, response.getResponse()); + } int nextResponse = 0; for (Sequence sequence : batchToCheck) { boolean leading = true; @@ -316,7 +333,14 @@ private List prepareQueryForMissingEvents(List toCheck) } addKeyFilter(i, sequence, builder); RuntimeUtils.combineFilters(builder, range); - result.add(RuntimeUtils.prepareRequest(builder.size(1).trackTotalHits(false), false, Strings.EMPTY_ARRAY)); + result.add( + RuntimeUtils.prepareRequest( + builder.size(1).trackTotalHits(false), + false, + allowPartialSearchResults, + Strings.EMPTY_ARRAY + ) + ); } else { leading = false; } @@ -361,6 +385,7 @@ private void advance(int stage, ActionListener listener) { * Execute the base query. */ private void baseCriterion(int baseStage, SearchResponse r, ActionListener listener) { + addShardFailures(shardFailures, r); SequenceCriterion base = criteria.get(baseStage); SearchHits hits = r.getHits(); @@ -731,8 +756,10 @@ private void doPayload(ActionListener listener) { log.trace("Sending payload for [{}] sequences", completed.size()); - if (completed.isEmpty()) { - listener.onResponse(new EmptyPayload(Type.SEQUENCE, timeTook())); + if (completed.isEmpty() || (allowPartialSequenceResults == false && shardFailures.isEmpty() == false)) { + listener.onResponse( + new EmptyPayload(Type.SEQUENCE, timeTook(), shardFailures.values().toArray(new ShardSearchFailure[shardFailures.size()])) + ); return; } @@ -741,7 +768,13 @@ private void doPayload(ActionListener listener) { if (criteria.get(matcher.firstPositiveStage).descending()) { Collections.reverse(completed); } - return new SequencePayload(completed, addMissingEventPlaceholders(listOfHits), false, timeTook()); + return new SequencePayload( + completed, + addMissingEventPlaceholders(listOfHits), + false, + timeTook(), + shardFailures.values().toArray(new ShardSearchFailure[0]) + ); })); } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/EqlPlugin.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/EqlPlugin.java index 084a5e74a47e8..210f88c991539 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/EqlPlugin.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/EqlPlugin.java @@ -60,6 +60,20 @@ public class EqlPlugin extends Plugin implements ActionPlugin, CircuitBreakerPlu Setting.Property.DeprecatedWarning ); + public static final Setting DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS = Setting.boolSetting( + "xpack.eql.default_allow_partial_results", + false, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public static final Setting DEFAULT_ALLOW_PARTIAL_SEQUENCE_RESULTS = Setting.boolSetting( + "xpack.eql.default_allow_partial_sequence_results", + false, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + public EqlPlugin() {} @Override @@ -86,7 +100,7 @@ private Collection createComponents(Client client, Settings settings, Cl */ @Override public List> getSettings() { - return List.of(EQL_ENABLED_SETTING); + return List.of(EQL_ENABLED_SETTING, DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS, DEFAULT_ALLOW_PARTIAL_SEQUENCE_RESULTS); } @Override diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/RestEqlSearchAction.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/RestEqlSearchAction.java index e24a4749f45cd..65def24563e5e 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/RestEqlSearchAction.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/RestEqlSearchAction.java @@ -64,6 +64,12 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient cli } eqlRequest.keepOnCompletion(request.paramAsBoolean("keep_on_completion", eqlRequest.keepOnCompletion())); eqlRequest.ccsMinimizeRoundtrips(request.paramAsBoolean("ccs_minimize_roundtrips", eqlRequest.ccsMinimizeRoundtrips())); + eqlRequest.allowPartialSearchResults( + request.paramAsBoolean("allow_partial_search_results", eqlRequest.allowPartialSearchResults()) + ); + eqlRequest.allowPartialSequenceResults( + request.paramAsBoolean("allow_partial_sequence_results", eqlRequest.allowPartialSequenceResults()) + ); } return channel -> { diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/TransportEqlSearchAction.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/TransportEqlSearchAction.java index c0141da2432ce..582352722fc58 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/TransportEqlSearchAction.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/plugin/TransportEqlSearchAction.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.internal.Client; @@ -144,7 +145,8 @@ public EqlSearchResponse initialResponse(EqlSearchTask task) { false, task.getExecutionId().getEncoded(), true, - true + true, + ShardSearchFailure.EMPTY_ARRAY ); } @@ -231,6 +233,12 @@ public static void operation( request.indicesOptions(), request.fetchSize(), request.maxSamplesPerKey(), + request.allowPartialSearchResults() == null + ? defaultAllowPartialSearchResults(clusterService) + : request.allowPartialSearchResults(), + request.allowPartialSequenceResults() == null + ? defaultAllowPartialSequenceResults(clusterService) + : request.allowPartialSequenceResults(), clientId, new TaskId(nodeId, task.getId()), task @@ -244,12 +252,34 @@ public static void operation( } } + private static boolean defaultAllowPartialSearchResults(ClusterService clusterService) { + if (clusterService.getClusterSettings() == null) { + return EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS.getDefault(Settings.EMPTY); + } + return clusterService.getClusterSettings().get(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEARCH_RESULTS); + } + + private static boolean defaultAllowPartialSequenceResults(ClusterService clusterService) { + if (clusterService.getClusterSettings() == null) { + return EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEQUENCE_RESULTS.getDefault(Settings.EMPTY); + } + return clusterService.getClusterSettings().get(EqlPlugin.DEFAULT_ALLOW_PARTIAL_SEQUENCE_RESULTS); + } + static EqlSearchResponse createResponse(Results results, AsyncExecutionId id) { EqlSearchResponse.Hits hits = new EqlSearchResponse.Hits(results.events(), results.sequences(), results.totalHits()); if (id != null) { - return new EqlSearchResponse(hits, results.tookTime().getMillis(), results.timedOut(), id.getEncoded(), false, false); + return new EqlSearchResponse( + hits, + results.tookTime().getMillis(), + results.timedOut(), + id.getEncoded(), + false, + false, + results.shardFailures() + ); } else { - return new EqlSearchResponse(hits, results.tookTime().getMillis(), results.timedOut()); + return new EqlSearchResponse(hits, results.tookTime().getMillis(), results.timedOut(), results.shardFailures()); } } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EmptyPayload.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EmptyPayload.java index 9822285465087..33ed5799cd073 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EmptyPayload.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EmptyPayload.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.eql.session; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.core.TimeValue; import java.util.List; @@ -17,14 +18,16 @@ public class EmptyPayload implements Payload { private final Type type; private final TimeValue timeTook; + private final ShardSearchFailure[] shardFailures; public EmptyPayload(Type type) { - this(type, TimeValue.ZERO); + this(type, TimeValue.ZERO, ShardSearchFailure.EMPTY_ARRAY); } - public EmptyPayload(Type type, TimeValue timeTook) { + public EmptyPayload(Type type, TimeValue timeTook, ShardSearchFailure[] shardFailures) { this.type = type; this.timeTook = timeTook; + this.shardFailures = shardFailures; } @Override @@ -46,4 +49,10 @@ public TimeValue timeTook() { public List values() { return emptyList(); } + + @Override + public ShardSearchFailure[] shardFailures() { + return shardFailures; + } + } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EqlConfiguration.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EqlConfiguration.java index 8dd8220fb63bc..8242b0b533ad3 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EqlConfiguration.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/EqlConfiguration.java @@ -30,6 +30,8 @@ public class EqlConfiguration extends org.elasticsearch.xpack.ql.session.Configu private final EqlSearchTask task; private final int fetchSize; private final int maxSamplesPerKey; + private final boolean allowPartialSearchResults; + private final boolean allowPartialSequenceResults; @Nullable private final QueryBuilder filter; @@ -50,6 +52,8 @@ public EqlConfiguration( IndicesOptions indicesOptions, int fetchSize, int maxSamplesPerKey, + boolean allowPartialSearchResults, + boolean allowPartialSequenceResults, String clientId, TaskId taskId, EqlSearchTask task @@ -67,6 +71,8 @@ public EqlConfiguration( this.task = task; this.fetchSize = fetchSize; this.maxSamplesPerKey = maxSamplesPerKey; + this.allowPartialSearchResults = allowPartialSearchResults; + this.allowPartialSequenceResults = allowPartialSequenceResults; } public String[] indices() { @@ -89,6 +95,14 @@ public int maxSamplesPerKey() { return maxSamplesPerKey; } + public boolean allowPartialSearchResults() { + return allowPartialSearchResults; + } + + public boolean allowPartialSequenceResults() { + return allowPartialSequenceResults; + } + public QueryBuilder filter() { return filter; } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/Payload.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/Payload.java index 1d82478e6db26..05e614714a5aa 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/Payload.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/Payload.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.eql.session; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.core.TimeValue; import java.util.List; @@ -29,4 +30,6 @@ enum Type { TimeValue timeTook(); List values(); + + ShardSearchFailure[] shardFailures(); } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/Results.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/Results.java index bb76c08c801cb..13886470f21f5 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/Results.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/session/Results.java @@ -9,6 +9,7 @@ import org.apache.lucene.search.TotalHits; import org.apache.lucene.search.TotalHits.Relation; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.core.TimeValue; import org.elasticsearch.xpack.eql.action.EqlSearchResponse.Event; import org.elasticsearch.xpack.eql.action.EqlSearchResponse.Sequence; @@ -23,18 +24,28 @@ public class Results { private final boolean timedOut; private final TimeValue tookTime; private final Type type; + private ShardSearchFailure[] shardFailures; public static Results fromPayload(Payload payload) { List values = payload.values(); - return new Results(new TotalHits(values.size(), Relation.EQUAL_TO), payload.timeTook(), false, values, payload.resultType()); + payload.shardFailures(); + return new Results( + new TotalHits(values.size(), Relation.EQUAL_TO), + payload.timeTook(), + false, + values, + payload.resultType(), + payload.shardFailures() + ); } - Results(TotalHits totalHits, TimeValue tookTime, boolean timedOut, List results, Type type) { + Results(TotalHits totalHits, TimeValue tookTime, boolean timedOut, List results, Type type, ShardSearchFailure[] shardFailures) { this.totalHits = totalHits; this.tookTime = tookTime; this.timedOut = timedOut; this.results = results; this.type = type; + this.shardFailures = shardFailures; } public TotalHits totalHits() { @@ -51,6 +62,10 @@ public List sequences() { return (type == Type.SEQUENCE || type == Type.SAMPLE) ? (List) results : null; } + public ShardSearchFailure[] shardFailures() { + return shardFailures; + } + public TimeValue tookTime() { return tookTime; } diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/util/SearchHitUtils.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/util/SearchHitUtils.java index 91795ac15b53e..2b5ec9718cfc4 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/util/SearchHitUtils.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/util/SearchHitUtils.java @@ -7,8 +7,12 @@ package org.elasticsearch.xpack.eql.util; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.search.SearchHit; +import java.util.Map; + import static org.elasticsearch.transport.RemoteClusterAware.buildRemoteIndexName; public final class SearchHitUtils { @@ -16,4 +20,12 @@ public final class SearchHitUtils { public static String qualifiedIndex(SearchHit hit) { return buildRemoteIndexName(hit.getClusterAlias(), hit.getIndex()); } + + public static void addShardFailures(Map shardFailures, SearchResponse r) { + if (r.getShardFailures() != null) { + for (ShardSearchFailure shardFailure : r.getShardFailures()) { + shardFailures.put(shardFailure.toString(), shardFailure); + } + } + } } diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/EqlTestUtils.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/EqlTestUtils.java index a1aa8e4bd98d7..75884fab4dbb3 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/EqlTestUtils.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/EqlTestUtils.java @@ -51,6 +51,8 @@ private EqlTestUtils() {} null, 123, 1, + false, + true, "", new TaskId("test", 123), null @@ -69,6 +71,8 @@ public static EqlConfiguration randomConfiguration() { randomIndicesOptions(), randomIntBetween(1, 1000), randomIntBetween(1, 1000), + randomBoolean(), + randomBoolean(), randomAlphaOfLength(16), new TaskId(randomAlphaOfLength(10), randomNonNegativeLong()), randomTask() diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlSearchRequestTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlSearchRequestTests.java index 0ff9fa9131b27..1a06aead910c8 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlSearchRequestTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlSearchRequestTests.java @@ -80,6 +80,8 @@ protected EqlSearchRequest createTestInstance() { .waitForCompletionTimeout(randomTimeValue()) .keepAlive(randomTimeValue()) .keepOnCompletion(randomBoolean()) + .allowPartialSearchResults(randomBoolean()) + .allowPartialSequenceResults(randomBoolean()) .fetchFields(randomFetchFields) .runtimeMappings(randomRuntimeMappings()) .resultPosition(randomFrom("tail", "head")) @@ -136,6 +138,12 @@ protected EqlSearchRequest mutateInstanceForVersion(EqlSearchRequest instance, T mutatedInstance.runtimeMappings(version.onOrAfter(TransportVersions.V_7_13_0) ? instance.runtimeMappings() : emptyMap()); mutatedInstance.resultPosition(version.onOrAfter(TransportVersions.V_7_17_8) ? instance.resultPosition() : "tail"); mutatedInstance.maxSamplesPerKey(version.onOrAfter(TransportVersions.V_8_7_0) ? instance.maxSamplesPerKey() : 1); + mutatedInstance.allowPartialSearchResults( + version.onOrAfter(TransportVersions.EQL_ALLOW_PARTIAL_SEARCH_RESULTS) ? instance.allowPartialSearchResults() : false + ); + mutatedInstance.allowPartialSequenceResults( + version.onOrAfter(TransportVersions.EQL_ALLOW_PARTIAL_SEARCH_RESULTS) ? instance.allowPartialSequenceResults() : false + ); return mutatedInstance; } diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlSearchResponseTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlSearchResponseTests.java index 6cb283d11848e..fa118a5256df1 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlSearchResponseTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/EqlSearchResponseTests.java @@ -9,6 +9,7 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; +import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.document.DocumentField; @@ -190,7 +191,7 @@ public static EqlSearchResponse createRandomEventsResponse(TotalHits totalHits, hits = new EqlSearchResponse.Hits(randomEvents(xType), null, totalHits); } if (randomBoolean()) { - return new EqlSearchResponse(hits, randomIntBetween(0, 1001), randomBoolean()); + return new EqlSearchResponse(hits, randomIntBetween(0, 1001), randomBoolean(), ShardSearchFailure.EMPTY_ARRAY); } else { return new EqlSearchResponse( hits, @@ -198,7 +199,8 @@ public static EqlSearchResponse createRandomEventsResponse(TotalHits totalHits, randomBoolean(), randomAlphaOfLength(10), randomBoolean(), - randomBoolean() + randomBoolean(), + ShardSearchFailure.EMPTY_ARRAY ); } } @@ -222,7 +224,7 @@ public static EqlSearchResponse createRandomSequencesResponse(TotalHits totalHit hits = new EqlSearchResponse.Hits(null, seq, totalHits); } if (randomBoolean()) { - return new EqlSearchResponse(hits, randomIntBetween(0, 1001), randomBoolean()); + return new EqlSearchResponse(hits, randomIntBetween(0, 1001), randomBoolean(), ShardSearchFailure.EMPTY_ARRAY); } else { return new EqlSearchResponse( hits, @@ -230,7 +232,8 @@ public static EqlSearchResponse createRandomSequencesResponse(TotalHits totalHit randomBoolean(), randomAlphaOfLength(10), randomBoolean(), - randomBoolean() + randomBoolean(), + ShardSearchFailure.EMPTY_ARRAY ); } } @@ -273,7 +276,8 @@ protected EqlSearchResponse mutateInstanceForVersion(EqlSearchResponse instance, instance.isTimeout(), instance.id(), instance.isRunning(), - instance.isPartial() + instance.isPartial(), + ShardSearchFailure.EMPTY_ARRAY ); } diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/LocalStateEQLXPackPlugin.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/LocalStateEQLXPackPlugin.java index 4d5201f544d72..33573b99546fb 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/LocalStateEQLXPackPlugin.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/action/LocalStateEQLXPackPlugin.java @@ -7,26 +7,41 @@ package org.elasticsearch.xpack.eql.action; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.indices.breaker.BreakerSettings; import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.plugins.CircuitBreakerPlugin; import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; import org.elasticsearch.xpack.eql.plugin.EqlPlugin; import org.elasticsearch.xpack.ql.plugin.QlPlugin; import java.nio.file.Path; -public class LocalStateEQLXPackPlugin extends LocalStateCompositeXPackPlugin { +public class LocalStateEQLXPackPlugin extends LocalStateCompositeXPackPlugin implements CircuitBreakerPlugin { + + private final EqlPlugin eqlPlugin; public LocalStateEQLXPackPlugin(final Settings settings, final Path configPath) { super(settings, configPath); LocalStateEQLXPackPlugin thisVar = this; - plugins.add(new EqlPlugin() { + this.eqlPlugin = new EqlPlugin() { @Override protected XPackLicenseState getLicenseState() { return thisVar.getLicenseState(); } - }); + }; + plugins.add(eqlPlugin); plugins.add(new QlPlugin()); } + @Override + public BreakerSettings getCircuitBreaker(Settings settings) { + return eqlPlugin.getCircuitBreaker(settings); + } + + @Override + public void setCircuitBreaker(CircuitBreaker circuitBreaker) { + eqlPlugin.setCircuitBreaker(circuitBreaker); + } } diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/assembler/ImplicitTiebreakerTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/assembler/ImplicitTiebreakerTests.java index 7bb6a228f6e48..abd928b04a9c7 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/assembler/ImplicitTiebreakerTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/assembler/ImplicitTiebreakerTests.java @@ -141,7 +141,15 @@ public void testImplicitTiebreakerBeingSet() { booleanArrayOf(stages, false), NOOP_CIRCUIT_BREAKER ); - TumblingWindow window = new TumblingWindow(client, criteria, null, matcher, Collections.emptyList()); + TumblingWindow window = new TumblingWindow( + client, + criteria, + null, + matcher, + Collections.emptyList(), + randomBoolean(), + randomBoolean() + ); window.execute(wrap(p -> {}, ex -> { throw ExceptionsHelper.convertToRuntime(ex); })); } } diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/assembler/SequenceSpecTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/assembler/SequenceSpecTests.java index a8ed842e94c44..f6aa851b2fff0 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/assembler/SequenceSpecTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/assembler/SequenceSpecTests.java @@ -277,7 +277,15 @@ public void test() throws Exception { ); QueryClient testClient = new TestQueryClient(); - TumblingWindow window = new TumblingWindow(testClient, criteria, null, matcher, Collections.emptyList()); + TumblingWindow window = new TumblingWindow( + testClient, + criteria, + null, + matcher, + Collections.emptyList(), + randomBoolean(), + randomBoolean() + ); // finally make the assertion at the end of the listener window.execute(ActionTestUtils.assertNoFailureListener(this::checkResults)); diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sample/CircuitBreakerTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sample/CircuitBreakerTests.java index dc132659417ff..80b1ff97b725d 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sample/CircuitBreakerTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sample/CircuitBreakerTests.java @@ -89,7 +89,7 @@ public void query(QueryRequest r, ActionListener l) {} @Override public void fetchHits(Iterable> refs, ActionListener>> listener) {} - }, mockCriteria(), randomIntBetween(10, 500), new Limit(1000, 0), CIRCUIT_BREAKER, 1); + }, mockCriteria(), randomIntBetween(10, 500), new Limit(1000, 0), CIRCUIT_BREAKER, 1, randomBoolean()); CIRCUIT_BREAKER.startBreaking(); iterator.pushToStack(new SampleIterator.Page(CB_STACK_SIZE_PRECISION - 1)); @@ -142,7 +142,8 @@ public void fetchHits(Iterable> refs, ActionListener> refs, ActionListener { // do nothing, we don't care about the query results }, ex -> { fail("Shouldn't have failed"); })); diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sequence/CircuitBreakerTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sequence/CircuitBreakerTests.java index fe1fca45364e3..58448d981fcca 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sequence/CircuitBreakerTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sequence/CircuitBreakerTests.java @@ -146,7 +146,15 @@ public void testCircuitBreakerTumblingWindow() { booleanArrayOf(stages, false), CIRCUIT_BREAKER ); - TumblingWindow window = new TumblingWindow(client, criteria, null, matcher, Collections.emptyList()); + TumblingWindow window = new TumblingWindow( + client, + criteria, + null, + matcher, + Collections.emptyList(), + randomBoolean(), + randomBoolean() + ); window.execute(ActionTestUtils.assertNoFailureListener(p -> {})); CIRCUIT_BREAKER.startBreaking(); @@ -228,7 +236,15 @@ private void assertMemoryCleared( booleanArrayOf(sequenceFiltersCount, false), eqlCircuitBreaker ); - TumblingWindow window = new TumblingWindow(eqlClient, criteria, null, matcher, Collections.emptyList()); + TumblingWindow window = new TumblingWindow( + eqlClient, + criteria, + null, + matcher, + Collections.emptyList(), + randomBoolean(), + randomBoolean() + ); window.execute(ActionListener.noop()); assertTrue(esClient.searchRequestsRemainingCount() == 0); // ensure all the search requests have been asked for @@ -271,7 +287,15 @@ public void testEqlCBCleanedUp_on_ParentCBBreak() { booleanArrayOf(sequenceFiltersCount, false), eqlCircuitBreaker ); - TumblingWindow window = new TumblingWindow(eqlClient, criteria, null, matcher, Collections.emptyList()); + TumblingWindow window = new TumblingWindow( + eqlClient, + criteria, + null, + matcher, + Collections.emptyList(), + randomBoolean(), + randomBoolean() + ); window.execute(wrap(p -> fail(), ex -> assertTrue(ex instanceof CircuitBreakingException))); } assertCriticalWarnings("[indices.breaker.total.limit] setting of [0%] is below the recommended minimum of 50.0% of the heap"); @@ -329,6 +353,8 @@ private QueryClient buildQueryClient(ESMockClient esClient, CircuitBreaker eqlCi null, 123, 1, + randomBoolean(), + randomBoolean(), "", new TaskId("test", 123), new EqlSearchTask( diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sequence/PITFailureTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sequence/PITFailureTests.java index 1a2f00463b49b..2eee6a262e73c 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sequence/PITFailureTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/execution/sequence/PITFailureTests.java @@ -83,6 +83,8 @@ public void testHandlingPitFailure() { null, 123, 1, + randomBoolean(), + randomBoolean(), "", new TaskId("test", 123), new EqlSearchTask( @@ -132,7 +134,15 @@ public void testHandlingPitFailure() { ); SequenceMatcher matcher = new SequenceMatcher(1, false, TimeValue.MINUS_ONE, null, booleanArrayOf(1, false), cb); - TumblingWindow window = new TumblingWindow(eqlClient, criteria, null, matcher, Collections.emptyList()); + TumblingWindow window = new TumblingWindow( + eqlClient, + criteria, + null, + matcher, + Collections.emptyList(), + randomBoolean(), + randomBoolean() + ); window.execute( wrap( p -> { fail("Search succeeded despite PIT failure"); }, From 51eb386a70e20bf17b42f44dda3e6a9bdd37321c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Mariusz=20J=C3=B3zala?= <377355+jozala@users.noreply.github.com> Date: Tue, 17 Dec 2024 17:48:57 +0100 Subject: [PATCH 33/44] [ci] Add ubuntu-2404-aarch64 to test matrix in platform-support (#118847) --- .buildkite/pipelines/periodic-platform-support.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.buildkite/pipelines/periodic-platform-support.yml b/.buildkite/pipelines/periodic-platform-support.yml index c5846a763f5e8..ea0d7b13b55b4 100644 --- a/.buildkite/pipelines/periodic-platform-support.yml +++ b/.buildkite/pipelines/periodic-platform-support.yml @@ -63,6 +63,7 @@ steps: image: - almalinux-8-aarch64 - ubuntu-2004-aarch64 + - ubuntu-2404-aarch64 GRADLE_TASK: - checkPart1 - checkPart2 From b8f4677254f7a0cd29c8ded0361f3ab201232cd5 Mon Sep 17 00:00:00 2001 From: Ievgen Degtiarenko Date: Tue, 17 Dec 2024 18:11:50 +0100 Subject: [PATCH 34/44] Replace build with scope parameter (#118643) This change replaces the build with scope parameter in order to make it cleaner when parameter is build and how it is used. --- .../org/elasticsearch/compute/ann/Fixed.java | 26 +++++++++++++------ .../compute/gen/EvaluatorImplementer.java | 15 ++++++++--- .../function/scalar/convert/FromBase64.java | 3 ++- .../function/scalar/convert/ToBase64.java | 3 ++- .../function/scalar/ip/IpPrefix.java | 3 ++- .../multivalue/MvPSeriesWeightedSum.java | 3 ++- .../scalar/multivalue/MvPercentile.java | 7 ++--- .../function/scalar/string/Concat.java | 3 ++- .../function/scalar/string/Left.java | 5 ++-- .../function/scalar/string/Repeat.java | 9 +++++-- .../function/scalar/string/Right.java | 5 ++-- .../function/scalar/string/Space.java | 3 ++- .../function/scalar/string/Split.java | 5 ++-- 13 files changed, 61 insertions(+), 29 deletions(-) diff --git a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Fixed.java b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Fixed.java index 62703fa400ff7..1f10abf3b9fb0 100644 --- a/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Fixed.java +++ b/x-pack/plugin/esql/compute/ann/src/main/java/org/elasticsearch/compute/ann/Fixed.java @@ -11,7 +11,6 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; -import java.util.function.Function; /** * Used on parameters on methods annotated with {@link Evaluator} to indicate @@ -27,12 +26,23 @@ boolean includeInToString() default true; /** - * Should the Evaluator's factory build this per evaluator with a - * {@code Function} or just take fixed implementation? - * This is typically set to {@code true} to use the {@link Function} - * to make "scratch" objects which have to be isolated in a single thread. - * This is typically set to {@code false} when the parameter is simply - * immutable and can be shared. + * Defines the scope of the parameter. + * - SINGLETON (default) will build a single instance and share it across all evaluators + * - THREAD_LOCAL will build a new instance for each evaluator thread */ - boolean build() default false; + Scope scope() default Scope.SINGLETON; + + /** + * Defines the parameter scope + */ + enum Scope { + /** + * Should be used for immutable parameters that can be shared across different threads + */ + SINGLETON, + /** + * Should be used for mutable or not thread safe parameters + */ + THREAD_LOCAL, + } } diff --git a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java index 5869eff23a9ab..b4a0cf9127f23 100644 --- a/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java +++ b/x-pack/plugin/esql/compute/gen/src/main/java/org/elasticsearch/compute/gen/EvaluatorImplementer.java @@ -16,6 +16,7 @@ import com.squareup.javapoet.TypeSpec; import org.elasticsearch.compute.ann.Fixed; +import org.elasticsearch.compute.ann.Fixed.Scope; import java.util.ArrayList; import java.util.Arrays; @@ -725,7 +726,7 @@ public String closeInvocation() { } } - private record FixedProcessFunctionArg(TypeName type, String name, boolean includeInToString, boolean build, boolean releasable) + private record FixedProcessFunctionArg(TypeName type, String name, boolean includeInToString, Scope scope, boolean releasable) implements ProcessFunctionArg { @Override @@ -762,12 +763,18 @@ public void implementFactoryCtor(MethodSpec.Builder builder) { } private TypeName factoryFieldType() { - return build ? ParameterizedTypeName.get(ClassName.get(Function.class), DRIVER_CONTEXT, type.box()) : type; + return switch (scope) { + case SINGLETON -> type; + case THREAD_LOCAL -> ParameterizedTypeName.get(ClassName.get(Function.class), DRIVER_CONTEXT, type.box()); + }; } @Override public String factoryInvocation(MethodSpec.Builder factoryMethodBuilder) { - return build ? name + ".apply(context)" : name; + return switch (scope) { + case SINGLETON -> name; + case THREAD_LOCAL -> name + ".apply(context)"; + }; } @Override @@ -1020,7 +1027,7 @@ private ProcessFunction( type, name, fixed.includeInToString(), - fixed.build(), + fixed.scope(), Types.extendsSuper(types, v.asType(), "org.elasticsearch.core.Releasable") ) ); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/FromBase64.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/FromBase64.java index 7f9d0d3f2e647..832c511a2dc50 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/FromBase64.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/FromBase64.java @@ -30,6 +30,7 @@ import java.util.Base64; import java.util.List; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; @@ -85,7 +86,7 @@ protected NodeInfo info() { } @Evaluator() - static BytesRef process(BytesRef field, @Fixed(includeInToString = false, build = true) BytesRefBuilder oScratch) { + static BytesRef process(BytesRef field, @Fixed(includeInToString = false, scope = THREAD_LOCAL) BytesRefBuilder oScratch) { byte[] bytes = new byte[field.length]; System.arraycopy(field.bytes, field.offset, bytes, 0, field.length); oScratch.grow(field.length); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBase64.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBase64.java index c23cef31f32f5..e78968bb209b6 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBase64.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/convert/ToBase64.java @@ -30,6 +30,7 @@ import java.util.Base64; import java.util.List; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; @@ -78,7 +79,7 @@ protected NodeInfo info() { } @Evaluator(warnExceptions = { ArithmeticException.class }) - static BytesRef process(BytesRef field, @Fixed(includeInToString = false, build = true) BytesRefBuilder oScratch) { + static BytesRef process(BytesRef field, @Fixed(includeInToString = false, scope = THREAD_LOCAL) BytesRefBuilder oScratch) { int outLength = Math.multiplyExact(4, (Math.addExact(field.length, 2) / 3)); byte[] bytes = new byte[field.length]; System.arraycopy(field.bytes, field.offset, bytes, 0, field.length); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/ip/IpPrefix.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/ip/IpPrefix.java index 26e75e752f681..5fc61c5c07b58 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/ip/IpPrefix.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/ip/IpPrefix.java @@ -30,6 +30,7 @@ import java.util.Arrays; import java.util.List; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.THIRD; @@ -138,7 +139,7 @@ static BytesRef process( BytesRef ip, int prefixLengthV4, int prefixLengthV6, - @Fixed(includeInToString = false, build = true) BytesRef scratch + @Fixed(includeInToString = false, scope = THREAD_LOCAL) BytesRef scratch ) { if (prefixLengthV4 < 0 || prefixLengthV4 > 32) { throw new IllegalArgumentException("Prefix length v4 must be in range [0, 32], found " + prefixLengthV4); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvPSeriesWeightedSum.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvPSeriesWeightedSum.java index cf49607893aae..4dd447f938880 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvPSeriesWeightedSum.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvPSeriesWeightedSum.java @@ -33,6 +33,7 @@ import java.util.Arrays; import java.util.List; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isFoldable; @@ -144,7 +145,7 @@ static void process( DoubleBlock.Builder builder, int position, DoubleBlock block, - @Fixed(includeInToString = false, build = true) CompensatedSum sum, + @Fixed(includeInToString = false, scope = THREAD_LOCAL) CompensatedSum sum, @Fixed double p ) { sum.reset(0, 0); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvPercentile.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvPercentile.java index f3a63c835bd34..4e4aee307f1c7 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvPercentile.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/multivalue/MvPercentile.java @@ -35,6 +35,7 @@ import java.util.Arrays; import java.util.List; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; @@ -167,7 +168,7 @@ static void process( int position, DoubleBlock values, double percentile, - @Fixed(includeInToString = false, build = true) DoubleSortingScratch scratch + @Fixed(includeInToString = false, scope = THREAD_LOCAL) DoubleSortingScratch scratch ) { int valueCount = values.getValueCount(position); int firstValueIndex = values.getFirstValueIndex(position); @@ -190,7 +191,7 @@ static void process( int position, IntBlock values, double percentile, - @Fixed(includeInToString = false, build = true) IntSortingScratch scratch + @Fixed(includeInToString = false, scope = THREAD_LOCAL) IntSortingScratch scratch ) { int valueCount = values.getValueCount(position); int firstValueIndex = values.getFirstValueIndex(position); @@ -213,7 +214,7 @@ static void process( int position, LongBlock values, double percentile, - @Fixed(includeInToString = false, build = true) LongSortingScratch scratch + @Fixed(includeInToString = false, scope = THREAD_LOCAL) LongSortingScratch scratch ) { int valueCount = values.getValueCount(position); int firstValueIndex = values.getFirstValueIndex(position); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Concat.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Concat.java index 46ecc9e026d3d..eb173029876d3 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Concat.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Concat.java @@ -32,6 +32,7 @@ import java.util.stream.Stream; import static org.elasticsearch.common.unit.ByteSizeUnit.MB; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.DEFAULT; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString; @@ -111,7 +112,7 @@ public ExpressionEvaluator.Factory toEvaluator(ToEvaluator toEvaluator) { } @Evaluator - static BytesRef process(@Fixed(includeInToString = false, build = true) BreakingBytesRefBuilder scratch, BytesRef[] values) { + static BytesRef process(@Fixed(includeInToString = false, scope = THREAD_LOCAL) BreakingBytesRefBuilder scratch, BytesRef[] values) { scratch.grow(checkedTotalLength(values)); scratch.clear(); for (int i = 0; i < values.length; i++) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Left.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Left.java index e7572caafd8f5..0d885e3f3c341 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Left.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Left.java @@ -30,6 +30,7 @@ import java.util.Arrays; import java.util.List; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString; @@ -77,8 +78,8 @@ public String getWriteableName() { @Evaluator static BytesRef process( - @Fixed(includeInToString = false, build = true) BytesRef out, - @Fixed(includeInToString = false, build = true) UnicodeUtil.UTF8CodePoint cp, + @Fixed(includeInToString = false, scope = THREAD_LOCAL) BytesRef out, + @Fixed(includeInToString = false, scope = THREAD_LOCAL) UnicodeUtil.UTF8CodePoint cp, BytesRef str, int length ) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Repeat.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Repeat.java index 2cc14399df2ae..e91f03de3dd7e 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Repeat.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Repeat.java @@ -31,6 +31,7 @@ import java.util.List; import static org.elasticsearch.common.unit.ByteSizeUnit.MB; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString; @@ -101,7 +102,7 @@ public boolean foldable() { @Evaluator(extraName = "Constant", warnExceptions = { IllegalArgumentException.class }) static BytesRef processConstantNumber( - @Fixed(includeInToString = false, build = true) BreakingBytesRefBuilder scratch, + @Fixed(includeInToString = false, scope = THREAD_LOCAL) BreakingBytesRefBuilder scratch, BytesRef str, @Fixed int number ) { @@ -109,7 +110,11 @@ static BytesRef processConstantNumber( } @Evaluator(warnExceptions = { IllegalArgumentException.class }) - static BytesRef process(@Fixed(includeInToString = false, build = true) BreakingBytesRefBuilder scratch, BytesRef str, int number) { + static BytesRef process( + @Fixed(includeInToString = false, scope = THREAD_LOCAL) BreakingBytesRefBuilder scratch, + BytesRef str, + int number + ) { if (number < 0) { throw new IllegalArgumentException("Number parameter cannot be negative, found [" + number + "]"); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Right.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Right.java index b069b984ea81e..e0ebed29cca72 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Right.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Right.java @@ -30,6 +30,7 @@ import java.util.Arrays; import java.util.List; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isString; @@ -77,8 +78,8 @@ public String getWriteableName() { @Evaluator static BytesRef process( - @Fixed(includeInToString = false, build = true) BytesRef out, - @Fixed(includeInToString = false, build = true) UnicodeUtil.UTF8CodePoint cp, + @Fixed(includeInToString = false, scope = THREAD_LOCAL) BytesRef out, + @Fixed(includeInToString = false, scope = THREAD_LOCAL) UnicodeUtil.UTF8CodePoint cp, BytesRef str, int length ) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Space.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Space.java index 6481ce5764e1f..3b9a466966911 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Space.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Space.java @@ -31,6 +31,7 @@ import java.util.List; import static org.elasticsearch.common.unit.ByteSizeUnit.MB; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.DEFAULT; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.isType; import static org.elasticsearch.xpack.esql.core.type.DataType.KEYWORD; @@ -82,7 +83,7 @@ protected TypeResolution resolveType() { } @Evaluator(warnExceptions = { IllegalArgumentException.class }) - static BytesRef process(@Fixed(includeInToString = false, build = true) BreakingBytesRefBuilder scratch, int number) { + static BytesRef process(@Fixed(includeInToString = false, scope = THREAD_LOCAL) BreakingBytesRefBuilder scratch, int number) { checkNumber(number); scratch.grow(number); scratch.setLength(number); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Split.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Split.java index b1f5da56d011b..24762122f755b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Split.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/function/scalar/string/Split.java @@ -29,6 +29,7 @@ import java.io.IOException; +import static org.elasticsearch.compute.ann.Fixed.Scope.THREAD_LOCAL; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.FIRST; import static org.elasticsearch.xpack.esql.core.expression.TypeResolutions.ParamOrdinal.SECOND; import static org.elasticsearch.xpack.esql.expression.EsqlTypeResolutions.isStringAndExact; @@ -110,7 +111,7 @@ static void process( BytesRefBlock.Builder builder, BytesRef str, @Fixed byte delim, - @Fixed(includeInToString = false, build = true) BytesRef scratch + @Fixed(includeInToString = false, scope = THREAD_LOCAL) BytesRef scratch ) { scratch.bytes = str.bytes; scratch.offset = str.offset; @@ -140,7 +141,7 @@ static void process( BytesRefBlock.Builder builder, BytesRef str, BytesRef delim, - @Fixed(includeInToString = false, build = true) BytesRef scratch + @Fixed(includeInToString = false, scope = THREAD_LOCAL) BytesRef scratch ) { checkDelimiter(delim); process(builder, str, delim.bytes[delim.offset], scratch); From d788761ea985a74aac90c6aee15fe32dea18e2ad Mon Sep 17 00:00:00 2001 From: Artem Prigoda Date: Tue, 17 Dec 2024 18:18:24 +0100 Subject: [PATCH 35/44] [test] Correctly mute MixedClusterClientYamlTestSuiteIT {cat.shards/10_basic/Help} (#116398) The `sync_id` field was removed from the `cat shards` output for 9.0 in #114246, this test shouldn't be run along with other 8.x clusters Resolve #116110 --- muted-tests.yml | 3 --- qa/mixed-cluster/build.gradle | 3 +++ 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 42845fda82180..93d1a6e6374b7 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -87,9 +87,6 @@ tests: issue: https://github.com/elastic/elasticsearch/issues/115816 - class: org.elasticsearch.xpack.application.connector.ConnectorIndexServiceTests issue: https://github.com/elastic/elasticsearch/issues/116087 -- class: org.elasticsearch.backwards.MixedClusterClientYamlTestSuiteIT - method: test {p0=cat.shards/10_basic/Help} - issue: https://github.com/elastic/elasticsearch/issues/116110 - class: org.elasticsearch.xpack.ml.integration.DatafeedJobsRestIT method: testLookbackWithIndicesOptions issue: https://github.com/elastic/elasticsearch/issues/116127 diff --git a/qa/mixed-cluster/build.gradle b/qa/mixed-cluster/build.gradle index d8f906b23d523..28bcac9f0242d 100644 --- a/qa/mixed-cluster/build.gradle +++ b/qa/mixed-cluster/build.gradle @@ -67,6 +67,9 @@ excludeList.add('indices.resolve_index/20_resolve_system_index/*') // Excluded because the error has changed excludeList.add('aggregations/percentiles_hdr_metric/Negative values test') +// sync_id is removed in 9.0 +excludeList.add("cat.shards/10_basic/Help") + def clusterPath = getPath() buildParams.bwcVersions.withWireCompatible { bwcVersion, baseName -> From 70087e3ded47e905dce9be09a969768345c0fa43 Mon Sep 17 00:00:00 2001 From: "elastic-renovate-prod[bot]" <174716857+elastic-renovate-prod[bot]@users.noreply.github.com> Date: Tue, 17 Dec 2024 17:58:19 +0000 Subject: [PATCH 36/44] Update docker.elastic.co/wolfi/chainguard-base:latest Docker digest to bfdeddb (#118868) Co-authored-by: elastic-renovate-prod[bot] <174716857+elastic-renovate-prod[bot]@users.noreply.github.com> --- .../main/java/org/elasticsearch/gradle/internal/DockerBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/DockerBase.java b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/DockerBase.java index d54eb798ce783..985c98bcd7883 100644 --- a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/DockerBase.java +++ b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/DockerBase.java @@ -22,7 +22,7 @@ public enum DockerBase { // Chainguard based wolfi image with latest jdk // This is usually updated via renovatebot // spotless:off - WOLFI("docker.elastic.co/wolfi/chainguard-base:latest@sha256:1b51ff6dba78c98d3e02b0cd64a8ce3238c7a40408d21e3af12a329d44db6f23", + WOLFI("docker.elastic.co/wolfi/chainguard-base:latest@sha256:bfdeddb33330a281950c2a54adef991dbbe6a42832bc505d13b11beaf50ae73f", "-wolfi", "apk" ), From f3a16649068799aabfdafa0bc0bacf71c4409687 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 17 Dec 2024 18:59:49 +0100 Subject: [PATCH 37/44] Add min. read-only index version compatible to DiscoveryNode (#118744) #118443 added a new index version for indices that can be opened in read-only mode by Lucene. This change adds this information to the discovery node's VersionInformation and the transport serialization logic. In a short future we'd like to use this information in methods like IndexMetadataVerifier#checkSupportedVersion and NodeJoineExecutor to allow opening indices in N-2 versions as read-only indices on ES V9. --- docs/reference/indices/shard-stores.asciidoc | 6 +- .../org/elasticsearch/TransportVersions.java | 1 + .../cluster/node/DiscoveryNode.java | 21 ++++++- .../cluster/node/DiscoveryNodes.java | 15 +++++ .../cluster/node/VersionInformation.java | 32 ++++++++-- .../HandshakingTransportAddressConnector.java | 1 + .../transport/ProxyConnectionStrategy.java | 1 + .../transport/SniffConnectionStrategy.java | 1 + .../reroute/ClusterRerouteResponseTests.java | 2 + .../cluster/ClusterStateTests.java | 6 ++ .../cluster/node/DiscoveryNodeTests.java | 61 +++++++++++++++++++ .../cluster/node/DiscoveryNodeUtils.java | 15 ++++- .../ClusterStatsMonitoringDocTests.java | 2 + 13 files changed, 152 insertions(+), 12 deletions(-) diff --git a/docs/reference/indices/shard-stores.asciidoc b/docs/reference/indices/shard-stores.asciidoc index 1b001a3175b8c..04b086a758f9d 100644 --- a/docs/reference/indices/shard-stores.asciidoc +++ b/docs/reference/indices/shard-stores.asciidoc @@ -172,8 +172,9 @@ The API returns the following response: "attributes": {}, "roles": [...], "version": "8.10.0", - "min_index_version": 7000099, - "max_index_version": 8100099 + "min_index_version": 8000099, + "min_read_only_index_version": 7000099, + "max_index_version": 9004000 }, "allocation_id": "2iNySv_OQVePRX-yaRH_lQ", <4> "allocation" : "primary|replica|unused" <5> @@ -193,6 +194,7 @@ The API returns the following response: // TESTRESPONSE[s/"roles": \[[^]]*\]/"roles": $body.$_path/] // TESTRESPONSE[s/"8.10.0"/\$node_version/] // TESTRESPONSE[s/"min_index_version": 7000099/"min_index_version": $body.$_path/] +// TESTRESPONSE[s/"min_index_version": 7000099/"min_index_version": $body.$_path/] // TESTRESPONSE[s/"max_index_version": 8100099/"max_index_version": $body.$_path/] diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 371af961720cc..d3e235f1cd82a 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -139,6 +139,7 @@ static TransportVersion def(int id) { public static final TransportVersion SEMANTIC_QUERY_LENIENT = def(8_807_00_0); public static final TransportVersion ESQL_QUERY_BUILDER_IN_SEARCH_FUNCTIONS = def(8_808_00_0); public static final TransportVersion EQL_ALLOW_PARTIAL_SEARCH_RESULTS = def(8_809_00_0); + public static final TransportVersion NODE_VERSION_INFORMATION_WITH_MIN_READ_ONLY_INDEX_VERSION = def(8_810_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java index 7bf367f99b929..7c757e7657853 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNode.java @@ -37,6 +37,7 @@ import java.util.SortedSet; import java.util.TreeSet; +import static org.elasticsearch.TransportVersions.NODE_VERSION_INFORMATION_WITH_MIN_READ_ONLY_INDEX_VERSION; import static org.elasticsearch.node.NodeRoleSettings.NODE_ROLES_SETTING; /** @@ -325,7 +326,17 @@ public DiscoveryNode(StreamInput in) throws IOException { } } this.roles = Collections.unmodifiableSortedSet(roles); - versionInfo = new VersionInformation(Version.readVersion(in), IndexVersion.readVersion(in), IndexVersion.readVersion(in)); + Version version = Version.readVersion(in); + IndexVersion minIndexVersion = IndexVersion.readVersion(in); + IndexVersion minReadOnlyIndexVersion; + if (in.getTransportVersion().onOrAfter(NODE_VERSION_INFORMATION_WITH_MIN_READ_ONLY_INDEX_VERSION)) { + minReadOnlyIndexVersion = IndexVersion.readVersion(in); + } else { + minReadOnlyIndexVersion = minIndexVersion; + + } + IndexVersion maxIndexVersion = IndexVersion.readVersion(in); + versionInfo = new VersionInformation(version, minIndexVersion, minReadOnlyIndexVersion, maxIndexVersion); if (in.getTransportVersion().onOrAfter(EXTERNAL_ID_VERSION)) { this.externalId = readStringLiteral.read(in); } else { @@ -360,6 +371,9 @@ public void writeTo(StreamOutput out) throws IOException { }); Version.writeVersion(versionInfo.nodeVersion(), out); IndexVersion.writeVersion(versionInfo.minIndexVersion(), out); + if (out.getTransportVersion().onOrAfter(NODE_VERSION_INFORMATION_WITH_MIN_READ_ONLY_INDEX_VERSION)) { + IndexVersion.writeVersion(versionInfo.minReadOnlyIndexVersion(), out); + } IndexVersion.writeVersion(versionInfo.maxIndexVersion(), out); if (out.getTransportVersion().onOrAfter(EXTERNAL_ID_VERSION)) { out.writeString(externalId); @@ -478,6 +492,10 @@ public IndexVersion getMinIndexVersion() { return versionInfo.minIndexVersion(); } + public IndexVersion getMinReadOnlyIndexVersion() { + return versionInfo.minReadOnlyIndexVersion(); + } + public IndexVersion getMaxIndexVersion() { return versionInfo.maxIndexVersion(); } @@ -577,6 +595,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endArray(); builder.field("version", versionInfo.buildVersion().toString()); builder.field("min_index_version", versionInfo.minIndexVersion()); + builder.field("min_read_only_index_version", versionInfo.minReadOnlyIndexVersion()); builder.field("max_index_version", versionInfo.maxIndexVersion()); builder.endObject(); return builder; diff --git a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java index 5e6dec7b68062..f733ab223fdd1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodes.java @@ -69,6 +69,7 @@ public class DiscoveryNodes implements Iterable, SimpleDiffable> tiersToNodeIds; @@ -84,6 +85,7 @@ private DiscoveryNodes( Version minNodeVersion, IndexVersion maxDataNodeCompatibleIndexVersion, IndexVersion minSupportedIndexVersion, + IndexVersion minReadOnlySupportedIndexVersion, Map> tiersToNodeIds ) { this.nodeLeftGeneration = nodeLeftGeneration; @@ -100,6 +102,8 @@ private DiscoveryNodes( this.maxNodeVersion = maxNodeVersion; this.maxDataNodeCompatibleIndexVersion = maxDataNodeCompatibleIndexVersion; this.minSupportedIndexVersion = minSupportedIndexVersion; + this.minReadOnlySupportedIndexVersion = minReadOnlySupportedIndexVersion; + assert minReadOnlySupportedIndexVersion.onOrBefore(minSupportedIndexVersion); assert (localNodeId == null) == (localNode == null); this.tiersToNodeIds = tiersToNodeIds; } @@ -118,6 +122,7 @@ public DiscoveryNodes withMasterNodeId(@Nullable String masterNodeId) { minNodeVersion, maxDataNodeCompatibleIndexVersion, minSupportedIndexVersion, + minReadOnlySupportedIndexVersion, tiersToNodeIds ); } @@ -374,6 +379,13 @@ public IndexVersion getMinSupportedIndexVersion() { return minSupportedIndexVersion; } + /** + * Returns the minimum index version for read-only indices supported by all nodes in the cluster + */ + public IndexVersion getMinReadOnlySupportedIndexVersion() { + return minReadOnlySupportedIndexVersion; + } + /** * Return the node-left generation, which is the number of times the cluster membership has been updated by removing one or more nodes. *

@@ -840,6 +852,7 @@ public DiscoveryNodes build() { Version maxNodeVersion = null; IndexVersion maxDataNodeCompatibleIndexVersion = null; IndexVersion minSupportedIndexVersion = null; + IndexVersion minReadOnlySupportedIndexVersion = null; for (Map.Entry nodeEntry : nodes.entrySet()) { DiscoveryNode discoNode = nodeEntry.getValue(); Version version = discoNode.getVersion(); @@ -849,6 +862,7 @@ public DiscoveryNodes build() { minNodeVersion = min(minNodeVersion, version); maxNodeVersion = max(maxNodeVersion, version); minSupportedIndexVersion = max(minSupportedIndexVersion, discoNode.getMinIndexVersion()); + minReadOnlySupportedIndexVersion = max(minReadOnlySupportedIndexVersion, discoNode.getMinReadOnlyIndexVersion()); } final long newNodeLeftGeneration; @@ -881,6 +895,7 @@ public DiscoveryNodes build() { Objects.requireNonNullElse(minNodeVersion, Version.CURRENT.minimumCompatibilityVersion()), Objects.requireNonNullElse(maxDataNodeCompatibleIndexVersion, IndexVersion.current()), Objects.requireNonNullElse(minSupportedIndexVersion, IndexVersions.MINIMUM_COMPATIBLE), + Objects.requireNonNullElse(minReadOnlySupportedIndexVersion, IndexVersions.MINIMUM_READONLY_COMPATIBLE), computeTiersToNodesMap(dataNodes) ); } diff --git a/server/src/main/java/org/elasticsearch/cluster/node/VersionInformation.java b/server/src/main/java/org/elasticsearch/cluster/node/VersionInformation.java index a4d0ff1eb55e4..852f31db69c92 100644 --- a/server/src/main/java/org/elasticsearch/cluster/node/VersionInformation.java +++ b/server/src/main/java/org/elasticsearch/cluster/node/VersionInformation.java @@ -18,20 +18,23 @@ /** * Represents the versions of various aspects of an Elasticsearch node. - * @param buildVersion The node {@link BuildVersion} - * @param minIndexVersion The minimum {@link IndexVersion} supported by this node - * @param maxIndexVersion The maximum {@link IndexVersion} supported by this node + * @param buildVersion The node {@link BuildVersion} + * @param minIndexVersion The minimum {@link IndexVersion} supported by this node + * @param minReadOnlyIndexVersion The minimum {@link IndexVersion} for read-only indices supported by this node + * @param maxIndexVersion The maximum {@link IndexVersion} supported by this node */ public record VersionInformation( BuildVersion buildVersion, Version nodeVersion, IndexVersion minIndexVersion, + IndexVersion minReadOnlyIndexVersion, IndexVersion maxIndexVersion ) { public static final VersionInformation CURRENT = new VersionInformation( BuildVersion.current(), IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current() ); @@ -39,11 +42,18 @@ public record VersionInformation( Objects.requireNonNull(buildVersion); Objects.requireNonNull(nodeVersion); Objects.requireNonNull(minIndexVersion); + Objects.requireNonNull(minReadOnlyIndexVersion); Objects.requireNonNull(maxIndexVersion); + assert minReadOnlyIndexVersion.onOrBefore(minIndexVersion) : minReadOnlyIndexVersion + " > " + minIndexVersion; } - public VersionInformation(BuildVersion version, IndexVersion minIndexVersion, IndexVersion maxIndexVersion) { - this(version, Version.CURRENT, minIndexVersion, maxIndexVersion); + public VersionInformation( + BuildVersion version, + IndexVersion minIndexVersion, + IndexVersion minReadOnlyIndexVersion, + IndexVersion maxIndexVersion + ) { + this(version, Version.CURRENT, minIndexVersion, minReadOnlyIndexVersion, maxIndexVersion); /* * Whilst DiscoveryNode.getVersion exists, we need to be able to get a Version from VersionInfo * This needs to be consistent - on serverless, BuildVersion has an id of -1, which translates @@ -57,7 +67,17 @@ public VersionInformation(BuildVersion version, IndexVersion minIndexVersion, In @Deprecated public VersionInformation(Version version, IndexVersion minIndexVersion, IndexVersion maxIndexVersion) { - this(BuildVersion.fromVersionId(version.id()), version, minIndexVersion, maxIndexVersion); + this(version, minIndexVersion, minIndexVersion, maxIndexVersion); + } + + @Deprecated + public VersionInformation( + Version version, + IndexVersion minIndexVersion, + IndexVersion minReadOnlyIndexVersion, + IndexVersion maxIndexVersion + ) { + this(BuildVersion.fromVersionId(version.id()), version, minIndexVersion, minReadOnlyIndexVersion, maxIndexVersion); } @Deprecated diff --git a/server/src/main/java/org/elasticsearch/discovery/HandshakingTransportAddressConnector.java b/server/src/main/java/org/elasticsearch/discovery/HandshakingTransportAddressConnector.java index ce849c26ab780..98715127351aa 100644 --- a/server/src/main/java/org/elasticsearch/discovery/HandshakingTransportAddressConnector.java +++ b/server/src/main/java/org/elasticsearch/discovery/HandshakingTransportAddressConnector.java @@ -110,6 +110,7 @@ private void openProbeConnection(ActionListener listener) new VersionInformation( Version.CURRENT.minimumCompatibilityVersion(), IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current() ) ), diff --git a/server/src/main/java/org/elasticsearch/transport/ProxyConnectionStrategy.java b/server/src/main/java/org/elasticsearch/transport/ProxyConnectionStrategy.java index d5047a61e4606..eb2eab75d3fe3 100644 --- a/server/src/main/java/org/elasticsearch/transport/ProxyConnectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/transport/ProxyConnectionStrategy.java @@ -303,6 +303,7 @@ public void onFailure(Exception e) { new VersionInformation( Version.CURRENT.minimumCompatibilityVersion(), IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current() ) ); diff --git a/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java b/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java index 2c198caf22354..854072c49e354 100644 --- a/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java @@ -505,6 +505,7 @@ private static DiscoveryNode resolveSeedNode(String clusterAlias, String address var seedVersion = new VersionInformation( Version.CURRENT.minimumCompatibilityVersion(), IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current() ); if (proxyAddress == null || proxyAddress.isEmpty()) { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponseTests.java index b59cc13a20ff2..69cff0fc45ac3 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/reroute/ClusterRerouteResponseTests.java @@ -127,6 +127,7 @@ public void testToXContentWithDeprecatedClusterState() { ], "version": "%s", "min_index_version": %s, + "min_read_only_index_version": %s, "max_index_version": %s } }, @@ -218,6 +219,7 @@ public void testToXContentWithDeprecatedClusterState() { clusterState.getNodes().get("node0").getEphemeralId(), Version.CURRENT, IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current(), IndexVersion.current(), IndexVersion.current() diff --git a/server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java b/server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java index 668aea70c23f2..5f4426b02ce1a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/ClusterStateTests.java @@ -213,6 +213,7 @@ public void testToXContent() throws IOException { ], "version": "%s", "min_index_version":%s, + "min_read_only_index_version":%s, "max_index_version":%s } }, @@ -389,6 +390,7 @@ public void testToXContent() throws IOException { ephemeralId, Version.CURRENT, IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current(), TransportVersion.current(), IndexVersion.current(), @@ -488,6 +490,7 @@ public void testToXContent_FlatSettingTrue_ReduceMappingFalse() throws IOExcepti ], "version" : "%s", "min_index_version" : %s, + "min_read_only_index_version" : %s, "max_index_version" : %s } }, @@ -663,6 +666,7 @@ public void testToXContent_FlatSettingTrue_ReduceMappingFalse() throws IOExcepti ephemeralId, Version.CURRENT, IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current(), TransportVersion.current(), IndexVersion.current(), @@ -762,6 +766,7 @@ public void testToXContent_FlatSettingFalse_ReduceMappingTrue() throws IOExcepti ], "version" : "%s", "min_index_version" : %s, + "min_read_only_index_version" : %s, "max_index_version" : %s } }, @@ -943,6 +948,7 @@ public void testToXContent_FlatSettingFalse_ReduceMappingTrue() throws IOExcepti ephemeralId, Version.CURRENT, IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current(), TransportVersion.current(), IndexVersion.current(), diff --git a/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java b/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java index 331b5d92ca94e..fa7633f0eaf75 100644 --- a/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/node/DiscoveryNodeTests.java @@ -31,6 +31,8 @@ import static java.util.Collections.emptySet; import static org.elasticsearch.test.NodeRoles.nonRemoteClusterClientNode; import static org.elasticsearch.test.NodeRoles.remoteClusterClientNode; +import static org.elasticsearch.test.TransportVersionUtils.getPreviousVersion; +import static org.elasticsearch.test.TransportVersionUtils.randomVersionBetween; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -221,6 +223,7 @@ public void testDiscoveryNodeToXContent() { ], "version" : "%s", "min_index_version" : %s, + "min_read_only_index_version" : %s, "max_index_version" : %s } }""", @@ -228,6 +231,7 @@ public void testDiscoveryNodeToXContent() { withExternalId ? "test-external-id" : "test-name", Version.CURRENT, IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current() ) ) @@ -250,4 +254,61 @@ public void testDiscoveryNodeToString() { assertThat(toString, containsString("{" + node.getBuildVersion() + "}")); assertThat(toString, containsString("{test-attr=val}"));// attributes } + + public void testDiscoveryNodeMinReadOnlyVersionSerialization() throws Exception { + var node = DiscoveryNodeUtils.create("_id", buildNewFakeTransportAddress(), VersionInformation.CURRENT); + + { + try (var out = new BytesStreamOutput()) { + out.setTransportVersion(TransportVersion.current()); + node.writeTo(out); + + try (var in = StreamInput.wrap(out.bytes().array())) { + in.setTransportVersion(TransportVersion.current()); + + var deserialized = new DiscoveryNode(in); + assertThat(deserialized.getId(), equalTo(node.getId())); + assertThat(deserialized.getAddress(), equalTo(node.getAddress())); + assertThat(deserialized.getMinIndexVersion(), equalTo(node.getMinIndexVersion())); + assertThat(deserialized.getMaxIndexVersion(), equalTo(node.getMaxIndexVersion())); + assertThat(deserialized.getMinReadOnlyIndexVersion(), equalTo(node.getMinReadOnlyIndexVersion())); + assertThat(deserialized.getVersionInformation(), equalTo(node.getVersionInformation())); + } + } + } + + { + var oldVersion = randomVersionBetween( + random(), + TransportVersions.MINIMUM_COMPATIBLE, + getPreviousVersion(TransportVersions.NODE_VERSION_INFORMATION_WITH_MIN_READ_ONLY_INDEX_VERSION) + ); + try (var out = new BytesStreamOutput()) { + out.setTransportVersion(oldVersion); + node.writeTo(out); + + try (var in = StreamInput.wrap(out.bytes().array())) { + in.setTransportVersion(oldVersion); + + var deserialized = new DiscoveryNode(in); + assertThat(deserialized.getId(), equalTo(node.getId())); + assertThat(deserialized.getAddress(), equalTo(node.getAddress())); + assertThat(deserialized.getMinIndexVersion(), equalTo(node.getMinIndexVersion())); + assertThat(deserialized.getMaxIndexVersion(), equalTo(node.getMaxIndexVersion())); + assertThat(deserialized.getMinReadOnlyIndexVersion(), equalTo(node.getMinIndexVersion())); + assertThat( + deserialized.getVersionInformation(), + equalTo( + new VersionInformation( + node.getBuildVersion(), + node.getMinIndexVersion(), + node.getMinIndexVersion(), + node.getMaxIndexVersion() + ) + ) + ); + } + } + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeUtils.java b/test/framework/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeUtils.java index 64f8fa88762b8..20368753eac1d 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeUtils.java @@ -76,6 +76,7 @@ public static class Builder { private BuildVersion buildVersion; private Version version; private IndexVersion minIndexVersion; + private IndexVersion minReadOnlyIndexVersion; private IndexVersion maxIndexVersion; private String externalId; @@ -125,16 +126,23 @@ public Builder version(Version version, IndexVersion minIndexVersion, IndexVersi this.buildVersion = BuildVersion.fromVersionId(version.id()); this.version = version; this.minIndexVersion = minIndexVersion; + this.minReadOnlyIndexVersion = minIndexVersion; this.maxIndexVersion = maxIndexVersion; return this; } - public Builder version(BuildVersion version, IndexVersion minIndexVersion, IndexVersion maxIndexVersion) { + public Builder version( + BuildVersion version, + IndexVersion minIndexVersion, + IndexVersion minReadOnlyIndexVersion, + IndexVersion maxIndexVersion + ) { // see comment in VersionInformation assert version.equals(BuildVersion.current()); this.buildVersion = version; this.version = Version.CURRENT; this.minIndexVersion = minIndexVersion; + this.minReadOnlyIndexVersion = minReadOnlyIndexVersion; this.maxIndexVersion = maxIndexVersion; return this; } @@ -143,6 +151,7 @@ public Builder version(VersionInformation versions) { this.buildVersion = versions.buildVersion(); this.version = versions.nodeVersion(); this.minIndexVersion = versions.minIndexVersion(); + this.minReadOnlyIndexVersion = versions.minReadOnlyIndexVersion(); this.maxIndexVersion = versions.maxIndexVersion(); return this; } @@ -170,10 +179,10 @@ public DiscoveryNode build() { } VersionInformation versionInfo; - if (minIndexVersion == null || maxIndexVersion == null) { + if (minIndexVersion == null || minReadOnlyIndexVersion == null || maxIndexVersion == null) { versionInfo = VersionInformation.inferVersions(version); } else { - versionInfo = new VersionInformation(buildVersion, version, minIndexVersion, maxIndexVersion); + versionInfo = new VersionInformation(buildVersion, version, minIndexVersion, minReadOnlyIndexVersion, maxIndexVersion); } return new DiscoveryNode(name, id, ephemeralId, hostName, hostAddress, address, attributes, roles, versionInfo, externalId); diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java index f4d50df4ff613..35da4abec223a 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/collector/cluster/ClusterStatsMonitoringDocTests.java @@ -462,6 +462,7 @@ public void testToXContent() throws IOException { pluginEsBuildVersion, Version.CURRENT, IndexVersions.MINIMUM_COMPATIBLE, + IndexVersions.MINIMUM_READONLY_COMPATIBLE, IndexVersion.current(), apmIndicesExist }; final String expectedJson = """ @@ -817,6 +818,7 @@ public void testToXContent() throws IOException { ], "version": "%s", "min_index_version":%s, + "min_read_only_index_version":%s, "max_index_version":%s } }, From a5c57ba966cfd088b8d79fd51fe3fb35163b22a2 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Tue, 17 Dec 2024 13:30:48 -0500 Subject: [PATCH 38/44] Adjust random_score default field to _seq_no field (#118671) In an effort to improve performance and continue to provide unique seeded scores for documents in the same index, we are switching from _id to _seq_no. Requiring a field that is "unique" for a field and to help with random scores is burdensome for the user. So, we should default to a unique field (per index) when the user provides a seed. Using `_seq_no` should be better as: - We don't have to grab stored fields values - Bytes used are generally smaller Additionally this removes the deprecation warning. Marking as "breaking" as it does change the scores & behavior, but the API provide is the same. --- .../src/main/resources/changelog-schema.json | 1 + docs/changelog/118671.yaml | 11 +++++++++++ .../RandomScoreFunctionBuilder.java | 17 ++--------------- .../ScoreFunctionBuilderTests.java | 1 - 4 files changed, 14 insertions(+), 16 deletions(-) create mode 100644 docs/changelog/118671.yaml diff --git a/build-tools-internal/src/main/resources/changelog-schema.json b/build-tools-internal/src/main/resources/changelog-schema.json index 451701d74d690..7d35951eaa2cf 100644 --- a/build-tools-internal/src/main/resources/changelog-schema.json +++ b/build-tools-internal/src/main/resources/changelog-schema.json @@ -295,6 +295,7 @@ "Painless", "REST API", "Rollup", + "Search", "System requirement", "Transform" ] diff --git a/docs/changelog/118671.yaml b/docs/changelog/118671.yaml new file mode 100644 index 0000000000000..3931cc4179037 --- /dev/null +++ b/docs/changelog/118671.yaml @@ -0,0 +1,11 @@ +pr: 118671 +summary: Adjust `random_score` default field to `_seq_no` field +area: Search +type: breaking +issues: [] +breaking: + title: Adjust `random_score` default field to `_seq_no` field + area: Search + details: When providing a 'seed' parameter to a 'random_score' function in the 'function_score' query but NOT providing a 'field', the default 'field' is switched from '_id' to '_seq_no'. + impact: The random scoring and ordering may change when providing a 'seed' and not providing a 'field' to a 'random_score' function. + notable: false diff --git a/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java b/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java index 6d4b2dd4ab1f5..88f1ab1ba5c2e 100644 --- a/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/functionscore/RandomScoreFunctionBuilder.java @@ -13,12 +13,10 @@ import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.logging.DeprecationCategory; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.lucene.search.function.RandomScoreFunction; import org.elasticsearch.common.lucene.search.function.ScoreFunction; -import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentParser; @@ -30,7 +28,6 @@ * A function that computes a random score for the matched documents */ public class RandomScoreFunctionBuilder extends ScoreFunctionBuilder { - private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(RandomScoreFunctionBuilder.class); public static final String NAME = "random_score"; private String field; @@ -140,17 +137,7 @@ protected ScoreFunction doToFunction(SearchExecutionContext context) { // DocID-based random score generation return new RandomScoreFunction(hash(context.nowInMillis()), salt, null); } else { - String fieldName; - if (field == null) { - deprecationLogger.warn( - DeprecationCategory.QUERIES, - "seed_requires_field", - "As of version 7.0 Elasticsearch will require that a [field] parameter is provided when a [seed] is set" - ); - fieldName = IdFieldMapper.NAME; - } else { - fieldName = field; - } + final String fieldName = Objects.requireNonNullElse(field, SeqNoFieldMapper.NAME); if (context.isFieldMapped(fieldName) == false) { if (context.hasMappings() == false) { // no mappings: the index is empty anyway diff --git a/server/src/test/java/org/elasticsearch/index/query/functionscore/ScoreFunctionBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/functionscore/ScoreFunctionBuilderTests.java index 8d060d94e4c21..b58ac513a6449 100644 --- a/server/src/test/java/org/elasticsearch/index/query/functionscore/ScoreFunctionBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/functionscore/ScoreFunctionBuilderTests.java @@ -66,7 +66,6 @@ public void testRandomScoreFunctionWithSeedNoField() throws Exception { Mockito.when(context.getFieldType(IdFieldMapper.NAME)).thenReturn(new KeywordFieldMapper.KeywordFieldType(IdFieldMapper.NAME)); Mockito.when(context.isFieldMapped(IdFieldMapper.NAME)).thenReturn(true); builder.toFunction(context); - assertWarnings("As of version 7.0 Elasticsearch will require that a [field] parameter is provided when a [seed] is set"); } public void testRandomScoreFunctionWithSeed() throws Exception { From bac70647260ba0396a904b918a3dd3576c757a0d Mon Sep 17 00:00:00 2001 From: Brian Seeders Date: Tue, 17 Dec 2024 13:34:26 -0500 Subject: [PATCH 39/44] Bump versions after 8.16.2 release --- .buildkite/pipelines/intake.yml | 2 +- .buildkite/pipelines/periodic-packaging.yml | 6 +++--- .buildkite/pipelines/periodic.yml | 10 +++++----- .ci/bwcVersions | 2 +- .ci/snapshotBwcVersions | 2 +- server/src/main/java/org/elasticsearch/Version.java | 1 + .../resources/org/elasticsearch/TransportVersions.csv | 1 + .../org/elasticsearch/index/IndexVersions.csv | 1 + 8 files changed, 14 insertions(+), 11 deletions(-) diff --git a/.buildkite/pipelines/intake.yml b/.buildkite/pipelines/intake.yml index 6e15d64154960..9efb9c8b498aa 100644 --- a/.buildkite/pipelines/intake.yml +++ b/.buildkite/pipelines/intake.yml @@ -56,7 +56,7 @@ steps: timeout_in_minutes: 300 matrix: setup: - BWC_VERSION: ["8.16.2", "8.17.1", "8.18.0", "9.0.0"] + BWC_VERSION: ["8.16.3", "8.17.1", "8.18.0", "9.0.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 diff --git a/.buildkite/pipelines/periodic-packaging.yml b/.buildkite/pipelines/periodic-packaging.yml index abd11068e7a65..b1e5a7bf933c9 100644 --- a/.buildkite/pipelines/periodic-packaging.yml +++ b/.buildkite/pipelines/periodic-packaging.yml @@ -287,8 +287,8 @@ steps: env: BWC_VERSION: 8.15.5 - - label: "{{matrix.image}} / 8.16.2 / packaging-tests-upgrade" - command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.16.2 + - label: "{{matrix.image}} / 8.16.3 / packaging-tests-upgrade" + command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.16.3 timeout_in_minutes: 300 matrix: setup: @@ -301,7 +301,7 @@ steps: machineType: custom-16-32768 buildDirectory: /dev/shm/bk env: - BWC_VERSION: 8.16.2 + BWC_VERSION: 8.16.3 - label: "{{matrix.image}} / 8.17.1 / packaging-tests-upgrade" command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.17.1 diff --git a/.buildkite/pipelines/periodic.yml b/.buildkite/pipelines/periodic.yml index f2d169cd2b30d..4c593bae62d7a 100644 --- a/.buildkite/pipelines/periodic.yml +++ b/.buildkite/pipelines/periodic.yml @@ -306,8 +306,8 @@ steps: - signal_reason: agent_stop limit: 3 - - label: 8.16.2 / bwc - command: .ci/scripts/run-gradle.sh -Dbwc.checkout.align=true v8.16.2#bwcTest + - label: 8.16.3 / bwc + command: .ci/scripts/run-gradle.sh -Dbwc.checkout.align=true v8.16.3#bwcTest timeout_in_minutes: 300 agents: provider: gcp @@ -316,7 +316,7 @@ steps: buildDirectory: /dev/shm/bk preemptible: true env: - BWC_VERSION: 8.16.2 + BWC_VERSION: 8.16.3 retry: automatic: - exit_status: "-1" @@ -448,7 +448,7 @@ steps: setup: ES_RUNTIME_JAVA: - openjdk21 - BWC_VERSION: ["8.16.2", "8.17.1", "8.18.0", "9.0.0"] + BWC_VERSION: ["8.16.3", "8.17.1", "8.18.0", "9.0.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 @@ -490,7 +490,7 @@ steps: ES_RUNTIME_JAVA: - openjdk21 - openjdk23 - BWC_VERSION: ["8.16.2", "8.17.1", "8.18.0", "9.0.0"] + BWC_VERSION: ["8.16.3", "8.17.1", "8.18.0", "9.0.0"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 diff --git a/.ci/bwcVersions b/.ci/bwcVersions index 3cb983373138f..cf12ee8c15419 100644 --- a/.ci/bwcVersions +++ b/.ci/bwcVersions @@ -15,7 +15,7 @@ BWC_VERSION: - "8.13.4" - "8.14.3" - "8.15.5" - - "8.16.2" + - "8.16.3" - "8.17.1" - "8.18.0" - "9.0.0" diff --git a/.ci/snapshotBwcVersions b/.ci/snapshotBwcVersions index e05c0774c9819..68c6ad5601546 100644 --- a/.ci/snapshotBwcVersions +++ b/.ci/snapshotBwcVersions @@ -1,5 +1,5 @@ BWC_VERSION: - - "8.16.2" + - "8.16.3" - "8.17.1" - "8.18.0" - "9.0.0" diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index 47c43eadcfb03..8873c9b0e281e 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -191,6 +191,7 @@ public class Version implements VersionId, ToXContentFragment { public static final Version V_8_16_0 = new Version(8_16_00_99); public static final Version V_8_16_1 = new Version(8_16_01_99); public static final Version V_8_16_2 = new Version(8_16_02_99); + public static final Version V_8_16_3 = new Version(8_16_03_99); public static final Version V_8_17_0 = new Version(8_17_00_99); public static final Version V_8_17_1 = new Version(8_17_01_99); public static final Version V_8_18_0 = new Version(8_18_00_99); diff --git a/server/src/main/resources/org/elasticsearch/TransportVersions.csv b/server/src/main/resources/org/elasticsearch/TransportVersions.csv index 08db0822dfef5..2016f59b58a3e 100644 --- a/server/src/main/resources/org/elasticsearch/TransportVersions.csv +++ b/server/src/main/resources/org/elasticsearch/TransportVersions.csv @@ -135,4 +135,5 @@ 8.15.5,8702003 8.16.0,8772001 8.16.1,8772004 +8.16.2,8772004 8.17.0,8797002 diff --git a/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv b/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv index afe696f31d323..3bfeeded6494c 100644 --- a/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv +++ b/server/src/main/resources/org/elasticsearch/index/IndexVersions.csv @@ -135,4 +135,5 @@ 8.15.5,8512000 8.16.0,8518000 8.16.1,8518000 +8.16.2,8518000 8.17.0,8521000 From 5cb2d280201020c4246afa428095ae2e24c20be2 Mon Sep 17 00:00:00 2001 From: Brian Seeders Date: Tue, 17 Dec 2024 13:35:51 -0500 Subject: [PATCH 40/44] Prune changelogs after 8.16.2 release --- docs/changelog/116358.yaml | 5 ----- docs/changelog/117153.yaml | 5 ----- docs/changelog/118380.yaml | 5 ----- 3 files changed, 15 deletions(-) delete mode 100644 docs/changelog/116358.yaml delete mode 100644 docs/changelog/117153.yaml delete mode 100644 docs/changelog/118380.yaml diff --git a/docs/changelog/116358.yaml b/docs/changelog/116358.yaml deleted file mode 100644 index 58b44a1e9bcf5..0000000000000 --- a/docs/changelog/116358.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 116358 -summary: Update Deberta tokenizer -area: Machine Learning -type: bug -issues: [] diff --git a/docs/changelog/117153.yaml b/docs/changelog/117153.yaml deleted file mode 100644 index f7640c0a7ed6a..0000000000000 --- a/docs/changelog/117153.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 117153 -summary: "ESQL: fix the column position in errors" -area: ES|QL -type: bug -issues: [] diff --git a/docs/changelog/118380.yaml b/docs/changelog/118380.yaml deleted file mode 100644 index 8b26c871fb172..0000000000000 --- a/docs/changelog/118380.yaml +++ /dev/null @@ -1,5 +0,0 @@ -pr: 118380 -summary: Restore original "is within leaf" value in `SparseVectorFieldMapper` -area: Mapping -type: bug -issues: [] From bde485a84520539f98d3f08913af4b8b73434e6b Mon Sep 17 00:00:00 2001 From: Valeriy Khakhutskyy <1292899+valeriy42@users.noreply.github.com> Date: Tue, 17 Dec 2024 20:23:00 +0100 Subject: [PATCH 41/44] [ML] Add dynamic templates for anomalies results index (#118845) This PR enables the resolution of field names containing "." after the rollover of the anomalies results index. This way, the object parts of the field name, e.g., for an influencer, will indeed have the object type and can be resolved in search and filter operations. --- .../ml/anomalydetection/results_index_mappings.json | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/core/template-resources/src/main/resources/ml/anomalydetection/results_index_mappings.json b/x-pack/plugin/core/template-resources/src/main/resources/ml/anomalydetection/results_index_mappings.json index 4415afe50a998..e0bde4715839e 100644 --- a/x-pack/plugin/core/template-resources/src/main/resources/ml/anomalydetection/results_index_mappings.json +++ b/x-pack/plugin/core/template-resources/src/main/resources/ml/anomalydetection/results_index_mappings.json @@ -5,7 +5,15 @@ }, "dynamic_templates" : [ { - "strings_as_keywords" : { + "map_objects": { + "match_mapping_type": "object", + "mapping": { + "type": "object" + } + } + }, + { + "non_objects_as_keywords" : { "match" : "*", "mapping" : { "type" : "keyword" From 4b90f01b49b820242608d54fae6b624fe950847d Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Tue, 17 Dec 2024 14:36:02 -0500 Subject: [PATCH 42/44] Esql - fix some test failures due to results ordering (#118692) fix some failing multi-shard tests. The backport for the PR that added these tests hasn't landed yet, so I'm going to just cherry pick this into there rather than do two back ports with conflict resolution. --- .../src/main/resources/date_nanos.csv-spec | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec index f4b5c98d596ae..47191148e0205 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec @@ -555,7 +555,8 @@ required_capability: date_nanos_bucket FROM date_nanos | WHERE millis > "2020-01-01" -| STATS ct = count(*) BY yr = BUCKET(nanos, 1 year); +| STATS ct = count(*) BY yr = BUCKET(nanos, 1 year) +| SORT yr DESC; ct:long | yr:date_nanos 8 | 2023-01-01T00:00:00.000000000Z @@ -567,7 +568,8 @@ required_capability: date_nanos_bucket FROM date_nanos | WHERE millis > "2020-01-01" -| STATS ct = count(*) BY yr = BUCKET(nanos, 5, "1999-01-01", NOW()); +| STATS ct = count(*) BY yr = BUCKET(nanos, 5, "1999-01-01", NOW()) +| SORT yr DESC; ct:long | yr:date_nanos 8 | 2023-01-01T00:00:00.000000000Z @@ -579,7 +581,8 @@ required_capability: date_nanos_bucket FROM date_nanos | WHERE millis > "2020-01-01" -| STATS ct = count(*) BY mo = BUCKET(nanos, 1 month); +| STATS ct = count(*) BY mo = BUCKET(nanos, 1 month) +| SORT mo DESC; ct:long | mo:date_nanos 8 | 2023-10-01T00:00:00.000000000Z @@ -591,7 +594,8 @@ required_capability: date_nanos_bucket FROM date_nanos | WHERE millis > "2020-01-01" -| STATS ct = count(*) BY mo = BUCKET(nanos, 20, "2023-01-01", "2023-12-31"); +| STATS ct = count(*) BY mo = BUCKET(nanos, 20, "2023-01-01", "2023-12-31") +| SORT mo DESC; ct:long | mo:date_nanos 8 | 2023-10-01T00:00:00.000000000Z @@ -603,18 +607,21 @@ required_capability: date_nanos_bucket FROM date_nanos | WHERE millis > "2020-01-01" -| STATS ct = count(*) BY mo = BUCKET(nanos, 55, "2023-01-01", "2023-12-31"); +| STATS ct = count(*) BY mo = BUCKET(nanos, 55, "2023-01-01", "2023-12-31") +| SORT mo DESC; ct:long | mo:date_nanos 8 | 2023-10-23T00:00:00.000000000Z ; + Bucket Date nanos by 10 minutes required_capability: date_trunc_date_nanos required_capability: date_nanos_bucket FROM date_nanos | WHERE millis > "2020-01-01" -| STATS ct = count(*) BY mn = BUCKET(nanos, 10 minutes); +| STATS ct = count(*) BY mn = BUCKET(nanos, 10 minutes) +| SORT mn DESC; ct:long | mn:date_nanos 4 | 2023-10-23T13:50:00.000000000Z From 517abe4ffdceb3c7ca771207fe20b0c813181810 Mon Sep 17 00:00:00 2001 From: Dianna Hohensee Date: Tue, 17 Dec 2024 14:51:45 -0500 Subject: [PATCH 43/44] ConnectTransportException returns retryable BAD_GATEWAY (#118681) ConnectTransportException and its subclasses previous translated to a INTERNAL_SERVER_ERROR HTTP 500 code. We are changing it to 502 BAD_GATEWAY so that users may choose to retry it on connectivity issues. Related ES-10214 Closes #118320 --- docs/changelog/118681.yaml | 6 ++++++ .../transport/ConnectTransportException.java | 13 +++++++++++++ .../elasticsearch/ExceptionSerializationTests.java | 1 + 3 files changed, 20 insertions(+) create mode 100644 docs/changelog/118681.yaml diff --git a/docs/changelog/118681.yaml b/docs/changelog/118681.yaml new file mode 100644 index 0000000000000..a186c05e6cd7d --- /dev/null +++ b/docs/changelog/118681.yaml @@ -0,0 +1,6 @@ +pr: 118681 +summary: '`ConnectTransportException` returns retryable BAD_GATEWAY' +area: Network +type: enhancement +issues: + - 118320 diff --git a/server/src/main/java/org/elasticsearch/transport/ConnectTransportException.java b/server/src/main/java/org/elasticsearch/transport/ConnectTransportException.java index 648d27c885843..302175cc4f5a0 100644 --- a/server/src/main/java/org/elasticsearch/transport/ConnectTransportException.java +++ b/server/src/main/java/org/elasticsearch/transport/ConnectTransportException.java @@ -13,6 +13,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.rest.RestStatus; import java.io.IOException; @@ -41,6 +42,18 @@ public ConnectTransportException(StreamInput in) throws IOException { } } + /** + * The ES REST API is a gateway to a single or multiple clusters. If there is an error connecting to other servers, then we should + * return a 502 BAD_GATEWAY status code instead of the parent class' 500 INTERNAL_SERVER_ERROR. Clients tend to retry on a 502 but not + * on a 500, and retrying may help on a connection error. + * + * @return a {@link RestStatus#BAD_GATEWAY} code + */ + @Override + public final RestStatus status() { + return RestStatus.BAD_GATEWAY; + } + @Override protected void writeTo(StreamOutput out, Writer nestedExceptionsWriter) throws IOException { super.writeTo(out, nestedExceptionsWriter); diff --git a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java index 2abe4157583cd..31f54f9a16359 100644 --- a/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/ExceptionSerializationTests.java @@ -409,6 +409,7 @@ public void testConnectTransportException() throws IOException { ex = serialize(new ConnectTransportException(node, "msg", "action", new NullPointerException())); assertEquals("[][" + transportAddress + "][action] msg", ex.getMessage()); assertThat(ex.getCause(), instanceOf(NullPointerException.class)); + assertEquals(RestStatus.BAD_GATEWAY, ex.status()); } public void testSearchPhaseExecutionException() throws IOException { From ae3c0d703257996ac4b07dfdc4543eb625ee9a60 Mon Sep 17 00:00:00 2001 From: Mark Tozzi Date: Tue, 17 Dec 2024 14:54:19 -0500 Subject: [PATCH 44/44] Esql implicit casting for date nanos (#118697) resolves #118476 This adds an implicit cast from string to date nanos, much the same as we do for millisecond dates. In the course of working on this, I found and fixed a couple of tests that were creating pre-epoch date nanos, which are not supported in elasticsearch. I also refactored the conversion code to use the standard DateUtils functions where appropriate, which caught some of the above errors in test data. --- docs/changelog/118697.yaml | 6 + .../src/main/resources/date_nanos.csv-spec | 131 ++++++++++++++++++ .../xpack/esql/action/EsqlCapabilities.java | 5 +- .../xpack/esql/analysis/Analyzer.java | 31 +++-- .../esql/type/EsqlDataTypeConverter.java | 14 +- .../esql/action/EsqlQueryResponseTests.java | 2 +- .../esql/type/EsqlDataTypeConverterTests.java | 12 +- 7 files changed, 179 insertions(+), 22 deletions(-) create mode 100644 docs/changelog/118697.yaml diff --git a/docs/changelog/118697.yaml b/docs/changelog/118697.yaml new file mode 100644 index 0000000000000..6e24e6ae4b47f --- /dev/null +++ b/docs/changelog/118697.yaml @@ -0,0 +1,6 @@ +pr: 118697 +summary: Esql implicit casting for date nanos +area: ES|QL +type: enhancement +issues: + - 118476 diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec index 47191148e0205..4206d6b48699f 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/resources/date_nanos.csv-spec @@ -216,6 +216,137 @@ millis:date | nanos:date_nanos | num:long 2023-10-23T13:33:34.937Z | 2023-10-23T13:33:34.937193000Z | 1698068014937193000 ; +implicit casting to nanos, date only +required_capability: date_nanos_type +required_capability: date_nanos_implicit_casting + +FROM date_nanos +| WHERE MV_MIN(nanos) > "2023-10-23" +| SORT nanos DESC +| KEEP millis, nanos; + +millis:date | nanos:date_nanos +2023-10-23T13:55:01.543Z | 2023-10-23T13:55:01.543123456Z +2023-10-23T13:53:55.832Z | 2023-10-23T13:53:55.832987654Z +2023-10-23T13:52:55.015Z | 2023-10-23T13:52:55.015787878Z +2023-10-23T13:51:54.732Z | 2023-10-23T13:51:54.732102837Z +2023-10-23T13:33:34.937Z | 2023-10-23T13:33:34.937193000Z +2023-10-23T12:27:28.948Z | 2023-10-23T12:27:28.948000000Z +2023-10-23T12:15:03.360Z | 2023-10-23T12:15:03.360103847Z +2023-10-23T12:15:03.360Z | 2023-10-23T12:15:03.360103847Z +; + +implicit casting to nanos, date only, equality test +required_capability: date_nanos_type +required_capability: date_nanos_implicit_casting + +FROM date_nanos +| WHERE MV_MIN(nanos) == "2023-10-23" +| SORT nanos DESC +| KEEP millis, nanos; + +millis:date | nanos:date_nanos +; + + +implicit casting to nanos, date plus time to seconds +required_capability: date_nanos_type +required_capability: date_nanos_implicit_casting + +FROM date_nanos +| WHERE MV_MIN(nanos) > "2023-10-23T00:00:00" +| SORT nanos DESC +| KEEP millis, nanos; + +millis:date | nanos:date_nanos +2023-10-23T13:55:01.543Z | 2023-10-23T13:55:01.543123456Z +2023-10-23T13:53:55.832Z | 2023-10-23T13:53:55.832987654Z +2023-10-23T13:52:55.015Z | 2023-10-23T13:52:55.015787878Z +2023-10-23T13:51:54.732Z | 2023-10-23T13:51:54.732102837Z +2023-10-23T13:33:34.937Z | 2023-10-23T13:33:34.937193000Z +2023-10-23T12:27:28.948Z | 2023-10-23T12:27:28.948000000Z +2023-10-23T12:15:03.360Z | 2023-10-23T12:15:03.360103847Z +2023-10-23T12:15:03.360Z | 2023-10-23T12:15:03.360103847Z +; + +implicit casting to nanos, date plus time to seconds, equality test +required_capability: date_nanos_type +required_capability: date_nanos_implicit_casting + +FROM date_nanos +| WHERE MV_MIN(nanos) == "2023-10-23T12:27:28" +| SORT nanos DESC +| KEEP millis, nanos; + +millis:date | nanos:date_nanos +; + +implicit casting to nanos, date plus time to millis +required_capability: date_nanos_type +required_capability: date_nanos_implicit_casting + +FROM date_nanos +| WHERE MV_MIN(nanos) > "2023-10-23T00:00:00.000" +| SORT nanos DESC +| KEEP millis, nanos; + +millis:date | nanos:date_nanos +2023-10-23T13:55:01.543Z | 2023-10-23T13:55:01.543123456Z +2023-10-23T13:53:55.832Z | 2023-10-23T13:53:55.832987654Z +2023-10-23T13:52:55.015Z | 2023-10-23T13:52:55.015787878Z +2023-10-23T13:51:54.732Z | 2023-10-23T13:51:54.732102837Z +2023-10-23T13:33:34.937Z | 2023-10-23T13:33:34.937193000Z +2023-10-23T12:27:28.948Z | 2023-10-23T12:27:28.948000000Z +2023-10-23T12:15:03.360Z | 2023-10-23T12:15:03.360103847Z +2023-10-23T12:15:03.360Z | 2023-10-23T12:15:03.360103847Z +; + +implicit casting to nanos, date plus time to millis, equality test +required_capability: date_nanos_type +required_capability: date_nanos_implicit_casting + +FROM date_nanos +| WHERE MV_MIN(nanos) == "2023-10-23T12:27:28.948" +| SORT nanos DESC +| KEEP millis, nanos; + +millis:date | nanos:date_nanos +2023-10-23T12:27:28.948Z | 2023-10-23T12:27:28.948000000Z +; + +implicit casting to nanos, date plus time to nanos +required_capability: date_nanos_type +required_capability: date_nanos_implicit_casting + +FROM date_nanos +| WHERE MV_MIN(nanos) > "2023-10-23T00:00:00.000000000" +| SORT nanos DESC +| KEEP millis, nanos; + +millis:date | nanos:date_nanos +2023-10-23T13:55:01.543Z | 2023-10-23T13:55:01.543123456Z +2023-10-23T13:53:55.832Z | 2023-10-23T13:53:55.832987654Z +2023-10-23T13:52:55.015Z | 2023-10-23T13:52:55.015787878Z +2023-10-23T13:51:54.732Z | 2023-10-23T13:51:54.732102837Z +2023-10-23T13:33:34.937Z | 2023-10-23T13:33:34.937193000Z +2023-10-23T12:27:28.948Z | 2023-10-23T12:27:28.948000000Z +2023-10-23T12:15:03.360Z | 2023-10-23T12:15:03.360103847Z +2023-10-23T12:15:03.360Z | 2023-10-23T12:15:03.360103847Z +; + +implicit casting to nanos, date plus time to nanos, equality test +required_capability: date_nanos_type +required_capability: date_nanos_implicit_casting + +FROM date_nanos +| WHERE MV_MIN(nanos) == "2023-10-23T12:27:28.948000000" +| SORT nanos DESC +| KEEP millis, nanos; + +millis:date | nanos:date_nanos +2023-10-23T12:27:28.948Z | 2023-10-23T12:27:28.948000000Z +; + date nanos greater than millis required_capability: date_nanos_type required_capability: date_nanos_compare_to_millis diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java index 4fcabb02b2d4f..f766beb76dd3d 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlCapabilities.java @@ -352,7 +352,10 @@ public enum Cap { * Support for mixed comparisons between nanosecond and millisecond dates */ DATE_NANOS_COMPARE_TO_MILLIS(), - + /** + * Support implicit casting of strings to date nanos + */ + DATE_NANOS_IMPLICIT_CASTING(), /** * Support Least and Greatest functions on Date Nanos type */ diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java index d59745f03f608..e15731ca79038 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/analysis/Analyzer.java @@ -118,6 +118,7 @@ import static org.elasticsearch.xpack.core.enrich.EnrichPolicy.GEO_MATCH_TYPE; import static org.elasticsearch.xpack.esql.core.type.DataType.BOOLEAN; import static org.elasticsearch.xpack.esql.core.type.DataType.DATETIME; +import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_NANOS; import static org.elasticsearch.xpack.esql.core.type.DataType.DATE_PERIOD; import static org.elasticsearch.xpack.esql.core.type.DataType.DOUBLE; import static org.elasticsearch.xpack.esql.core.type.DataType.FLOAT; @@ -1050,21 +1051,23 @@ private BitSet gatherPreAnalysisMetrics(LogicalPlan plan, BitSet b) { /** * Cast string literals in ScalarFunction, EsqlArithmeticOperation, BinaryComparison, In and GroupingFunction to desired data types. * For example, the string literals in the following expressions will be cast implicitly to the field data type on the left hand side. - * date > "2024-08-21" - * date in ("2024-08-21", "2024-08-22", "2024-08-23") - * date = "2024-08-21" + 3 days - * ip == "127.0.0.1" - * version != "1.0" - * bucket(dateField, "1 month") - * date_trunc("1 minute", dateField) - * + *

    + *
  • date > "2024-08-21"
  • + *
  • date in ("2024-08-21", "2024-08-22", "2024-08-23")
  • + *
  • date = "2024-08-21" + 3 days
  • + *
  • ip == "127.0.0.1"
  • + *
  • version != "1.0"
  • + *
  • bucket(dateField, "1 month")
  • + *
  • date_trunc("1 minute", dateField)
  • + *
* If the inputs to Coalesce are mixed numeric types, cast the rest of the numeric field or value to the first numeric data type if * applicable. For example, implicit casting converts: - * Coalesce(Long, Int) to Coalesce(Long, Long) - * Coalesce(null, Long, Int) to Coalesce(null, Long, Long) - * Coalesce(Double, Long, Int) to Coalesce(Double, Double, Double) - * Coalesce(null, Double, Long, Int) to Coalesce(null, Double, Double, Double) - * + *
    + *
  • Coalesce(Long, Int) to Coalesce(Long, Long)
  • + *
  • Coalesce(null, Long, Int) to Coalesce(null, Long, Long)
  • + *
  • Coalesce(Double, Long, Int) to Coalesce(Double, Double, Double)
  • + *
  • Coalesce(null, Double, Long, Int) to Coalesce(null, Double, Double, Double)
  • + *
* Coalesce(Int, Long) will NOT be converted to Coalesce(Long, Long) or Coalesce(Int, Int). */ private static class ImplicitCasting extends ParameterizedRule { @@ -1245,7 +1248,7 @@ private static boolean supportsImplicitTemporalCasting(Expression e, BinaryOpera } private static boolean supportsStringImplicitCasting(DataType type) { - return type == DATETIME || type == IP || type == VERSION || type == BOOLEAN; + return type == DATETIME || type == DATE_NANOS || type == IP || type == VERSION || type == BOOLEAN; } private static UnresolvedAttribute unresolvedAttribute(Expression value, String type, Exception e) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java index 6ba2d8451f956..0847f71b1fb01 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverter.java @@ -13,6 +13,8 @@ import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.common.time.DateFormatters; +import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.search.DocValueFormat; import org.elasticsearch.xpack.esql.core.InvalidArgumentException; import org.elasticsearch.xpack.esql.core.QlIllegalArgumentException; @@ -51,7 +53,6 @@ import java.time.Period; import java.time.ZoneId; import java.time.temporal.ChronoField; -import java.time.temporal.TemporalAccessor; import java.time.temporal.TemporalAmount; import java.util.List; import java.util.Locale; @@ -200,6 +201,9 @@ public static Converter converterFor(DataType from, DataType to) { if (to == DataType.DATETIME) { return EsqlConverter.STRING_TO_DATETIME; } + if (to == DATE_NANOS) { + return EsqlConverter.STRING_TO_DATE_NANOS; + } if (to == DataType.IP) { return EsqlConverter.STRING_TO_IP; } @@ -514,13 +518,12 @@ public static long dateTimeToLong(String dateTime, DateFormatter formatter) { } public static long dateNanosToLong(String dateNano) { - return dateNanosToLong(dateNano, DateFormatter.forPattern("strict_date_optional_time_nanos")); + return dateNanosToLong(dateNano, DEFAULT_DATE_NANOS_FORMATTER); } public static long dateNanosToLong(String dateNano, DateFormatter formatter) { - TemporalAccessor parsed = formatter.parse(dateNano); - long nanos = parsed.getLong(ChronoField.INSTANT_SECONDS) * 1_000_000_000 + parsed.getLong(ChronoField.NANO_OF_SECOND); - return nanos; + Instant parsed = DateFormatters.from(formatter.parse(dateNano)).toInstant(); + return DateUtils.toLong(parsed); } public static String dateTimeToString(long dateTime) { @@ -639,6 +642,7 @@ public enum EsqlConverter implements Converter { STRING_TO_TIME_DURATION(x -> EsqlDataTypeConverter.parseTemporalAmount(x, DataType.TIME_DURATION)), STRING_TO_CHRONO_FIELD(EsqlDataTypeConverter::stringToChrono), STRING_TO_DATETIME(x -> EsqlDataTypeConverter.dateTimeToLong((String) x)), + STRING_TO_DATE_NANOS(x -> EsqlDataTypeConverter.dateNanosToLong((String) x)), STRING_TO_IP(x -> EsqlDataTypeConverter.stringToIP((String) x)), STRING_TO_VERSION(x -> EsqlDataTypeConverter.stringToVersion((String) x)), STRING_TO_DOUBLE(x -> EsqlDataTypeConverter.stringToDouble((String) x)), diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java index 35364089127cc..2deedb927331d 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/action/EsqlQueryResponseTests.java @@ -204,7 +204,7 @@ private Page randomPage(List columns) { case BOOLEAN -> ((BooleanBlock.Builder) builder).appendBoolean(randomBoolean()); case UNSUPPORTED -> ((BytesRefBlock.Builder) builder).appendNull(); // TODO - add a random instant thing here? - case DATE_NANOS -> ((LongBlock.Builder) builder).appendLong(randomLong()); + case DATE_NANOS -> ((LongBlock.Builder) builder).appendLong(randomNonNegativeLong()); case VERSION -> ((BytesRefBlock.Builder) builder).appendBytesRef(new Version(randomIdentifier()).toBytesRef()); case GEO_POINT -> ((BytesRefBlock.Builder) builder).appendBytesRef(GEO.asWkb(GeometryTestUtils.randomPoint())); case CARTESIAN_POINT -> ((BytesRefBlock.Builder) builder).appendBytesRef(CARTESIAN.asWkb(ShapeTestUtils.randomPoint())); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverterTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverterTests.java index 8a57dfa968ccd..9a30c2281d742 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverterTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/type/EsqlDataTypeConverterTests.java @@ -7,9 +7,11 @@ package org.elasticsearch.xpack.esql.type; +import org.elasticsearch.common.time.DateUtils; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.esql.core.type.DataType; +import java.time.Instant; import java.util.Arrays; import java.util.List; @@ -50,11 +52,19 @@ public class EsqlDataTypeConverterTests extends ESTestCase { public void testNanoTimeToString() { - long expected = randomLong(); + long expected = randomNonNegativeLong(); long actual = EsqlDataTypeConverter.dateNanosToLong(EsqlDataTypeConverter.nanoTimeToString(expected)); assertEquals(expected, actual); } + public void testStringToDateNanos() { + assertEquals( + DateUtils.toLong(Instant.parse("2023-01-01T00:00:00.000Z")), + EsqlDataTypeConverter.convert("2023-01-01T00:00:00.000000000", DATE_NANOS) + ); + assertEquals(DateUtils.toLong(Instant.parse("2023-01-01T00:00:00.000Z")), EsqlDataTypeConverter.convert("2023-01-01", DATE_NANOS)); + } + public void testCommonTypeNull() { for (DataType dataType : DataType.values()) { assertEqualsCommonType(dataType, NULL, dataType);