From e8b543b8cd6e19e29e16df19524356ab014bce03 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 23 May 2018 23:15:19 -0400 Subject: [PATCH 01/20] Force stable file modes for built packages (#30823) If you have an unusual umask (e.g., 0002) and clone the GitHub repository then files that we stick into our packages like the README.textile and the license will have a file mode of 0664 on disk yet we expect them to be 0644. Additionally, the same thing happens with compiled artifacts like JARs. We try to set a default file mode yet it does not seem to take everywhere. This commit adds explicit file modes in some places that we were relying on the defaults to ensure that the built artifacts have a consistent file mode regardless of the underlying build host. --- distribution/build.gradle | 2 ++ distribution/packages/build.gradle | 10 ++++++++++ 2 files changed, 12 insertions(+) diff --git a/distribution/build.gradle b/distribution/build.gradle index 940a4152bfd55..5f6f0b1579cea 100644 --- a/distribution/build.gradle +++ b/distribution/build.gradle @@ -242,6 +242,8 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) { if (it.relativePath.segments[-2] == 'bin') { // bin files, wherever they are within modules (eg platform specific) should be executable it.mode = 0755 + } else { + it.mode = 0644 } } if (oss) { diff --git a/distribution/packages/build.gradle b/distribution/packages/build.gradle index a6759a2e4f183..b15e5668e3dbe 100644 --- a/distribution/packages/build.gradle +++ b/distribution/packages/build.gradle @@ -122,6 +122,7 @@ Closure commonPackageConfig(String type, boolean oss) { } from(rootProject.projectDir) { include 'README.textile' + fileMode 0644 } into('modules') { with copySpec { @@ -135,6 +136,11 @@ Closure commonPackageConfig(String type, boolean oss) { for (int i = segments.length - 2; i > 0 && segments[i] != 'modules'; --i) { directory('/' + segments[0..i].join('/'), 0755) } + if (segments[-2] == 'bin') { + fcp.mode = 0755 + } else { + fcp.mode = 0644 + } } } } @@ -153,6 +159,7 @@ Closure commonPackageConfig(String type, boolean oss) { include oss ? 'APACHE-LICENSE-2.0.txt' : 'ELASTIC-LICENSE.txt' rename { 'LICENSE.txt' } } + fileMode 0644 } } @@ -180,14 +187,17 @@ Closure commonPackageConfig(String type, boolean oss) { // ========= systemd ========= into('/usr/lib/tmpfiles.d') { from "${packagingFiles}/systemd/elasticsearch.conf" + fileMode 0644 } into('/usr/lib/systemd/system') { fileType CONFIG | NOREPLACE from "${packagingFiles}/systemd/elasticsearch.service" + fileMode 0644 } into('/usr/lib/sysctl.d') { fileType CONFIG | NOREPLACE from "${packagingFiles}/systemd/sysctl/elasticsearch.conf" + fileMode 0644 } // ========= sysV init ========= From 2c7559c575c5654a2f7842746efea3bf4429fc0f Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Wed, 23 May 2018 23:20:13 -0700 Subject: [PATCH 02/20] Packaging: Ensure upgrade_is_oss flag file is always deleted (#30732) This commit ensures the delete of the upgrade_is_oss indicator for the packaging tests is always deleted before each run. It works by moving the check on version which skips the task into the doFirst block, replacing the onlyIf. closes #30682 --- .../elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy index 6e8a5fd15edf1..455d30f95db32 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/vagrant/VagrantTestPlugin.groovy @@ -11,6 +11,7 @@ import org.gradle.api.internal.artifacts.dependencies.DefaultProjectDependency import org.gradle.api.tasks.Copy import org.gradle.api.tasks.Delete import org.gradle.api.tasks.Exec +import org.gradle.api.tasks.StopExecutionException import org.gradle.api.tasks.TaskState import static java.util.Collections.unmodifiableList @@ -285,8 +286,10 @@ class VagrantTestPlugin implements Plugin { dependsOn copyPackagingArchives doFirst { project.delete("${archivesDir}/upgrade_is_oss") + if (project.extensions.esvagrant.upgradeFromVersion.before('6.3.0')) { + throw new StopExecutionException("upgrade version is before 6.3.0") + } } - onlyIf { project.extensions.esvagrant.upgradeFromVersion.onOrAfter('6.3.0') } file "${archivesDir}/upgrade_is_oss" contents '' } From 0bdfb5c5b5b79f4c9d99e112420b6d0d9d95537d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 24 May 2018 09:46:48 +0200 Subject: [PATCH 03/20] Send client headers from TransportClient (#30803) This change adds a simple header to the transport client that is present on the servers thread context that ensures we can detect if a transport client talks to the server in a specific request. This change also adds a header for xpack to detect if the client has xpack installed. --- .../client/transport/TransportClient.java | 4 +++- .../elasticsearch/transport/TcpTransport.java | 1 + .../client/AbstractClientHeadersTestCase.java | 3 ++- .../client/transport/TransportClientTests.java | 17 +++++++++++++++++ .../xpack/core/XPackClientPlugin.java | 2 ++ 5 files changed, 25 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/client/transport/TransportClient.java b/server/src/main/java/org/elasticsearch/client/transport/TransportClient.java index fd56186fd9d4f..2aeea08d1a575 100644 --- a/server/src/main/java/org/elasticsearch/client/transport/TransportClient.java +++ b/server/src/main/java/org/elasticsearch/client/transport/TransportClient.java @@ -19,6 +19,7 @@ package org.elasticsearch.client.transport; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; @@ -129,7 +130,8 @@ private static ClientTemplate buildTemplate(Settings providedSettings, Settings providedSettings = Settings.builder().put(providedSettings).put(Node.NODE_NAME_SETTING.getKey(), "_client_").build(); } final PluginsService pluginsService = newPluginService(providedSettings, plugins); - final Settings settings = Settings.builder().put(defaultSettings).put(pluginsService.updatedSettings()).build(); + final Settings settings = Settings.builder().put(defaultSettings).put(pluginsService.updatedSettings()).put(ThreadContext.PREFIX + + "." + "transport_client", true).build(); final List resourcesToClose = new ArrayList<>(); final ThreadPool threadPool = new ThreadPool(settings); resourcesToClose.add(() -> ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS)); diff --git a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java index 04a882f3e8b45..0b3d4e1b0a1ef 100644 --- a/server/src/main/java/org/elasticsearch/transport/TcpTransport.java +++ b/server/src/main/java/org/elasticsearch/transport/TcpTransport.java @@ -1438,6 +1438,7 @@ public final void messageReceived(BytesReference reference, TcpChannel channel) streamIn = new NamedWriteableAwareStreamInput(streamIn, namedWriteableRegistry); streamIn.setVersion(version); threadPool.getThreadContext().readHeaders(streamIn); + threadPool.getThreadContext().putTransient("_remote_address", remoteAddress); if (TransportStatus.isRequest(status)) { handleRequest(channel, profileName, streamIn, requestId, messageLengthBytes, version, remoteAddress, status); } else { diff --git a/server/src/test/java/org/elasticsearch/client/AbstractClientHeadersTestCase.java b/server/src/test/java/org/elasticsearch/client/AbstractClientHeadersTestCase.java index 8c1b22f7fb171..db9f9d83c816a 100644 --- a/server/src/test/java/org/elasticsearch/client/AbstractClientHeadersTestCase.java +++ b/server/src/test/java/org/elasticsearch/client/AbstractClientHeadersTestCase.java @@ -139,6 +139,8 @@ public void testOverrideHeader() throws Exception { protected static void assertHeaders(Map headers, Map expected) { assertNotNull(headers); + headers = new HashMap<>(headers); + headers.remove("transport_client"); // default header on TPC assertEquals(expected.size(), headers.size()); for (Map.Entry expectedEntry : expected.entrySet()) { assertEquals(headers.get(expectedEntry.getKey()), expectedEntry.getValue()); @@ -146,7 +148,6 @@ protected static void assertHeaders(Map headers, Map headers = new HashMap<>(); Settings asSettings = HEADER_SETTINGS.getAsSettings(ThreadContext.PREFIX); assertHeaders(pool.getThreadContext().getHeaders(), asSettings.keySet().stream().collect(Collectors.toMap(Function.identity(), k -> asSettings.get(k)))); diff --git a/server/src/test/java/org/elasticsearch/client/transport/TransportClientTests.java b/server/src/test/java/org/elasticsearch/client/transport/TransportClientTests.java index c97418bae373a..1830698d90c6f 100644 --- a/server/src/test/java/org/elasticsearch/client/transport/TransportClientTests.java +++ b/server/src/test/java/org/elasticsearch/client/transport/TransportClientTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESTestCase; @@ -63,6 +64,17 @@ public void testPluginNamedWriteablesRegistered() { } } + public void testDefaultHeaderContainsPlugins() { + Settings baseSettings = Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) + .build(); + try (TransportClient client = new MockTransportClient(baseSettings, Arrays.asList(MockPlugin.class))) { + ThreadContext threadContext = client.threadPool().getThreadContext(); + assertEquals("true", threadContext.getHeader("transport_client")); + assertEquals("true", threadContext.getHeader("test")); + } + } + public static class MockPlugin extends Plugin { @Override @@ -70,6 +82,11 @@ public List getNamedWriteables() { return Arrays.asList(new Entry[]{ new Entry(MockNamedWriteable.class, MockNamedWriteable.NAME, MockNamedWriteable::new)}); } + @Override + public Settings additionalSettings() { + return Settings.builder().put(ThreadContext.PREFIX + "." + "test", true).build(); + } + public class MockNamedWriteable implements NamedWriteable { static final String NAME = "mockNamedWritable"; diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index 4853588bd3ead..c719cdcbd022e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -16,6 +16,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.license.DeleteLicenseAction; @@ -193,6 +194,7 @@ static Settings additionalSettings(final Settings settings, final boolean enable final Settings.Builder builder = Settings.builder(); builder.put(SecuritySettings.addTransportSettings(settings)); builder.put(SecuritySettings.addUserSettings(settings)); + builder.put(ThreadContext.PREFIX + "." + "has_xpack", true); return builder.build(); } else { return Settings.EMPTY; From ff0b6c795ae12d8eca7274fe9712993cdeb7a820 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 24 May 2018 09:05:09 +0100 Subject: [PATCH 04/20] Decouple ClusterStateTaskListener & ClusterApplier (#30809) Today, the `ClusterApplier` and `MasterService` both use the `ClusterStateTaskListener` interface to notify their callers when asynchronous activities have completed. However, this is not wholly appropriate: none of the callers into the `ClusterApplier` care about the `ClusterState` arguments that they receive. This change introduces a dedicated ClusterApplyListener interface for callers into the `ClusterApplier`, to distinguish these listeners from the real `ClusterStateTaskListener`s that are waiting for responses from the `MasterService`. --- .../cluster/service/ClusterApplier.java | 21 ++++++++++- .../service/ClusterApplierService.java | 33 ++++++++--------- .../discovery/single/SingleNodeDiscovery.java | 8 ++-- .../discovery/zen/ZenDiscovery.java | 8 ++-- .../service/ClusterApplierServiceTests.java | 37 ++++++++++--------- .../single/SingleNodeDiscoveryTests.java | 5 +-- .../discovery/zen/ZenDiscoveryUnitTests.java | 5 +-- .../store/IndicesStoreIntegrationIT.java | 5 ++- .../test/ClusterServiceUtils.java | 10 ++--- 9 files changed, 71 insertions(+), 61 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java index 0a2ef347d0665..c587ab272e903 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplier.java @@ -20,7 +20,6 @@ package org.elasticsearch.cluster.service; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskListener; import java.util.function.Supplier; @@ -38,11 +37,29 @@ public interface ClusterApplier { * @param clusterStateSupplier the cluster state supplier which provides the latest cluster state to apply * @param listener callback that is invoked after cluster state is applied */ - void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterStateTaskListener listener); + void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterApplyListener listener); /** * Creates a new cluster state builder that is initialized with the cluster name and all initial cluster state customs. */ ClusterState.Builder newClusterStateBuilder(); + /** + * Listener for results of cluster state application + */ + interface ClusterApplyListener { + /** + * Called on successful cluster state application + * @param source information where the cluster state came from + */ + default void onSuccess(String source) { + } + + /** + * Called on failure during cluster state application + * @param source information where the cluster state came from + * @param e exception that occurred + */ + void onFailure(String source, Exception e); + } } diff --git a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java index 01fa5837387c8..2fb7c25671c88 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java @@ -27,7 +27,6 @@ import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.ClusterStateTaskConfig; -import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.LocalNodeMasterListener; import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.TimeoutClusterStateListener; @@ -141,10 +140,10 @@ protected synchronized void doStart() { } class UpdateTask extends SourcePrioritizedRunnable implements Function { - final ClusterStateTaskListener listener; + final ClusterApplyListener listener; final Function updateFunction; - UpdateTask(Priority priority, String source, ClusterStateTaskListener listener, + UpdateTask(Priority priority, String source, ClusterApplyListener listener, Function updateFunction) { super(priority, source); this.listener = listener; @@ -301,7 +300,7 @@ public void run() { } public void runOnApplierThread(final String source, Consumer clusterStateConsumer, - final ClusterStateTaskListener listener, Priority priority) { + final ClusterApplyListener listener, Priority priority) { submitStateUpdateTask(source, ClusterStateTaskConfig.build(priority), (clusterState) -> { clusterStateConsumer.accept(clusterState); @@ -311,13 +310,13 @@ public void runOnApplierThread(final String source, Consumer clust } public void runOnApplierThread(final String source, Consumer clusterStateConsumer, - final ClusterStateTaskListener listener) { + final ClusterApplyListener listener) { runOnApplierThread(source, clusterStateConsumer, listener, Priority.HIGH); } @Override public void onNewClusterState(final String source, final Supplier clusterStateSupplier, - final ClusterStateTaskListener listener) { + final ClusterApplyListener listener) { Function applyFunction = currentState -> { ClusterState nextState = clusterStateSupplier.get(); if (nextState != null) { @@ -331,12 +330,12 @@ public void onNewClusterState(final String source, final Supplier private void submitStateUpdateTask(final String source, final ClusterStateTaskConfig config, final Function executor, - final ClusterStateTaskListener listener) { + final ClusterApplyListener listener) { if (!lifecycle.started()) { return; } try { - UpdateTask updateTask = new UpdateTask(config.priority(), source, new SafeClusterStateTaskListener(listener, logger), executor); + UpdateTask updateTask = new UpdateTask(config.priority(), source, new SafeClusterApplyListener(listener, logger), executor); if (config.timeout() != null) { threadPoolExecutor.execute(updateTask, config.timeout(), () -> threadPool.generic().execute( @@ -417,7 +416,7 @@ protected void runTask(UpdateTask task) { } if (previousClusterState == newClusterState) { - task.listener.clusterStateProcessed(task.source, newClusterState, newClusterState); + task.listener.onSuccess(task.source); TimeValue executionTime = TimeValue.timeValueMillis(Math.max(0, TimeValue.nsecToMSec(currentTimeInNanos() - startTimeNS))); logger.debug("processing [{}]: took [{}] no change in cluster state", task.source, executionTime); warnAboutSlowTaskIfNeeded(executionTime, task.source); @@ -486,7 +485,7 @@ private void applyChanges(UpdateTask task, ClusterState previousClusterState, Cl callClusterStateListeners(clusterChangedEvent); - task.listener.clusterStateProcessed(task.source, previousClusterState, newClusterState); + task.listener.onSuccess(task.source); } private void callClusterStateAppliers(ClusterChangedEvent clusterChangedEvent) { @@ -511,11 +510,11 @@ private void callClusterStateListeners(ClusterChangedEvent clusterChangedEvent) }); } - private static class SafeClusterStateTaskListener implements ClusterStateTaskListener { - private final ClusterStateTaskListener listener; + private static class SafeClusterApplyListener implements ClusterApplyListener { + private final ClusterApplyListener listener; private final Logger logger; - SafeClusterStateTaskListener(ClusterStateTaskListener listener, Logger logger) { + SafeClusterApplyListener(ClusterApplyListener listener, Logger logger) { this.listener = listener; this.logger = logger; } @@ -532,14 +531,12 @@ public void onFailure(String source, Exception e) { } @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { try { - listener.clusterStateProcessed(source, oldState, newState); + listener.onSuccess(source); } catch (Exception e) { logger.error(new ParameterizedMessage( - "exception thrown by listener while notifying of cluster state processed from [{}], old cluster state:\n" + - "{}\nnew cluster state:\n{}", - source, oldState, newState), e); + "exception thrown by listener while notifying of cluster state processed from [{}]", source), e); } } } diff --git a/server/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java b/server/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java index 94ea33d1a16ab..cd775e29f5a2f 100644 --- a/server/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java +++ b/server/src/main/java/org/elasticsearch/discovery/single/SingleNodeDiscovery.java @@ -22,18 +22,16 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterApplier; +import org.elasticsearch.cluster.service.ClusterApplier.ClusterApplyListener; import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoveryStats; -import org.elasticsearch.discovery.zen.PendingClusterStateStats; -import org.elasticsearch.discovery.zen.PublishClusterStateStats; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -65,9 +63,9 @@ public synchronized void publish(final ClusterChangedEvent event, clusterState = event.state(); CountDownLatch latch = new CountDownLatch(1); - ClusterStateTaskListener listener = new ClusterStateTaskListener() { + ClusterApplyListener listener = new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); ackListener.onNodeAck(transportService.getLocalNode(), null); } diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 02b2822fcf431..55ecf7ca25fa6 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -21,7 +21,6 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.logging.log4j.util.Supplier; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; @@ -34,12 +33,11 @@ import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.block.ClusterBlocks; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterApplier; +import org.elasticsearch.cluster.service.ClusterApplier.ClusterApplyListener; import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.Priority; import org.elasticsearch.common.component.AbstractLifecycleComponent; @@ -789,9 +787,9 @@ boolean processNextCommittedClusterState(String reason) { clusterApplier.onNewClusterState("apply cluster state (from master [" + reason + "])", this::clusterState, - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { try { pendingStatesQueue.markAsProcessed(newClusterState); } catch (Exception e) { diff --git a/server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java index 7a8261776bd41..3e7c415db7b96 100644 --- a/server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/service/ClusterApplierServiceTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.service.ClusterApplier.ClusterApplyListener; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; @@ -135,9 +136,9 @@ public void testClusterStateUpdateLogging() throws Exception { clusterApplierService.currentTimeOverride = System.nanoTime(); clusterApplierService.runOnApplierThread("test1", currentState -> clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos(), - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } @@ -151,9 +152,9 @@ public void onFailure(String source, Exception e) { clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(2).nanos(); throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task"); }, - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { fail(); } @@ -166,9 +167,9 @@ public void onFailure(String source, Exception e) { // We don't check logging for this on since there is no guarantee that it will occur before our check clusterApplierService.runOnApplierThread("test3", currentState -> {}, - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } @@ -216,9 +217,9 @@ public void testLongClusterStateUpdateLogging() throws Exception { clusterApplierService.currentTimeOverride = System.nanoTime(); clusterApplierService.runOnApplierThread("test1", currentState -> clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(1).nanos(), - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); processedFirstTask.countDown(); } @@ -234,9 +235,9 @@ public void onFailure(String source, Exception e) { clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(32).nanos(); throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task"); }, - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { fail(); } @@ -247,9 +248,9 @@ public void onFailure(String source, Exception e) { }); clusterApplierService.runOnApplierThread("test3", currentState -> clusterApplierService.currentTimeOverride += TimeValue.timeValueSeconds(34).nanos(), - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } @@ -262,9 +263,9 @@ public void onFailure(String source, Exception e) { // We don't check logging for this on since there is no guarantee that it will occur before our check clusterApplierService.runOnApplierThread("test4", currentState -> {}, - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } @@ -340,10 +341,10 @@ public void testClusterStateApplierCantSampleClusterState() throws InterruptedEx CountDownLatch latch = new CountDownLatch(1); clusterApplierService.onNewClusterState("test", () -> ClusterState.builder(clusterApplierService.state()).build(), - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } @@ -390,9 +391,9 @@ public void onTimeout(TimeValue timeout) { CountDownLatch latch = new CountDownLatch(1); clusterApplierService.onNewClusterState("test", () -> ClusterState.builder(clusterApplierService.state()).build(), - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } diff --git a/server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java b/server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java index 23a510a257f21..d045adcaead21 100644 --- a/server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/single/SingleNodeDiscoveryTests.java @@ -23,7 +23,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterApplier; @@ -72,9 +71,9 @@ public ClusterState.Builder newClusterStateBuilder() { @Override public void onNewClusterState(String source, Supplier clusterStateSupplier, - ClusterStateTaskListener listener) { + ClusterApplyListener listener) { clusterState.set(clusterStateSupplier.get()); - listener.clusterStateProcessed(source, clusterState.get(), clusterState.get()); + listener.onSuccess(source); } }); discovery.start(); diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index 0ecb5a296f570..a2121d3ca4e37 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ESAllocationTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -314,8 +313,8 @@ public ClusterState.Builder newClusterStateBuilder() { } @Override - public void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterStateTaskListener listener) { - listener.clusterStateProcessed(source, clusterStateSupplier.get(), clusterStateSupplier.get()); + public void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterApplyListener listener) { + listener.onSuccess(source); } }; ZenDiscovery zenDiscovery = new ZenDiscovery(settings, threadPool, service, new NamedWriteableRegistry(ClusterModule.getNamedWriteables()), diff --git a/server/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java b/server/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java index 7f6155979c916..6b57c9757366a 100644 --- a/server/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java +++ b/server/src/test/java/org/elasticsearch/indices/store/IndicesStoreIntegrationIT.java @@ -36,6 +36,7 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; +import org.elasticsearch.cluster.service.ClusterApplier.ClusterApplyListener; import org.elasticsearch.cluster.service.ClusterApplierService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; @@ -446,9 +447,9 @@ public void testShardActiveElseWhere() throws Exception { .routingTable(RoutingTable.builder().add(indexRoutingTableBuilder).build()) .build(); CountDownLatch latch = new CountDownLatch(1); - clusterApplierService.onNewClusterState("test", () -> newState, new ClusterStateTaskListener() { + clusterApplierService.onNewClusterState("test", () -> newState, new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java index df1e216f4bbac..8c4076e327d70 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ClusterServiceUtils.java @@ -24,13 +24,13 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterApplier; +import org.elasticsearch.cluster.service.ClusterApplier.ClusterApplyListener; import org.elasticsearch.cluster.service.ClusterApplierService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService; @@ -72,9 +72,9 @@ public static void setState(ClusterApplierService executor, ClusterState cluster CountDownLatch latch = new CountDownLatch(1); AtomicReference exception = new AtomicReference<>(); executor.onNewClusterState("test setting state", - () -> ClusterState.builder(clusterState).version(clusterState.version() + 1).build(), new ClusterStateTaskListener() { + () -> ClusterState.builder(clusterState).version(clusterState.version() + 1).build(), new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } @@ -163,9 +163,9 @@ public static BiConsumer createClusterStatePub CountDownLatch latch = new CountDownLatch(1); AtomicReference ex = new AtomicReference<>(); clusterApplier.onNewClusterState("mock_publish_to_self[" + event.source() + "]", () -> event.state(), - new ClusterStateTaskListener() { + new ClusterApplyListener() { @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + public void onSuccess(String source) { latch.countDown(); } From aafcd85f50e9ae0d996da2a8fc43503f1ba0bcd2 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Thu, 24 May 2018 09:17:17 +0100 Subject: [PATCH 05/20] Move persistent task registrations to core (#30755) Persistent tasks was moved from X-Pack to core in #28455. However, registration of the named writables and named X-content was left in X-Pack. This change moves the registration of the named writables and named X-content into core. Additionally, the persistent task actions are no longer registered in the X-Pack client plugin, as they are already registered in ActionModule. --- .../elasticsearch/cluster/ClusterModule.java | 9 ++++++++ .../PersistentTasksCustomMetaData.java | 1 - .../persistent/TestPersistentTasksPlugin.java | 10 --------- .../xpack/core/XPackClientPlugin.java | 22 +------------------ .../MlNativeAutodetectIntegTestCase.java | 6 ----- 5 files changed, 10 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java index 9baa47fbc2600..7f16c3f85ffc6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -69,8 +69,11 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.ingest.IngestMetadata; +import org.elasticsearch.persistent.PersistentTasksCustomMetaData; +import org.elasticsearch.persistent.PersistentTasksNodeService; import org.elasticsearch.plugins.ClusterPlugin; import org.elasticsearch.script.ScriptMetaData; +import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskResultsService; import java.util.ArrayList; @@ -140,6 +143,10 @@ public static List getNamedWriteables() { registerMetaDataCustom(entries, IngestMetadata.TYPE, IngestMetadata::new, IngestMetadata::readDiffFrom); registerMetaDataCustom(entries, ScriptMetaData.TYPE, ScriptMetaData::new, ScriptMetaData::readDiffFrom); registerMetaDataCustom(entries, IndexGraveyard.TYPE, IndexGraveyard::new, IndexGraveyard::readDiffFrom); + registerMetaDataCustom(entries, PersistentTasksCustomMetaData.TYPE, PersistentTasksCustomMetaData::new, + PersistentTasksCustomMetaData::readDiffFrom); + // Task Status (not Diffable) + entries.add(new Entry(Task.Status.class, PersistentTasksNodeService.Status.NAME, PersistentTasksNodeService.Status::new)); return entries; } @@ -154,6 +161,8 @@ public static List getNamedXWriteables() { ScriptMetaData::fromXContent)); entries.add(new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(IndexGraveyard.TYPE), IndexGraveyard::fromXContent)); + entries.add(new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(PersistentTasksCustomMetaData.TYPE), + PersistentTasksCustomMetaData::fromXContent)); return entries; } diff --git a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java index 6611ff7f2a3cc..6d2c21a764ad5 100644 --- a/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java +++ b/server/src/main/java/org/elasticsearch/persistent/PersistentTasksCustomMetaData.java @@ -25,7 +25,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.NamedDiff; import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.Strings; diff --git a/server/src/test/java/org/elasticsearch/persistent/TestPersistentTasksPlugin.java b/server/src/test/java/org/elasticsearch/persistent/TestPersistentTasksPlugin.java index 35c261056721c..556d6d1983e63 100644 --- a/server/src/test/java/org/elasticsearch/persistent/TestPersistentTasksPlugin.java +++ b/server/src/test/java/org/elasticsearch/persistent/TestPersistentTasksPlugin.java @@ -33,9 +33,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.NamedDiff; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.ParseField; @@ -100,12 +98,6 @@ public List> getPersistentTasksExecutor(ClusterServic public List getNamedWriteables() { return Arrays.asList( new NamedWriteableRegistry.Entry(PersistentTaskParams.class, TestPersistentTasksExecutor.NAME, TestParams::new), - new NamedWriteableRegistry.Entry(Task.Status.class, - PersistentTasksNodeService.Status.NAME, PersistentTasksNodeService.Status::new), - new NamedWriteableRegistry.Entry(MetaData.Custom.class, PersistentTasksCustomMetaData.TYPE, - PersistentTasksCustomMetaData::new), - new NamedWriteableRegistry.Entry(NamedDiff.class, PersistentTasksCustomMetaData.TYPE, - PersistentTasksCustomMetaData::readDiffFrom), new NamedWriteableRegistry.Entry(Task.Status.class, TestPersistentTasksExecutor.NAME, Status::new) ); } @@ -113,8 +105,6 @@ public List getNamedWriteables() { @Override public List getNamedXContent() { return Arrays.asList( - new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(PersistentTasksCustomMetaData.TYPE), - PersistentTasksCustomMetaData::fromXContent), new NamedXContentRegistry.Entry(PersistentTaskParams.class, new ParseField(TestPersistentTasksExecutor.NAME), TestParams::fromXContent), new NamedXContentRegistry.Entry(Task.Status.class, new ParseField(TestPersistentTasksExecutor.NAME), Status::fromXContent) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index c719cdcbd022e..0b22cd86fe6a0 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -91,13 +91,7 @@ import org.elasticsearch.xpack.core.ml.datafeed.DatafeedState; import org.elasticsearch.xpack.core.ml.job.config.JobTaskStatus; import org.elasticsearch.xpack.core.monitoring.MonitoringFeatureSetUsage; -import org.elasticsearch.persistent.CompletionPersistentTaskAction; import org.elasticsearch.persistent.PersistentTaskParams; -import org.elasticsearch.persistent.PersistentTasksCustomMetaData; -import org.elasticsearch.persistent.PersistentTasksNodeService; -import org.elasticsearch.persistent.RemovePersistentTaskAction; -import org.elasticsearch.persistent.StartPersistentTaskAction; -import org.elasticsearch.persistent.UpdatePersistentTaskStatusAction; import org.elasticsearch.xpack.core.rollup.RollupFeatureSetUsage; import org.elasticsearch.xpack.core.rollup.RollupField; import org.elasticsearch.xpack.core.rollup.action.DeleteRollupJobAction; @@ -255,11 +249,6 @@ public List getClientActions() { GetCalendarEventsAction.INSTANCE, PostCalendarEventsAction.INSTANCE, PersistJobAction.INSTANCE, - // licensing - StartPersistentTaskAction.INSTANCE, - UpdatePersistentTaskStatusAction.INSTANCE, - RemovePersistentTaskAction.INSTANCE, - CompletionPersistentTaskAction.INSTANCE, // security ClearRealmCacheAction.INSTANCE, ClearRolesCacheAction.INSTANCE, @@ -324,18 +313,12 @@ public List getNamedWriteables() { // ML - Custom metadata new NamedWriteableRegistry.Entry(MetaData.Custom.class, "ml", MlMetadata::new), new NamedWriteableRegistry.Entry(NamedDiff.class, "ml", MlMetadata.MlMetadataDiff::new), - new NamedWriteableRegistry.Entry(MetaData.Custom.class, PersistentTasksCustomMetaData.TYPE, - PersistentTasksCustomMetaData::new), - new NamedWriteableRegistry.Entry(NamedDiff.class, PersistentTasksCustomMetaData.TYPE, - PersistentTasksCustomMetaData::readDiffFrom), // ML - Persistent action requests new NamedWriteableRegistry.Entry(PersistentTaskParams.class, StartDatafeedAction.TASK_NAME, StartDatafeedAction.DatafeedParams::new), new NamedWriteableRegistry.Entry(PersistentTaskParams.class, OpenJobAction.TASK_NAME, OpenJobAction.JobParams::new), // ML - Task statuses - new NamedWriteableRegistry.Entry(Task.Status.class, PersistentTasksNodeService.Status.NAME, - PersistentTasksNodeService.Status::new), new NamedWriteableRegistry.Entry(Task.Status.class, JobTaskStatus.NAME, JobTaskStatus::new), new NamedWriteableRegistry.Entry(Task.Status.class, DatafeedState.NAME, DatafeedState::fromStream), new NamedWriteableRegistry.Entry(XPackFeatureSet.Usage.class, XPackField.MACHINE_LEARNING, @@ -370,8 +353,6 @@ public List getNamedXContent() { // ML - Custom metadata new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField("ml"), parser -> MlMetadata.METADATA_PARSER.parse(parser, null).build()), - new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(PersistentTasksCustomMetaData.TYPE), - PersistentTasksCustomMetaData::fromXContent), // ML - Persistent action requests new NamedXContentRegistry.Entry(PersistentTaskParams.class, new ParseField(StartDatafeedAction.TASK_NAME), StartDatafeedAction.DatafeedParams::fromXContent), @@ -387,8 +368,7 @@ public List getNamedXContent() { new NamedXContentRegistry.Entry(MetaData.Custom.class, new ParseField(LicensesMetaData.TYPE), LicensesMetaData::fromXContent), //rollup - new NamedXContentRegistry.Entry(PersistentTaskParams.class, new ParseField(RollupField.TASK_NAME), - parser -> RollupJob.fromXContent(parser)), + new NamedXContentRegistry.Entry(PersistentTaskParams.class, new ParseField(RollupField.TASK_NAME), RollupJob::fromXContent), new NamedXContentRegistry.Entry(Task.Status.class, new ParseField(RollupJobStatus.NAME), RollupJobStatus::fromXContent) ); } diff --git a/x-pack/qa/ml-native-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java b/x-pack/qa/ml-native-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java index f3b29a48aa964..f70efc72506d3 100644 --- a/x-pack/qa/ml-native-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java +++ b/x-pack/qa/ml-native-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java @@ -26,8 +26,6 @@ import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.persistent.PersistentTaskParams; -import org.elasticsearch.persistent.PersistentTasksCustomMetaData; -import org.elasticsearch.persistent.PersistentTasksNodeService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; @@ -447,14 +445,10 @@ protected void ensureClusterStateConsistency() throws IOException { List entries = new ArrayList<>(ClusterModule.getNamedWriteables()); entries.addAll(new SearchModule(Settings.EMPTY, true, Collections.emptyList()).getNamedWriteables()); entries.add(new NamedWriteableRegistry.Entry(MetaData.Custom.class, "ml", MlMetadata::new)); - entries.add(new NamedWriteableRegistry.Entry(MetaData.Custom.class, PersistentTasksCustomMetaData.TYPE, - PersistentTasksCustomMetaData::new)); entries.add(new NamedWriteableRegistry.Entry(PersistentTaskParams.class, StartDatafeedAction.TASK_NAME, StartDatafeedAction.DatafeedParams::new)); entries.add(new NamedWriteableRegistry.Entry(PersistentTaskParams.class, OpenJobAction.TASK_NAME, OpenJobAction.JobParams::new)); - entries.add(new NamedWriteableRegistry.Entry(Task.Status.class, PersistentTasksNodeService.Status.NAME, - PersistentTasksNodeService.Status::new)); entries.add(new NamedWriteableRegistry.Entry(Task.Status.class, JobTaskStatus.NAME, JobTaskStatus::new)); entries.add(new NamedWriteableRegistry.Entry(Task.Status.class, DatafeedState.NAME, DatafeedState::fromStream)); entries.add(new NamedWriteableRegistry.Entry(ClusterState.Custom.class, TokenMetaData.TYPE, TokenMetaData::new)); From 3f78b3f5e1b1166f3566363616ca6e8ec1ba1b20 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 24 May 2018 11:20:00 +0200 Subject: [PATCH 06/20] [Docs] Explain incomplete dates in range queries (#30689) The current documentation isn't very clear about how incomplete dates are treated when specifying custom formats in a `range` query. This change adds a note explaining how missing month or year coordinates translate to dates that have the missings slots filled with unix time start date (1970-01-01) Closes #30634 --- docs/reference/query-dsl/range-query.asciidoc | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/reference/query-dsl/range-query.asciidoc b/docs/reference/query-dsl/range-query.asciidoc index a0005ff3ff22f..620a175ff39a5 100644 --- a/docs/reference/query-dsl/range-query.asciidoc +++ b/docs/reference/query-dsl/range-query.asciidoc @@ -109,6 +109,12 @@ GET _search -------------------------------------------------- // CONSOLE +Note that if the date misses some of the year, month and day coordinates, the +missing parts are filled with the start of +https://en.wikipedia.org/wiki/Unix_time[unix time], which is January 1st, 1970. +This means, that when e.g. specifying `dd` as the format, a value like `"gte" : 10` +will translate to `1970-01-10T00:00:00.000Z`. + ===== Time zone in range queries Dates can be converted from another timezone to UTC either by specifying the From 39c4f89f9b0b237ae986b6a51e381f87706fe81d Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 24 May 2018 15:16:20 +0200 Subject: [PATCH 07/20] Move Watcher versioning setting to meta field (#30832) The .watcher-history-* template is currently using a plugin-custom index setting xpack.watcher.template.version, which prevents this template from being installed in a mixed OSS / X-Pack cluster, ultimately leading to the situation where an X-Pack node is constantly spamming an OSS master with (failed) template updates. Other X-Pack templates (e.g. security-index-template or security_audit_log) achieve the same versioning functionality by using a custom _meta field in the mapping instead. This commit switches the .watcher-history-* template to use the _meta field instead. --- x-pack/plugin/core/src/main/resources/watch-history.json | 4 +++- .../main/java/org/elasticsearch/xpack/watcher/Watcher.java | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/main/resources/watch-history.json b/x-pack/plugin/core/src/main/resources/watch-history.json index d158281c264d2..86a967fc14fe5 100644 --- a/x-pack/plugin/core/src/main/resources/watch-history.json +++ b/x-pack/plugin/core/src/main/resources/watch-history.json @@ -2,7 +2,6 @@ "index_patterns": [ ".watcher-history-${xpack.watcher.template.version}*" ], "order": 2147483647, "settings": { - "xpack.watcher.template.version": "${xpack.watcher.template.version}", "index.number_of_shards": 1, "index.number_of_replicas": 0, "index.auto_expand_replicas": "0-1", @@ -10,6 +9,9 @@ }, "mappings": { "doc": { + "_meta": { + "watcher-history-version": "${xpack.watcher.template.version}" + }, "dynamic_templates": [ { "disabled_payload_fields": { diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java index 9a18b6c857d56..f7d51d328a797 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java @@ -196,6 +196,8 @@ public class Watcher extends Plugin implements ActionPlugin, ScriptPlugin { + // This setting is only here for backward compatibility reasons as 6.x indices made use of it. It can be removed in 8.x. + @Deprecated public static final Setting INDEX_WATCHER_TEMPLATE_VERSION_SETTING = new Setting<>("index.xpack.watcher.template.version", "", Function.identity(), Setting.Property.IndexScope); public static final Setting ENCRYPT_SENSITIVE_DATA_SETTING = From 8bbfdf1f459706156f58ed56885cc8f82b61811d Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 24 May 2018 17:02:47 +0200 Subject: [PATCH 08/20] Use remote client in TransportFieldCapsAction (#30838) We now have a remote cluster client exposed which can talk to a given remote cluster and manages reconnects etc. This makes code more readable than using the transport layer directly. --- .../TransportFieldCapabilitiesAction.java | 60 +++++-------------- 1 file changed, 14 insertions(+), 46 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java index 2313d9f5fc690..2adea56730ee4 100644 --- a/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java +++ b/server/src/main/java/org/elasticsearch/action/fieldcaps/TransportFieldCapabilitiesAction.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; @@ -33,10 +34,6 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteClusterService; -import org.elasticsearch.transport.Transport; -import org.elasticsearch.transport.TransportException; -import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import java.util.ArrayList; @@ -49,7 +46,6 @@ public class TransportFieldCapabilitiesAction extends HandledTransportAction remoteIndices : remoteClusterIndices.entrySet()) { String clusterAlias = remoteIndices.getKey(); OriginalIndices originalIndices = remoteIndices.getValue(); - // if we are connected this is basically a no-op, if we are not we try to connect in parallel in a non-blocking fashion - remoteClusterService.ensureConnected(clusterAlias, ActionListener.wrap(v -> { - Transport.Connection connection = remoteClusterService.getConnection(clusterAlias); - FieldCapabilitiesRequest remoteRequest = new FieldCapabilitiesRequest(); - remoteRequest.setMergeResults(false); // we need to merge on this node - remoteRequest.indicesOptions(originalIndices.indicesOptions()); - remoteRequest.indices(originalIndices.indices()); - remoteRequest.fields(request.fields()); - transportService.sendRequest(connection, FieldCapabilitiesAction.NAME, remoteRequest, TransportRequestOptions.EMPTY, - new TransportResponseHandler() { - - @Override - public FieldCapabilitiesResponse newInstance() { - return new FieldCapabilitiesResponse(); - } - - @Override - public void handleResponse(FieldCapabilitiesResponse response) { - try { - for (FieldCapabilitiesIndexResponse res : response.getIndexResponses()) { - indexResponses.add(new FieldCapabilitiesIndexResponse(RemoteClusterAware. - buildRemoteIndexName(clusterAlias, res.getIndexName()), res.get())); - } - } finally { - onResponse.run(); - } - } - - @Override - public void handleException(TransportException exp) { - onResponse.run(); - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; - } - }); - }, e -> onResponse.run())); + Client remoteClusterClient = remoteClusterService.getRemoteClusterClient(threadPool, clusterAlias); + FieldCapabilitiesRequest remoteRequest = new FieldCapabilitiesRequest(); + remoteRequest.setMergeResults(false); // we need to merge on this node + remoteRequest.indicesOptions(originalIndices.indicesOptions()); + remoteRequest.indices(originalIndices.indices()); + remoteRequest.fields(request.fields()); + remoteClusterClient.fieldCaps(remoteRequest, ActionListener.wrap(response -> { + for (FieldCapabilitiesIndexResponse res : response.getIndexResponses()) { + indexResponses.add(new FieldCapabilitiesIndexResponse(RemoteClusterAware. + buildRemoteIndexName(clusterAlias, res.getIndexName()), res.get())); + } + onResponse.run(); + }, failure -> onResponse.run())); } - } } From 638a719370eba388c1c2c901639c4c454faa3346 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Thu, 24 May 2018 08:55:14 -0700 Subject: [PATCH 09/20] Ensure that ip_range aggregations always return bucket keys. (#30701) --- .../bucket/iprange-aggregation.asciidoc | 2 + .../test/search.aggregation/40_range.yml | 28 +++++---- .../bucket/range/InternalBinaryRange.java | 59 +++++++++---------- .../bucket/range/ParsedBinaryRange.java | 25 ++------ .../search/aggregations/bucket/IpRangeIT.java | 26 +++++--- .../range/InternalBinaryRangeTests.java | 11 ++++ 6 files changed, 80 insertions(+), 71 deletions(-) diff --git a/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc b/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc index c8bd896b037fa..0aabd3a71ed30 100644 --- a/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/iprange-aggregation.asciidoc @@ -37,10 +37,12 @@ Response: "ip_ranges": { "buckets" : [ { + "key": "*-10.0.0.5", "to": "10.0.0.5", "doc_count": 10 }, { + "key": "10.0.0.5-*", "from": "10.0.0.5", "doc_count": 260 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml index 9a07e6f8ad580..bc845928a0460 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml @@ -144,25 +144,18 @@ setup: - length: { aggregations.ip_range.buckets: 3 } -# ip_range does not automatically add keys to buckets, see #21045 -# - match: { aggregations.ip_range.buckets.0.key: "*-192.168.0.0" } - - is_false: aggregations.ip_range.buckets.0.from - match: { aggregations.ip_range.buckets.0.to: "192.168.0.0" } - match: { aggregations.ip_range.buckets.0.doc_count: 1 } -# - match: { aggregations.ip_range.buckets.1.key: "192.168.0.0-192.169.0.0" } - - match: { aggregations.ip_range.buckets.1.from: "192.168.0.0" } - match: { aggregations.ip_range.buckets.1.to: "192.169.0.0" } - match: { aggregations.ip_range.buckets.1.doc_count: 2 } -# - match: { aggregations.ip_range.buckets.2.key: "192.169.0.0-*" } - - match: { aggregations.ip_range.buckets.2.from: "192.169.0.0" } - is_false: aggregations.ip_range.buckets.2.to @@ -177,24 +170,18 @@ setup: - length: { aggregations.ip_range.buckets: 3 } -# - match: { aggregations.ip_range.buckets.0.key: "*-192.168.0.0" } - - is_false: aggregations.ip_range.buckets.0.from - match: { aggregations.ip_range.buckets.0.to: "192.168.0.0" } - match: { aggregations.ip_range.buckets.0.doc_count: 1 } -# - match: { aggregations.ip_range.buckets.1.key: "192.168.0.0-192.169.0.0" } - - match: { aggregations.ip_range.buckets.1.from: "192.168.0.0" } - match: { aggregations.ip_range.buckets.1.to: "192.169.0.0" } - match: { aggregations.ip_range.buckets.1.doc_count: 2 } -# - match: { aggregations.ip_range.buckets.2.key: "192.169.0.0-*" } - - match: { aggregations.ip_range.buckets.2.from: "192.169.0.0" } - is_false: aggregations.ip_range.buckets.2.to @@ -223,6 +210,21 @@ setup: - match: { aggregations.ip_range.buckets.1.doc_count: 2 } +--- +"IP Range Key Generation": + - skip: + version: " - 6.99.99" + reason: "Before 7.0.0, ip_range did not always generate bucket keys (see #21045)." + + - do: + search: + body: { "size" : 0, "aggs" : { "ip_range" : { "ip_range" : { "field" : "ip", "ranges": [ { "to": "192.168.0.0" }, { "from": "192.168.0.0", "to": "192.169.0.0" }, { "from": "192.169.0.0" } ] } } } } + + - length: { aggregations.ip_range.buckets: 3 } + - match: { aggregations.ip_range.buckets.0.key: "*-192.168.0.0" } + - match: { aggregations.ip_range.buckets.1.key: "192.168.0.0-192.169.0.0" } + - match: { aggregations.ip_range.buckets.2.key: "192.169.0.0-*" } + --- "Date range": - do: diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRange.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRange.java index afa3be702cc33..c647a38f7e06e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRange.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRange.java @@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.bucket.range; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -57,35 +58,41 @@ public Bucket(DocValueFormat format, boolean keyed, String key, BytesRef from, B long docCount, InternalAggregations aggregations) { this.format = format; this.keyed = keyed; - this.key = key; + this.key = key != null ? key : generateKey(from, to, format); this.from = from; this.to = to; this.docCount = docCount; this.aggregations = aggregations; } - // for serialization - private Bucket(StreamInput in, DocValueFormat format, boolean keyed) throws IOException { - this.format = format; - this.keyed = keyed; - key = in.readOptionalString(); - if (in.readBoolean()) { - from = in.readBytesRef(); - } else { - from = null; - } - if (in.readBoolean()) { - to = in.readBytesRef(); - } else { - to = null; - } - docCount = in.readLong(); - aggregations = InternalAggregations.readAggregations(in); + private static String generateKey(BytesRef from, BytesRef to, DocValueFormat format) { + StringBuilder builder = new StringBuilder() + .append(from == null ? "*" : format.format(from)) + .append("-") + .append(to == null ? "*" : format.format(to)); + return builder.toString(); + } + + private static Bucket createFromStream(StreamInput in, DocValueFormat format, boolean keyed) throws IOException { + String key = in.getVersion().onOrAfter(Version.V_7_0_0_alpha1) + ? in.readString() + : in.readOptionalString(); + + BytesRef from = in.readBoolean() ? in.readBytesRef() : null; + BytesRef to = in.readBoolean() ? in.readBytesRef() : null; + long docCount = in.readLong(); + InternalAggregations aggregations = InternalAggregations.readAggregations(in); + + return new Bucket(format, keyed, key, from, to, docCount, aggregations); } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalString(key); + if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + out.writeString(key); + } else { + out.writeOptionalString(key); + } out.writeBoolean(from != null); if (from != null) { out.writeBytesRef(from); @@ -122,19 +129,10 @@ public Aggregations getAggregations() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { String key = this.key; if (keyed) { - if (key == null) { - StringBuilder keyBuilder = new StringBuilder(); - keyBuilder.append(from == null ? "*" : format.format(from)); - keyBuilder.append("-"); - keyBuilder.append(to == null ? "*" : format.format(to)); - key = keyBuilder.toString(); - } builder.startObject(key); } else { builder.startObject(); - if (key != null) { - builder.field(CommonFields.KEY.getPreferredName(), key); - } + builder.field(CommonFields.KEY.getPreferredName(), key); } if (from != null) { builder.field(CommonFields.FROM.getPreferredName(), getFrom()); @@ -208,10 +206,9 @@ public InternalBinaryRange(StreamInput in) throws IOException { super(in); format = in.readNamedWriteable(DocValueFormat.class); keyed = in.readBoolean(); - buckets = in.readList(stream -> new Bucket(stream, format, keyed)); + buckets = in.readList(stream -> Bucket.createFromStream(stream, format, keyed)); } - @Override protected void doWriteTo(StreamOutput out) throws IOException { out.writeNamedWriteable(format); diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ParsedBinaryRange.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ParsedBinaryRange.java index ccfe3f3670f91..79b1cd6cc0d09 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ParsedBinaryRange.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ParsedBinaryRange.java @@ -98,18 +98,16 @@ public String getToAsString() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { if (isKeyed()) { - builder.startObject(key != null ? key : rangeKey(from, to)); + builder.startObject(key); } else { builder.startObject(); - if (key != null) { - builder.field(CommonFields.KEY.getPreferredName(), key); - } + builder.field(CommonFields.KEY.getPreferredName(), key); } if (from != null) { - builder.field(CommonFields.FROM.getPreferredName(), getFrom()); + builder.field(CommonFields.FROM.getPreferredName(), from); } if (to != null) { - builder.field(CommonFields.TO.getPreferredName(), getTo()); + builder.field(CommonFields.TO.getPreferredName(), to); } builder.field(CommonFields.DOC_COUNT.getPreferredName(), getDocCount()); getAggregations().toXContentInternal(builder, params); @@ -123,10 +121,9 @@ static ParsedBucket fromXContent(final XContentParser parser, final boolean keye XContentParser.Token token = parser.currentToken(); String currentFieldName = parser.currentName(); - String rangeKey = null; if (keyed) { ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser::getTokenLocation); - rangeKey = currentFieldName; + bucket.key = currentFieldName; ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser::getTokenLocation); } @@ -150,19 +147,7 @@ static ParsedBucket fromXContent(final XContentParser parser, final boolean keye } } bucket.setAggregations(new Aggregations(aggregations)); - - if (keyed) { - if (rangeKey(bucket.from, bucket.to).equals(rangeKey)) { - bucket.key = null; - } else { - bucket.key = rangeKey; - } - } return bucket; } - - private static String rangeKey(String from, String to) { - return (from == null ? "*" : from) + '-' + (to == null ? "*" : to); - } } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/IpRangeIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/IpRangeIT.java index 17450b31450d5..ffa7f97015105 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/IpRangeIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/IpRangeIT.java @@ -18,14 +18,8 @@ */ package org.elasticsearch.search.aggregations.bucket; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.function.Function; - -import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchPhaseExecutionException; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.MockScriptPlugin; @@ -35,6 +29,12 @@ import org.elasticsearch.search.aggregations.bucket.range.Range; import org.elasticsearch.test.ESIntegTestCase; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.function.Function; + import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; import static org.hamcrest.Matchers.containsString; @@ -91,16 +91,19 @@ public void testSingleValuedField() { Range.Bucket bucket1 = range.getBuckets().get(0); assertNull(bucket1.getFrom()); assertEquals("192.168.1.0", bucket1.getTo()); + assertEquals("*-192.168.1.0", bucket1.getKey()); assertEquals(0, bucket1.getDocCount()); Range.Bucket bucket2 = range.getBuckets().get(1); assertEquals("192.168.1.0", bucket2.getFrom()); assertEquals("192.168.1.10", bucket2.getTo()); + assertEquals("192.168.1.0-192.168.1.10", bucket2.getKey()); assertEquals(1, bucket2.getDocCount()); Range.Bucket bucket3 = range.getBuckets().get(2); assertEquals("192.168.1.10", bucket3.getFrom()); assertNull(bucket3.getTo()); + assertEquals("192.168.1.10-*", bucket3.getKey()); assertEquals(2, bucket3.getDocCount()); } @@ -118,16 +121,19 @@ public void testMultiValuedField() { Range.Bucket bucket1 = range.getBuckets().get(0); assertNull(bucket1.getFrom()); assertEquals("192.168.1.0", bucket1.getTo()); + assertEquals("*-192.168.1.0", bucket1.getKey()); assertEquals(1, bucket1.getDocCount()); Range.Bucket bucket2 = range.getBuckets().get(1); assertEquals("192.168.1.0", bucket2.getFrom()); assertEquals("192.168.1.10", bucket2.getTo()); + assertEquals("192.168.1.0-192.168.1.10", bucket2.getKey()); assertEquals(1, bucket2.getDocCount()); Range.Bucket bucket3 = range.getBuckets().get(2); assertEquals("192.168.1.10", bucket3.getFrom()); assertNull(bucket3.getTo()); + assertEquals("192.168.1.10-*", bucket3.getKey()); assertEquals(2, bucket3.getDocCount()); } @@ -169,16 +175,19 @@ public void testPartiallyUnmapped() { Range.Bucket bucket1 = range.getBuckets().get(0); assertNull(bucket1.getFrom()); assertEquals("192.168.1.0", bucket1.getTo()); + assertEquals("*-192.168.1.0", bucket1.getKey()); assertEquals(0, bucket1.getDocCount()); Range.Bucket bucket2 = range.getBuckets().get(1); assertEquals("192.168.1.0", bucket2.getFrom()); assertEquals("192.168.1.10", bucket2.getTo()); + assertEquals("192.168.1.0-192.168.1.10", bucket2.getKey()); assertEquals(1, bucket2.getDocCount()); Range.Bucket bucket3 = range.getBuckets().get(2); assertEquals("192.168.1.10", bucket3.getFrom()); assertNull(bucket3.getTo()); + assertEquals("192.168.1.10-*", bucket3.getKey()); assertEquals(2, bucket3.getDocCount()); } @@ -196,16 +205,19 @@ public void testUnmapped() { Range.Bucket bucket1 = range.getBuckets().get(0); assertNull(bucket1.getFrom()); assertEquals("192.168.1.0", bucket1.getTo()); + assertEquals("*-192.168.1.0", bucket1.getKey()); assertEquals(0, bucket1.getDocCount()); Range.Bucket bucket2 = range.getBuckets().get(1); assertEquals("192.168.1.0", bucket2.getFrom()); assertEquals("192.168.1.10", bucket2.getTo()); + assertEquals("192.168.1.0-192.168.1.10", bucket2.getKey()); assertEquals(0, bucket2.getDocCount()); Range.Bucket bucket3 = range.getBuckets().get(2); assertEquals("192.168.1.10", bucket3.getFrom()); assertNull(bucket3.getTo()); + assertEquals("192.168.1.10-*", bucket3.getKey()); assertEquals(0, bucket3.getDocCount()); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java index 68785fc387661..00d0c7e050908 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRangeTests.java @@ -157,4 +157,15 @@ protected InternalBinaryRange mutateInstance(InternalBinaryRange instance) { } return new InternalBinaryRange(name, format, keyed, buckets, pipelineAggregators, metaData); } + + /** + * Checks the invariant that bucket keys are always non-null, even if null keys + * were originally provided. + */ + public void testKeyGeneration() { + InternalBinaryRange range = createTestInstance(); + for (InternalBinaryRange.Bucket bucket : range.getBuckets()) { + assertNotNull(bucket.getKey()); + } + } } From 2b8d3e852007fcef1c77ca0d60dcdb521ebfd73a Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Thu, 24 May 2018 10:28:46 -0600 Subject: [PATCH 10/20] Security: fix dynamic mapping updates with aliases (#30787) This commit fixes an issue with dynamic mapping updates when an index operation is performed against an alias and when the user only has permissions to the alias. Dynamic mapping updates resolve the concrete index early to prevent issues so the information about the alias that the triggering operation was being executed against is lost. When security is enabled and a user only has privileges to the alias, this dynamic mapping update would be rejected as it is executing against the concrete index and not the alias. In order to handle this situation, the security code needs to look at the concrete index and the authorized indices of the user; if the concrete index is not authorized the code will attempt to find an alias that the user has permissions to update the mappings of. Closes #30597 --- .../authz/IndicesAndAliasesResolver.java | 55 ++++++++++-- .../authz/IndicesAndAliasesResolverTests.java | 30 +++++-- .../security/authz/30_dynamic_put_mapping.yml | 90 +++++++++++++++++++ 3 files changed, 165 insertions(+), 10 deletions(-) create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/30_dynamic_put_mapping.yml diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java index 4c4d0afc10d86..941bf13daf584 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolver.java @@ -14,11 +14,14 @@ import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.cluster.metadata.AliasMetaData; import org.elasticsearch.cluster.metadata.AliasOrIndex; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; @@ -35,6 +38,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.concurrent.CopyOnWriteArraySet; @@ -42,7 +46,7 @@ import static org.elasticsearch.xpack.core.security.authz.IndicesAndAliasesResolverField.NO_INDEX_PLACEHOLDER; -public class IndicesAndAliasesResolver { +class IndicesAndAliasesResolver { //`*,-*` what we replace indices with if we need Elasticsearch to return empty responses without throwing exception private static final String[] NO_INDICES_ARRAY = new String[] { "*", "-*" }; @@ -51,7 +55,7 @@ public class IndicesAndAliasesResolver { private final IndexNameExpressionResolver nameExpressionResolver; private final RemoteClusterResolver remoteClusterResolver; - public IndicesAndAliasesResolver(Settings settings, ClusterService clusterService) { + IndicesAndAliasesResolver(Settings settings, ClusterService clusterService) { this.nameExpressionResolver = new IndexNameExpressionResolver(settings); this.remoteClusterResolver = new RemoteClusterResolver(settings, clusterService.getClusterSettings()); } @@ -85,7 +89,7 @@ public IndicesAndAliasesResolver(Settings settings, ClusterService clusterServic * Otherwise, N will be added to the local index list. */ - public ResolvedIndices resolve(TransportRequest request, MetaData metaData, AuthorizedIndices authorizedIndices) { + ResolvedIndices resolve(TransportRequest request, MetaData metaData, AuthorizedIndices authorizedIndices) { if (request instanceof IndicesAliasesRequest) { ResolvedIndices.Builder resolvedIndicesBuilder = new ResolvedIndices.Builder(); IndicesAliasesRequest indicesAliasesRequest = (IndicesAliasesRequest) request; @@ -116,7 +120,7 @@ ResolvedIndices resolveIndicesAndAliases(IndicesRequest indicesRequest, MetaData */ assert indicesRequest.indices() == null || indicesRequest.indices().length == 0 : "indices are: " + Arrays.toString(indicesRequest.indices()); // Arrays.toString() can handle null values - all good - resolvedIndicesBuilder.addLocal(((PutMappingRequest) indicesRequest).getConcreteIndex().getName()); + resolvedIndicesBuilder.addLocal(getPutMappingIndexOrAlias((PutMappingRequest) indicesRequest, authorizedIndices, metaData)); } else if (indicesRequest instanceof IndicesRequest.Replaceable) { IndicesRequest.Replaceable replaceable = (IndicesRequest.Replaceable) indicesRequest; final boolean replaceWildcards = indicesRequest.indicesOptions().expandWildcardsOpen() @@ -213,7 +217,48 @@ ResolvedIndices resolveIndicesAndAliases(IndicesRequest indicesRequest, MetaData return resolvedIndicesBuilder.build(); } - public static boolean allowsRemoteIndices(IndicesRequest request) { + /** + * Special handling of the value to authorize for a put mapping request. Dynamic put mapping + * requests use a concrete index, but we allow permissions to be defined on aliases so if the + * request's concrete index is not in the list of authorized indices, then we need to look to + * see if this can be authorized against an alias + */ + static String getPutMappingIndexOrAlias(PutMappingRequest request, AuthorizedIndices authorizedIndices, MetaData metaData) { + final String concreteIndexName = request.getConcreteIndex().getName(); + final List authorizedIndicesList = authorizedIndices.get(); + + // validate that the concrete index exists, otherwise there is no remapping that we could do + final AliasOrIndex aliasOrIndex = metaData.getAliasAndIndexLookup().get(concreteIndexName); + final String resolvedAliasOrIndex; + if (aliasOrIndex == null) { + resolvedAliasOrIndex = concreteIndexName; + } else if (aliasOrIndex.isAlias()) { + throw new IllegalStateException("concrete index [" + concreteIndexName + "] is an alias but should not be"); + } else if (authorizedIndicesList.contains(concreteIndexName)) { + // user is authorized to put mappings for this index + resolvedAliasOrIndex = concreteIndexName; + } else { + // the user is not authorized to put mappings for this index, but could have been + // authorized for a write using an alias that triggered a dynamic mapping update + ImmutableOpenMap> foundAliases = + metaData.findAliases(Strings.EMPTY_ARRAY, new String[] { concreteIndexName }); + List aliasMetaData = foundAliases.get(concreteIndexName); + if (aliasMetaData != null) { + Optional foundAlias = aliasMetaData.stream() + .map(AliasMetaData::alias) + .filter(authorizedIndicesList::contains) + .filter(aliasName -> metaData.getAliasAndIndexLookup().get(aliasName).getIndices().size() == 1) + .findFirst(); + resolvedAliasOrIndex = foundAlias.orElse(concreteIndexName); + } else { + resolvedAliasOrIndex = concreteIndexName; + } + } + + return resolvedAliasOrIndex; + } + + static boolean allowsRemoteIndices(IndicesRequest request) { return request instanceof SearchRequest || request instanceof FieldCapabilitiesRequest || request instanceof GraphExploreRequest; } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index b080b5924ce7a..d7c974bdc6e2a 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -39,10 +39,12 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.search.internal.ShardSearchTransportRequest; import org.elasticsearch.test.ESTestCase; @@ -149,7 +151,10 @@ public void setup() { new IndicesPrivileges[] { IndicesPrivileges.builder().indices(authorizedIndices).privileges("all").build() }, null)); roleMap.put("dash", new RoleDescriptor("dash", null, new IndicesPrivileges[] { IndicesPrivileges.builder().indices(dashIndices).privileges("all").build() }, null)); - roleMap.put("test", new RoleDescriptor("role", new String[] { "monitor" }, null, null)); + roleMap.put("test", new RoleDescriptor("test", new String[] { "monitor" }, null, null)); + roleMap.put("alias_read_write", new RoleDescriptor("alias_read_write", null, + new IndicesPrivileges[] { IndicesPrivileges.builder().indices("barbaz", "foofoobar").privileges("read", "write").build() }, + null)); roleMap.put(ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR.getName(), ReservedRolesStore.SUPERUSER_ROLE_DESCRIPTOR); final FieldPermissionsCache fieldPermissionsCache = new FieldPermissionsCache(Settings.EMPTY); doAnswer((i) -> { @@ -651,7 +656,7 @@ public void testResolveWildcardsIndicesAliasesRequestNoMatchingIndices() { request.addAliasAction(AliasActions.add().alias("alias2").index("bar*")); request.addAliasAction(AliasActions.add().alias("alias3").index("non_matching_*")); //if a single operation contains wildcards and ends up being resolved to no indices, it makes the whole request fail - expectThrows(IndexNotFoundException.class, + expectThrows(IndexNotFoundException.class, () -> resolveIndices(request, buildAuthorizedIndices(user, IndicesAliasesAction.NAME))); } @@ -1180,10 +1185,10 @@ public void testIndicesExists() { assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(userNoIndices, IndicesExistsAction.NAME))); } - + { IndicesExistsRequest request = new IndicesExistsRequest("does_not_exist"); - + assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, IndicesExistsAction.NAME))); } @@ -1228,7 +1233,7 @@ public void testNonXPackUserAccessingSecurityIndex() { List indices = resolveIndices(request, authorizedIndices).getLocal(); assertThat(indices, not(hasItem(SecurityIndexManager.SECURITY_INDEX_NAME))); } - + { IndicesAliasesRequest aliasesRequest = new IndicesAliasesRequest(); aliasesRequest.addAliasAction(AliasActions.add().alias("security_alias1").index("*")); @@ -1317,6 +1322,21 @@ public void testAliasDateMathExpressionNotSupported() { assertThat(request.aliases(), arrayContainingInAnyOrder("")); } + public void testDynamicPutMappingRequestFromAlias() { + PutMappingRequest request = new PutMappingRequest(Strings.EMPTY_ARRAY).setConcreteIndex(new Index("foofoo", UUIDs.base64UUID())); + User user = new User("alias-writer", "alias_read_write"); + AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, PutMappingAction.NAME); + + String putMappingIndexOrAlias = IndicesAndAliasesResolver.getPutMappingIndexOrAlias(request, authorizedIndices, metaData); + assertEquals("barbaz", putMappingIndexOrAlias); + + // multiple indices map to an alias so we can only return the concrete index + final String index = randomFrom("foo", "foobar"); + request = new PutMappingRequest(Strings.EMPTY_ARRAY).setConcreteIndex(new Index(index, UUIDs.base64UUID())); + putMappingIndexOrAlias = IndicesAndAliasesResolver.getPutMappingIndexOrAlias(request, authorizedIndices, metaData); + assertEquals(index, putMappingIndexOrAlias); + } + // TODO with the removal of DeleteByQuery is there another way to test resolving a write action? diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/30_dynamic_put_mapping.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/30_dynamic_put_mapping.yml new file mode 100644 index 0000000000000..3fca1ee563305 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/security/authz/30_dynamic_put_mapping.yml @@ -0,0 +1,90 @@ +--- +setup: + - skip: + features: headers + + - do: + cluster.health: + wait_for_status: yellow + + - do: + xpack.security.put_role: + name: "alias_write_role" + body: > + { + "indices": [ + { "names": ["write_alias"], "privileges": ["write"] } + ] + } + + - do: + xpack.security.put_user: + username: "test_user" + body: > + { + "password" : "x-pack-test-password", + "roles" : [ "alias_write_role" ], + "full_name" : "user with privileges to write via alias" + } + + - do: + indices.create: + index: write_index_1 + body: + settings: + index: + number_of_shards: 1 + number_of_replicas: 0 + + - do: + indices.put_alias: + index: write_index_1 + name: write_alias + +--- +teardown: + - do: + xpack.security.delete_user: + username: "test_user" + ignore: 404 + + - do: + xpack.security.delete_role: + name: "alias_write_role" + ignore: 404 + + - do: + indices.delete_alias: + index: "write_index_1" + name: [ "write_alias" ] + ignore: 404 + + - do: + indices.delete: + index: [ "write_index_1" ] + ignore: 404 + +--- +"Test indexing documents into an alias with dynamic mappings": + + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + create: + id: 1 + index: write_alias + type: doc + body: > + { + "name" : "doc1" + } + + - do: + headers: { Authorization: "Basic dGVzdF91c2VyOngtcGFjay10ZXN0LXBhc3N3b3Jk" } # test_user + create: + id: 2 + index: write_alias + type: doc + body: > + { + "name2" : "doc2" + } From 9cb6b90a99a50aab38b7a8a12944f49ca362aeed Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 24 May 2018 18:31:13 +0200 Subject: [PATCH 11/20] [Tests] Move templated _rank_eval tests (#30679) This change moves the ranking evaluation tests that use templates to the existing yml rest tests instead of extending ESIntegTestCase. Closes #30628 --- .../build.gradle | 3 - .../rankeval/SmokeMultipleTemplatesIT.java | 159 ---------------- .../test/rank-eval/10_rank_eval_templated.yml | 171 ++++++++++++++++++ .../test/rank-eval/30_template.yml | 72 -------- 4 files changed, 171 insertions(+), 234 deletions(-) delete mode 100644 qa/smoke-test-rank-eval-with-mustache/src/test/java/org/elasticsearch/index/rankeval/SmokeMultipleTemplatesIT.java create mode 100644 qa/smoke-test-rank-eval-with-mustache/src/test/resources/rest-api-spec/test/rank-eval/10_rank_eval_templated.yml delete mode 100644 qa/smoke-test-rank-eval-with-mustache/src/test/resources/rest-api-spec/test/rank-eval/30_template.yml diff --git a/qa/smoke-test-rank-eval-with-mustache/build.gradle b/qa/smoke-test-rank-eval-with-mustache/build.gradle index 122c2603719a0..175eb18cff7af 100644 --- a/qa/smoke-test-rank-eval-with-mustache/build.gradle +++ b/qa/smoke-test-rank-eval-with-mustache/build.gradle @@ -31,6 +31,3 @@ dependencies { * and will be fixed later. * Tracked by https://github.com/elastic/elasticsearch/issues/30628 */ -if ("zip".equals(integTestCluster.distribution)) { - integTestRunner.enabled = false -} diff --git a/qa/smoke-test-rank-eval-with-mustache/src/test/java/org/elasticsearch/index/rankeval/SmokeMultipleTemplatesIT.java b/qa/smoke-test-rank-eval-with-mustache/src/test/java/org/elasticsearch/index/rankeval/SmokeMultipleTemplatesIT.java deleted file mode 100644 index 0ad78ad0c7a7e..0000000000000 --- a/qa/smoke-test-rank-eval-with-mustache/src/test/java/org/elasticsearch/index/rankeval/SmokeMultipleTemplatesIT.java +++ /dev/null @@ -1,159 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.rankeval; - -import org.elasticsearch.index.rankeval.RankEvalSpec.ScriptWithId; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.script.Script; -import org.elasticsearch.script.ScriptType; -import org.elasticsearch.test.ESIntegTestCase; -import org.junit.Before; - -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; -import java.util.List; -import java.util.Map; -import java.util.Set; - - -public class SmokeMultipleTemplatesIT extends ESIntegTestCase { - - private static final String MATCH_TEMPLATE = "match_template"; - - @Override - protected Collection> transportClientPlugins() { - return Arrays.asList(RankEvalPlugin.class); - } - - @Override - protected Collection> nodePlugins() { - return Arrays.asList(RankEvalPlugin.class); - } - - @Before - public void setup() { - createIndex("test"); - ensureGreen(); - - client().prepareIndex("test", "testtype").setId("1") - .setSource("text", "berlin", "title", "Berlin, Germany").get(); - client().prepareIndex("test", "testtype").setId("2") - .setSource("text", "amsterdam").get(); - client().prepareIndex("test", "testtype").setId("3") - .setSource("text", "amsterdam").get(); - client().prepareIndex("test", "testtype").setId("4") - .setSource("text", "amsterdam").get(); - client().prepareIndex("test", "testtype").setId("5") - .setSource("text", "amsterdam").get(); - client().prepareIndex("test", "testtype").setId("6") - .setSource("text", "amsterdam").get(); - refresh(); - } - - public void testPrecisionAtRequest() throws IOException { - - List specifications = new ArrayList<>(); - Map ams_params = new HashMap<>(); - ams_params.put("querystring", "amsterdam"); - RatedRequest amsterdamRequest = new RatedRequest( - "amsterdam_query", createRelevant("2", "3", "4", "5"), ams_params, MATCH_TEMPLATE); - - specifications.add(amsterdamRequest); - - Map berlin_params = new HashMap<>(); - berlin_params.put("querystring", "berlin"); - RatedRequest berlinRequest = new RatedRequest( - "berlin_query", createRelevant("1"), berlin_params, MATCH_TEMPLATE); - specifications.add(berlinRequest); - - PrecisionAtK metric = new PrecisionAtK(); - - ScriptWithId template = - new ScriptWithId( - MATCH_TEMPLATE, - new Script( - ScriptType.INLINE, - "mustache", "{\"query\": {\"match\": {\"text\": \"{{querystring}}\"}}}", - new HashMap<>())); - Set templates = new HashSet<>(); - templates.add(template); - RankEvalSpec task = new RankEvalSpec(specifications, metric, templates); - RankEvalRequestBuilder builder = new RankEvalRequestBuilder(client(), RankEvalAction.INSTANCE, new RankEvalRequest()); - builder.setRankEvalSpec(task); - - RankEvalResponse response = client().execute(RankEvalAction.INSTANCE, builder.request().indices("test")).actionGet(); - assertEquals(0.9, response.getEvaluationResult(), Double.MIN_VALUE); - } - - public void testTemplateWithAggsFails() { - String template = "{ \"aggs\" : { \"avg_grade\" : { \"avg\" : { \"field\" : \"grade\" }}}}"; - assertTemplatedRequestFailures(template, "Query in rated requests should not contain aggregations."); - } - - public void testTemplateWithSuggestFails() { - String template = "{\"suggest\" : {\"my-suggestion\" : {\"text\" : \"Elastic\",\"term\" : {\"field\" : \"message\"}}}}"; - assertTemplatedRequestFailures(template, "Query in rated requests should not contain a suggest section."); - } - - public void testTemplateWithHighlighterFails() { - String template = "{\"highlight\" : { \"fields\" : {\"content\" : {}}}}"; - assertTemplatedRequestFailures(template, "Query in rated requests should not contain a highlighter section."); - } - - public void testTemplateWithProfileFails() { - String template = "{\"profile\" : \"true\" }"; - assertTemplatedRequestFailures(template, "Query in rated requests should not use profile."); - } - - public void testTemplateWithExplainFails() { - String template = "{\"explain\" : \"true\" }"; - assertTemplatedRequestFailures(template, "Query in rated requests should not use explain."); - } - - private static void assertTemplatedRequestFailures(String template, String expectedMessage) { - List ratedDocs = Arrays.asList(new RatedDocument("index1", "id1", 1)); - RatedRequest ratedRequest = new RatedRequest("id", ratedDocs, Collections.singletonMap("param1", "value1"), "templateId"); - Collection templates = Collections.singletonList(new ScriptWithId("templateId", - new Script(ScriptType.INLINE, Script.DEFAULT_TEMPLATE_LANG, template, Collections.emptyMap()))); - RankEvalSpec rankEvalSpec = new RankEvalSpec(Collections.singletonList(ratedRequest), new PrecisionAtK(), templates); - RankEvalRequest rankEvalRequest = new RankEvalRequest(rankEvalSpec, new String[] { "test" }); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> client().execute(RankEvalAction.INSTANCE, rankEvalRequest).actionGet()); - assertEquals(expectedMessage, e.getMessage()); - } - - private static List createRelevant(String... docs) { - List relevant = new ArrayList<>(); - for (String doc : docs) { - relevant.add(new RatedDocument("test", doc, Rating.RELEVANT.ordinal())); - } - return relevant; - } - - public enum Rating { - IRRELEVANT, RELEVANT; - } - - } diff --git a/qa/smoke-test-rank-eval-with-mustache/src/test/resources/rest-api-spec/test/rank-eval/10_rank_eval_templated.yml b/qa/smoke-test-rank-eval-with-mustache/src/test/resources/rest-api-spec/test/rank-eval/10_rank_eval_templated.yml new file mode 100644 index 0000000000000..f0c564d363904 --- /dev/null +++ b/qa/smoke-test-rank-eval-with-mustache/src/test/resources/rest-api-spec/test/rank-eval/10_rank_eval_templated.yml @@ -0,0 +1,171 @@ +setup: + + - do: + indices.create: + index: test + body: + settings: + index: + number_of_shards: 1 + - do: + index: + index: test + type: _doc + id: 1 + body: { "text": "berlin", "title" : "Berlin, Germany" } + + - do: + index: + index: test + type: _doc + id: 2 + body: { "text": "amsterdam" } + + - do: + index: + index: test + type: _doc + id: 3 + body: { "text": "amsterdam" } + + - do: + index: + index: test + type: _doc + id: 4 + body: { "text": "amsterdam" } + + - do: + index: + index: test + type: _doc + id: 5 + body: { "text": "amsterdam" } + + - do: + index: + index: test + type: _doc + id: 6 + body: { "text": "amsterdam" } + + - do: + indices.refresh: {} + +--- +"Basic rank-eval request with template": + + - skip: + version: " - 6.1.99" + reason: the ranking evaluation feature is available since 6.2 + + - do: + rank_eval: + body: { + "templates": [ { "id": "match", "template": {"source": "{\"query\": { \"match\" : {\"text\" : \"{{query_string}}\" }}}" }} ], + "requests" : [ + { + "id": "amsterdam_query", + "params": { "query_string": "amsterdam" }, + "template_id": "match", + "ratings": [ + {"_index": "test", "_id": "2", "rating": 1}, + {"_index": "test", "_id": "3", "rating": 1}, + {"_index": "test", "_id": "4", "rating": 1}, + {"_index": "test", "_id": "5", "rating": 1},] + }, + { + "id" : "berlin_query", + "params": { "query_string": "berlin" }, + "template_id": "match", + "ratings": [{"_index": "test", "_id": "1", "rating": 1}] + } + ], + "metric" : { "precision": { }} + } + + - match: {quality_level: 0.9} + - match: {details.amsterdam_query.unknown_docs.0._id: "6"} + +--- +"Test illegal request parts": + + - do: + catch: /Query in rated requests should not contain aggregations./ + rank_eval: + body: { + "templates": [ { "id": "match", "template": {"source": "{ \"aggs\" : { \"avg_grade\" : { \"avg\" : { \"field\" : \"grade\" }}}}" }} ], + "requests" : [ + { + "id": "amsterdam_query", + "params": { "query_string": "amsterdam" }, + "template_id": "match", + "ratings": [] + } + ], + "metric" : { "precision": { }} + } + + - do: + catch: /Query in rated requests should not contain a suggest section./ + rank_eval: + body: { + "templates": [ { "id": "match", "template": {"source": "{\"suggest\" : {\"my-suggestion\" : {\"text\" : \"Elastic\",\"term\" : {\"field\" : \"message\"}}}}" }} ], + "requests" : [ + { + "id": "amsterdam_query", + "params": { "query_string": "amsterdam" }, + "template_id": "match", + "ratings": [] + } + ], + "metric" : { "precision": { }} + } + + - do: + catch: /Query in rated requests should not contain a highlighter section./ + rank_eval: + body: { + "templates": [ { "id": "match", "template": {"source": "{\"highlight\" : { \"fields\" : {\"content\" : {}}}}" }} ], + "requests" : [ + { + "id": "amsterdam_query", + "params": { "query_string": "amsterdam" }, + "template_id": "match", + "ratings": [] + } + ], + "metric" : { "precision": { }} + } + + - do: + catch: /Query in rated requests should not use profile./ + rank_eval: + body: { + "templates": [ { "id": "match", "template": {"source": "{\"profile\" : \"true\" }" }} ], + "requests" : [ + { + "id": "amsterdam_query", + "params": { "query_string": "amsterdam" }, + "template_id": "match", + "ratings": [] + } + ], + "metric" : { "precision": { }} + } + + - do: + catch: /Query in rated requests should not use explain./ + rank_eval: + body: { + "templates": [ { "id": "match", "template": {"source": "{\"explain\" : \"true\" }" }} ], + "requests" : [ + { + "id": "amsterdam_query", + "params": { "query_string": "amsterdam" }, + "template_id": "match", + "ratings": [] + } + ], + "metric" : { "precision": { }} + } diff --git a/qa/smoke-test-rank-eval-with-mustache/src/test/resources/rest-api-spec/test/rank-eval/30_template.yml b/qa/smoke-test-rank-eval-with-mustache/src/test/resources/rest-api-spec/test/rank-eval/30_template.yml deleted file mode 100644 index 692a2e2123058..0000000000000 --- a/qa/smoke-test-rank-eval-with-mustache/src/test/resources/rest-api-spec/test/rank-eval/30_template.yml +++ /dev/null @@ -1,72 +0,0 @@ ---- -"Template request": - - - skip: - version: " - 6.1.99" - reason: the ranking evaluation feature is available since 6.2 - - - do: - indices.create: - index: foo - body: - settings: - index: - number_of_shards: 1 - - do: - index: - index: foo - type: bar - id: doc1 - body: { "text": "berlin" } - - - do: - index: - index: foo - type: bar - id: doc2 - body: { "text": "amsterdam" } - - - do: - index: - index: foo - type: bar - id: doc3 - body: { "text": "amsterdam" } - - - do: - index: - index: foo - type: bar - id: doc4 - body: { "text": "something about amsterdam and berlin" } - - - do: - indices.refresh: {} - - - do: - rank_eval: - body: { - "templates": [ { "id": "match", "template": {"source": "{\"query\": { \"match\" : {\"text\" : \"{{query_string}}\" }}}" }} ], - "requests" : [ - { - "id": "amsterdam_query", - "params": { "query_string": "amsterdam" }, - "template_id": "match", - "ratings": [ - {"_index": "foo", "_id": "doc1", "rating": 0}, - {"_index": "foo", "_id": "doc2", "rating": 1}, - {"_index": "foo", "_id": "doc3", "rating": 1}] - }, - { - "id" : "berlin_query", - "params": { "query_string": "berlin" }, - "template_id": "match", - "ratings": [{"_index": "foo", "_id": "doc1", "rating": 1}] - } - ], - "metric" : { "precision": { }} - } - - - match: {quality_level: 0.5833333333333333} - - match: {details.berlin_query.unknown_docs.0._id: "doc4"} - - match: {details.amsterdam_query.unknown_docs.0._id: "doc4"} From b3a4acdf20b2ab180b82c06c1d4871dcda4155f2 Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Thu, 24 May 2018 10:43:10 -0600 Subject: [PATCH 12/20] Limit user to single concurrent auth per realm (#30794) This commit reworks the way our realms perform caching in order to limit each principal to a single ongoing authentication per realm. In other words, this means that multiple requests made by the same user will not trigger more that one authentication attempt at a time if no entry has been stored in the cache. If an entry is present in our cache, there is no restriction on the number of concurrent authentications performed for this user. This change enables us to limit the load we place on an external system like an LDAP server and also preserve resources such as CPU on expensive operations such as BCrypt authentication. Closes #30355 --- .../org/elasticsearch/common/cache/Cache.java | 79 +++++-- .../util/concurrent/ListenableFuture.java | 115 ++++++++++ .../common/cache/CacheTests.java | 56 +++++ .../concurrent/ListenableFutureTests.java | 118 ++++++++++ .../xpack/security/Security.java | 2 +- .../xpack/security/authc/InternalRealms.java | 4 +- .../security/authc/esnative/NativeRealm.java | 5 +- .../authc/esnative/ReservedRealm.java | 6 +- .../xpack/security/authc/file/FileRealm.java | 9 +- .../xpack/security/authc/ldap/LdapRealm.java | 2 +- .../support/CachingUsernamePasswordRealm.java | 205 ++++++++++------- .../user/TransportGetUsersActionTests.java | 30 ++- .../user/TransportPutUserActionTests.java | 6 +- .../authc/esnative/NativeRealmTests.java | 7 +- .../authc/esnative/ReservedRealmTests.java | 32 +-- .../security/authc/file/FileRealmTests.java | 46 ++-- .../CachingUsernamePasswordRealmTests.java | 217 ++++++++++++------ .../mapper/NativeRoleMappingStoreTests.java | 2 +- 18 files changed, 713 insertions(+), 228 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java create mode 100644 server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java diff --git a/server/src/main/java/org/elasticsearch/common/cache/Cache.java b/server/src/main/java/org/elasticsearch/common/cache/Cache.java index 0db4d718709d8..beb2819f2e6dc 100644 --- a/server/src/main/java/org/elasticsearch/common/cache/Cache.java +++ b/server/src/main/java/org/elasticsearch/common/cache/Cache.java @@ -68,6 +68,7 @@ * @param The type of the values */ public class Cache { + // positive if entries have an expiration private long expireAfterAccessNanos = -1; @@ -282,6 +283,39 @@ void remove(K key, Consumer>> onRemoval) { } } + /** + * remove an entry from the segment iff the future is done and the value is equal to the + * expected value + * + * @param key the key of the entry to remove from the cache + * @param value the value expected to be associated with the key + * @param onRemoval a callback for the removed entry + */ + void remove(K key, V value, Consumer>> onRemoval) { + CompletableFuture> future; + boolean removed = false; + try (ReleasableLock ignored = writeLock.acquire()) { + future = map.get(key); + try { + if (future != null) { + if (future.isDone()) { + Entry entry = future.get(); + if (Objects.equals(value, entry.value)) { + removed = map.remove(key, future); + } + } + } + } catch (ExecutionException | InterruptedException e) { + throw new IllegalStateException(e); + } + } + + if (future != null && removed) { + segmentStats.eviction(); + onRemoval.accept(future); + } + } + private static class SegmentStats { private final LongAdder hits = new LongAdder(); private final LongAdder misses = new LongAdder(); @@ -314,7 +348,7 @@ void eviction() { Entry tail; // lock protecting mutations to the LRU list - private ReleasableLock lruLock = new ReleasableLock(new ReentrantLock()); + private final ReleasableLock lruLock = new ReleasableLock(new ReentrantLock()); /** * Returns the value to which the specified key is mapped, or null if this map contains no mapping for the key. @@ -455,6 +489,19 @@ private void put(K key, V value, long now) { } } + private final Consumer>> invalidationConsumer = f -> { + try { + Entry entry = f.get(); + try (ReleasableLock ignored = lruLock.acquire()) { + delete(entry, RemovalNotification.RemovalReason.INVALIDATED); + } + } catch (ExecutionException e) { + // ok + } catch (InterruptedException e) { + throw new IllegalStateException(e); + } + }; + /** * Invalidate the association for the specified key. A removal notification will be issued for invalidated * entries with {@link org.elasticsearch.common.cache.RemovalNotification.RemovalReason} INVALIDATED. @@ -463,18 +510,20 @@ private void put(K key, V value, long now) { */ public void invalidate(K key) { CacheSegment segment = getCacheSegment(key); - segment.remove(key, f -> { - try { - Entry entry = f.get(); - try (ReleasableLock ignored = lruLock.acquire()) { - delete(entry, RemovalNotification.RemovalReason.INVALIDATED); - } - } catch (ExecutionException e) { - // ok - } catch (InterruptedException e) { - throw new IllegalStateException(e); - } - }); + segment.remove(key, invalidationConsumer); + } + + /** + * Invalidate the entry for the specified key and value. If the value provided is not equal to the value in + * the cache, no removal will occur. A removal notification will be issued for invalidated + * entries with {@link org.elasticsearch.common.cache.RemovalNotification.RemovalReason} INVALIDATED. + * + * @param key the key whose mapping is to be invalidated from the cache + * @param value the expected value that should be associated with the key + */ + public void invalidate(K key, V value) { + CacheSegment segment = getCacheSegment(key); + segment.remove(key, value, invalidationConsumer); } /** @@ -625,7 +674,7 @@ public void remove() { Entry entry = current; if (entry != null) { CacheSegment segment = getCacheSegment(entry.key); - segment.remove(entry.key, f -> {}); + segment.remove(entry.key, entry.value, f -> {}); try (ReleasableLock ignored = lruLock.acquire()) { current = null; delete(entry, RemovalNotification.RemovalReason.INVALIDATED); @@ -710,7 +759,7 @@ private void evictEntry(Entry entry) { CacheSegment segment = getCacheSegment(entry.key); if (segment != null) { - segment.remove(entry.key, f -> {}); + segment.remove(entry.key, entry.value, f -> {}); } delete(entry, RemovalNotification.RemovalReason.EVICTED); } diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java new file mode 100644 index 0000000000000..d50f57aaafaa5 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java @@ -0,0 +1,115 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.util.concurrent; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.collect.Tuple; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * A future implementation that allows for the result to be passed to listeners waiting for + * notification. This is useful for cases where a computation is requested many times + * concurrently, but really only needs to be performed a single time. Once the computation + * has been performed the registered listeners will be notified by submitting a runnable + * for execution in the provided {@link ExecutorService}. If the computation has already + * been performed, a request to add a listener will simply result in execution of the listener + * on the calling thread. + */ +public final class ListenableFuture extends BaseFuture implements ActionListener { + + private volatile boolean done = false; + private final List, ExecutorService>> listeners = new ArrayList<>(); + + /** + * Adds a listener to this future. If the future has not yet completed, the listener will be + * notified of a response or exception in a runnable submitted to the ExecutorService provided. + * If the future has completed, the listener will be notified immediately without forking to + * a different thread. + */ + public void addListener(ActionListener listener, ExecutorService executor) { + if (done) { + // run the callback directly, we don't hold the lock and don't need to fork! + notifyListener(listener, EsExecutors.newDirectExecutorService()); + } else { + final boolean run; + // check done under lock since it could have been modified and protect modifications + // to the list under lock + synchronized (this) { + if (done) { + run = true; + } else { + listeners.add(new Tuple<>(listener, executor)); + run = false; + } + } + + if (run) { + // run the callback directly, we don't hold the lock and don't need to fork! + notifyListener(listener, EsExecutors.newDirectExecutorService()); + } + } + } + + @Override + protected synchronized void done() { + done = true; + listeners.forEach(t -> notifyListener(t.v1(), t.v2())); + // release references to any listeners as we no longer need them and will live + // much longer than the listeners in most cases + listeners.clear(); + } + + private void notifyListener(ActionListener listener, ExecutorService executorService) { + try { + executorService.submit(() -> { + try { + // call get in a non-blocking fashion as we could be on a network thread + // or another thread like the scheduler, which we should never block! + V value = FutureUtils.get(this, 0L, TimeUnit.NANOSECONDS); + listener.onResponse(value); + } catch (Exception e) { + listener.onFailure(e); + } + }); + } catch (Exception e) { + listener.onFailure(e); + } + } + + @Override + public void onResponse(V v) { + final boolean set = set(v); + if (set == false) { + throw new IllegalStateException("did not set value, value or exception already set?"); + } + } + + @Override + public void onFailure(Exception e) { + final boolean set = setException(e); + if (set == false) { + throw new IllegalStateException("did not set exception, value already set or exception already set?"); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/common/cache/CacheTests.java b/server/src/test/java/org/elasticsearch/common/cache/CacheTests.java index fe64fd16af68c..3b183cce40b86 100644 --- a/server/src/test/java/org/elasticsearch/common/cache/CacheTests.java +++ b/server/src/test/java/org/elasticsearch/common/cache/CacheTests.java @@ -457,6 +457,62 @@ public void testNotificationOnInvalidate() { assertEquals(notifications, invalidated); } + // randomly invalidate some cached entries, then check that a lookup for each of those and only those keys is null + public void testInvalidateWithValue() { + Cache cache = CacheBuilder.builder().build(); + for (int i = 0; i < numberOfEntries; i++) { + cache.put(i, Integer.toString(i)); + } + Set keys = new HashSet<>(); + for (Integer key : cache.keys()) { + if (rarely()) { + if (randomBoolean()) { + cache.invalidate(key, key.toString()); + keys.add(key); + } else { + // invalidate with incorrect value + cache.invalidate(key, Integer.toString(key * randomIntBetween(2, 10))); + } + } + } + for (int i = 0; i < numberOfEntries; i++) { + if (keys.contains(i)) { + assertNull(cache.get(i)); + } else { + assertNotNull(cache.get(i)); + } + } + } + + // randomly invalidate some cached entries, then check that we receive invalidate notifications for those and only + // those entries + public void testNotificationOnInvalidateWithValue() { + Set notifications = new HashSet<>(); + Cache cache = + CacheBuilder.builder() + .removalListener(notification -> { + assertEquals(RemovalNotification.RemovalReason.INVALIDATED, notification.getRemovalReason()); + notifications.add(notification.getKey()); + }) + .build(); + for (int i = 0; i < numberOfEntries; i++) { + cache.put(i, Integer.toString(i)); + } + Set invalidated = new HashSet<>(); + for (int i = 0; i < numberOfEntries; i++) { + if (rarely()) { + if (randomBoolean()) { + cache.invalidate(i, Integer.toString(i)); + invalidated.add(i); + } else { + // invalidate with incorrect value + cache.invalidate(i, Integer.toString(i * randomIntBetween(2, 10))); + } + } + } + assertEquals(notifications, invalidated); + } + // invalidate all cached entries, then check that the cache is empty public void testInvalidateAll() { Cache cache = CacheBuilder.builder().build(); diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java new file mode 100644 index 0000000000000..712656777f970 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/ListenableFutureTests.java @@ -0,0 +1,118 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.util.concurrent; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESTestCase; +import org.junit.After; + +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; + +public class ListenableFutureTests extends ESTestCase { + + private ExecutorService executorService; + + @After + public void stopExecutorService() throws InterruptedException { + if (executorService != null) { + terminate(executorService); + } + } + + public void testListenableFutureNotifiesListeners() { + ListenableFuture future = new ListenableFuture<>(); + AtomicInteger notifications = new AtomicInteger(0); + final int numberOfListeners = scaledRandomIntBetween(1, 12); + for (int i = 0; i < numberOfListeners; i++) { + future.addListener(ActionListener.wrap(notifications::incrementAndGet), EsExecutors.newDirectExecutorService()); + } + + future.onResponse(""); + assertEquals(numberOfListeners, notifications.get()); + assertTrue(future.isDone()); + } + + public void testListenableFutureNotifiesListenersOnException() { + ListenableFuture future = new ListenableFuture<>(); + AtomicInteger notifications = new AtomicInteger(0); + final int numberOfListeners = scaledRandomIntBetween(1, 12); + final Exception exception = new RuntimeException(); + for (int i = 0; i < numberOfListeners; i++) { + future.addListener(ActionListener.wrap(s -> fail("this should never be called"), e -> { + assertEquals(exception, e); + notifications.incrementAndGet(); + }), EsExecutors.newDirectExecutorService()); + } + + future.onFailure(exception); + assertEquals(numberOfListeners, notifications.get()); + assertTrue(future.isDone()); + } + + public void testConcurrentListenerRegistrationAndCompletion() throws BrokenBarrierException, InterruptedException { + final int numberOfThreads = scaledRandomIntBetween(2, 32); + final int completingThread = randomIntBetween(0, numberOfThreads - 1); + final ListenableFuture future = new ListenableFuture<>(); + executorService = EsExecutors.newFixed("testConcurrentListenerRegistrationAndCompletion", numberOfThreads, 1000, + EsExecutors.daemonThreadFactory("listener"), new ThreadContext(Settings.EMPTY)); + final CyclicBarrier barrier = new CyclicBarrier(1 + numberOfThreads); + final CountDownLatch listenersLatch = new CountDownLatch(numberOfThreads - 1); + final AtomicInteger numResponses = new AtomicInteger(0); + final AtomicInteger numExceptions = new AtomicInteger(0); + + for (int i = 0; i < numberOfThreads; i++) { + final int threadNum = i; + Thread thread = new Thread(() -> { + try { + barrier.await(); + if (threadNum == completingThread) { + future.onResponse(""); + } else { + future.addListener(ActionListener.wrap(s -> { + assertEquals("", s); + numResponses.incrementAndGet(); + listenersLatch.countDown(); + }, e -> { + logger.error("caught unexpected exception", e); + numExceptions.incrementAndGet(); + listenersLatch.countDown(); + }), executorService); + } + barrier.await(); + } catch (InterruptedException | BrokenBarrierException e) { + throw new AssertionError(e); + } + }); + thread.start(); + } + + barrier.await(); + barrier.await(); + listenersLatch.await(); + + assertEquals(numberOfThreads - 1, numResponses.get()); + assertEquals(0, numExceptions.get()); + } +} 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 133093df33a13..6d12b6472f18f 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 @@ -410,7 +410,7 @@ Collection createComponents(Client client, ThreadPool threadPool, Cluste final NativeRoleMappingStore nativeRoleMappingStore = new NativeRoleMappingStore(settings, client, securityIndex.get()); final AnonymousUser anonymousUser = new AnonymousUser(settings); final ReservedRealm reservedRealm = new ReservedRealm(env, settings, nativeUsersStore, - anonymousUser, securityIndex.get(), threadPool.getThreadContext()); + anonymousUser, securityIndex.get(), threadPool); Map realmFactories = new HashMap<>(InternalRealms.getFactories(threadPool, resourceWatcherService, getSslService(), nativeUsersStore, nativeRoleMappingStore, securityIndex.get())); for (SecurityExtension extension : securityExtensions) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java index 1e38e6fd10391..d8d0d26f99e0d 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/InternalRealms.java @@ -93,9 +93,9 @@ public static Map getFactories(ThreadPool threadPool, Res SecurityIndexManager securityIndex) { Map map = new HashMap<>(); - map.put(FileRealmSettings.TYPE, config -> new FileRealm(config, resourceWatcherService)); + map.put(FileRealmSettings.TYPE, config -> new FileRealm(config, resourceWatcherService, threadPool)); map.put(NativeRealmSettings.TYPE, config -> { - final NativeRealm nativeRealm = new NativeRealm(config, nativeUsersStore); + final NativeRealm nativeRealm = new NativeRealm(config, nativeUsersStore, threadPool); securityIndex.addIndexStateListener(nativeRealm::onSecurityIndexStateChange); return nativeRealm; }); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealm.java index c9ccdbb75c0bb..af2bfcf0d6c14 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealm.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.security.authc.esnative; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.core.security.authc.esnative.NativeRealmSettings; @@ -24,8 +25,8 @@ public class NativeRealm extends CachingUsernamePasswordRealm { private final NativeUsersStore userStore; - public NativeRealm(RealmConfig config, NativeUsersStore usersStore) { - super(NativeRealmSettings.TYPE, config); + public NativeRealm(RealmConfig config, NativeUsersStore usersStore, ThreadPool threadPool) { + super(NativeRealmSettings.TYPE, config, threadPool); this.userStore = usersStore; } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealm.java index 7dbcea908722c..3946a01784b16 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealm.java @@ -14,8 +14,8 @@ import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.security.SecurityField; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; @@ -66,8 +66,8 @@ public class ReservedRealm extends CachingUsernamePasswordRealm { private final SecurityIndexManager securityIndex; public ReservedRealm(Environment env, Settings settings, NativeUsersStore nativeUsersStore, AnonymousUser anonymousUser, - SecurityIndexManager securityIndex, ThreadContext threadContext) { - super(TYPE, new RealmConfig(TYPE, Settings.EMPTY, settings, env, threadContext)); + SecurityIndexManager securityIndex, ThreadPool threadPool) { + super(TYPE, new RealmConfig(TYPE, Settings.EMPTY, settings, env, threadPool.getThreadContext()), threadPool); this.nativeUsersStore = nativeUsersStore; this.realmEnabled = XPackSettings.RESERVED_REALM_ENABLED_SETTING.get(settings); this.anonymousUser = anonymousUser; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/file/FileRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/file/FileRealm.java index 9e85b4505210e..88656b9e01e30 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/file/FileRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/file/FileRealm.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.security.authc.file; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.RealmConfig; @@ -21,13 +22,13 @@ public class FileRealm extends CachingUsernamePasswordRealm { private final FileUserPasswdStore userPasswdStore; private final FileUserRolesStore userRolesStore; - public FileRealm(RealmConfig config, ResourceWatcherService watcherService) { - this(config, new FileUserPasswdStore(config, watcherService), new FileUserRolesStore(config, watcherService)); + public FileRealm(RealmConfig config, ResourceWatcherService watcherService, ThreadPool threadPool) { + this(config, new FileUserPasswdStore(config, watcherService), new FileUserRolesStore(config, watcherService), threadPool); } // pkg private for testing - FileRealm(RealmConfig config, FileUserPasswdStore userPasswdStore, FileUserRolesStore userRolesStore) { - super(FileRealmSettings.TYPE, config); + FileRealm(RealmConfig config, FileUserPasswdStore userPasswdStore, FileUserRolesStore userRolesStore, ThreadPool threadPool) { + super(FileRealmSettings.TYPE, config, threadPool); this.userPasswdStore = userPasswdStore; userPasswdStore.addListener(this::expireAll); this.userRolesStore = userRolesStore; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealm.java index ceb28ada76a97..a7c6efdda3114 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ldap/LdapRealm.java @@ -67,7 +67,7 @@ public LdapRealm(String type, RealmConfig config, SSLService sslService, // pkg private for testing LdapRealm(String type, RealmConfig config, SessionFactory sessionFactory, UserRoleMapper roleMapper, ThreadPool threadPool) { - super(type, config); + super(type, config, threadPool); this.sessionFactory = sessionFactory; this.roleMapper = roleMapper; this.threadPool = threadPool; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java index e5a90c0855fdc..8dae5275eda14 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java @@ -5,11 +5,15 @@ */ package org.elasticsearch.xpack.security.authc.support; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.cache.CacheBuilder; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.ListenableFuture; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.AuthenticationToken; import org.elasticsearch.xpack.core.security.authc.RealmConfig; @@ -21,18 +25,21 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; public abstract class CachingUsernamePasswordRealm extends UsernamePasswordRealm implements CachingRealm { - private final Cache cache; + private final Cache>> cache; + private final ThreadPool threadPool; final Hasher hasher; - protected CachingUsernamePasswordRealm(String type, RealmConfig config) { + protected CachingUsernamePasswordRealm(String type, RealmConfig config, ThreadPool threadPool) { super(type, config); hasher = Hasher.resolve(CachingUsernamePasswordRealmSettings.CACHE_HASH_ALGO_SETTING.get(config.settings()), Hasher.SSHA256); + this.threadPool = threadPool; TimeValue ttl = CachingUsernamePasswordRealmSettings.CACHE_TTL_SETTING.get(config.settings()); if (ttl.getNanos() > 0) { - cache = CacheBuilder.builder() + cache = CacheBuilder.>>builder() .setExpireAfterWrite(ttl) .setMaximumWeight(CachingUsernamePasswordRealmSettings.CACHE_MAX_USERS_SETTING.get(config.settings())) .build(); @@ -78,74 +85,95 @@ public final void authenticate(AuthenticationToken authToken, ActionListener listener) { - UserWithHash userWithHash = cache.get(token.principal()); - if (userWithHash == null) { - if (logger.isDebugEnabled()) { - logger.debug("user [{}] not found in cache for realm [{}], proceeding with normal authentication", - token.principal(), name()); - } - doAuthenticateAndCache(token, ActionListener.wrap((result) -> { - if (result.isAuthenticated()) { - final User user = result.getUser(); - logger.debug("realm [{}] authenticated user [{}], with roles [{}]", name(), token.principal(), user.roles()); - } - listener.onResponse(result); - }, listener::onFailure)); - } else if (userWithHash.hasHash()) { - if (userWithHash.verify(token.credentials())) { - if (userWithHash.user.enabled()) { - User user = userWithHash.user; - logger.debug("realm [{}] authenticated user [{}], with roles [{}]", name(), token.principal(), user.roles()); - listener.onResponse(AuthenticationResult.success(user)); - } else { - // We successfully authenticated, but the cached user is disabled. - // Reload the primary record to check whether the user is still disabled - cache.invalidate(token.principal()); - doAuthenticateAndCache(token, ActionListener.wrap((result) -> { - if (result.isAuthenticated()) { - final User user = result.getUser(); - logger.debug("realm [{}] authenticated user [{}] (enabled:{}), with roles [{}]", name(), token.principal(), - user.enabled(), user.roles()); - } - listener.onResponse(result); - }, listener::onFailure)); + try { + final SetOnce authenticatedUser = new SetOnce<>(); + final AtomicBoolean createdAndStartedFuture = new AtomicBoolean(false); + final ListenableFuture> future = cache.computeIfAbsent(token.principal(), k -> { + final ListenableFuture> created = new ListenableFuture<>(); + if (createdAndStartedFuture.compareAndSet(false, true) == false) { + throw new IllegalStateException("something else already started this. how?"); } - } else { - cache.invalidate(token.principal()); - doAuthenticateAndCache(token, ActionListener.wrap((result) -> { + return created; + }); + + if (createdAndStartedFuture.get()) { + doAuthenticate(token, ActionListener.wrap(result -> { if (result.isAuthenticated()) { final User user = result.getUser(); - logger.debug("cached user's password changed. realm [{}] authenticated user [{}], with roles [{}]", - name(), token.principal(), user.roles()); + authenticatedUser.set(user); + final UserWithHash userWithHash = new UserWithHash(user, token.credentials(), hasher); + future.onResponse(new Tuple<>(result, userWithHash)); + } else { + future.onResponse(new Tuple<>(result, null)); } - listener.onResponse(result); - }, listener::onFailure)); + }, future::onFailure)); } - } else { - cache.invalidate(token.principal()); - doAuthenticateAndCache(token, ActionListener.wrap((result) -> { - if (result.isAuthenticated()) { - final User user = result.getUser(); - logger.debug("cached user came from a lookup and could not be used for authentication. " + - "realm [{}] authenticated user [{}] with roles [{}]", name(), token.principal(), user.roles()); + + future.addListener(ActionListener.wrap(tuple -> { + if (tuple != null) { + final UserWithHash userWithHash = tuple.v2(); + final boolean performedAuthentication = createdAndStartedFuture.get() && userWithHash != null && + tuple.v2().user == authenticatedUser.get(); + handleResult(future, createdAndStartedFuture.get(), performedAuthentication, token, tuple, listener); + } else { + handleFailure(future, createdAndStartedFuture.get(), token, new IllegalStateException("unknown error authenticating"), + listener); } - listener.onResponse(result); - }, listener::onFailure)); + }, e -> handleFailure(future, createdAndStartedFuture.get(), token, e, listener)), + threadPool.executor(ThreadPool.Names.GENERIC)); + } catch (ExecutionException e) { + listener.onResponse(AuthenticationResult.unsuccessful("", e)); } } - private void doAuthenticateAndCache(UsernamePasswordToken token, ActionListener listener) { - ActionListener wrapped = ActionListener.wrap((result) -> { - Objects.requireNonNull(result, "AuthenticationResult cannot be null"); - if (result.getStatus() == AuthenticationResult.Status.SUCCESS) { - UserWithHash userWithHash = new UserWithHash(result.getUser(), token.credentials(), hasher); - // it doesn't matter if we already computed it elsewhere - cache.put(token.principal(), userWithHash); + private void handleResult(ListenableFuture> future, boolean createdAndStartedFuture, + boolean performedAuthentication, UsernamePasswordToken token, + Tuple result, ActionListener listener) { + final AuthenticationResult authResult = result.v1(); + if (authResult == null) { + // this was from a lookup; clear and redo + cache.invalidate(token.principal(), future); + authenticateWithCache(token, listener); + } else if (authResult.isAuthenticated()) { + if (performedAuthentication) { + listener.onResponse(authResult); + } else { + UserWithHash userWithHash = result.v2(); + if (userWithHash.verify(token.credentials())) { + if (userWithHash.user.enabled()) { + User user = userWithHash.user; + logger.debug("realm [{}] authenticated user [{}], with roles [{}]", + name(), token.principal(), user.roles()); + listener.onResponse(AuthenticationResult.success(user)); + } else { + // re-auth to see if user has been enabled + cache.invalidate(token.principal(), future); + authenticateWithCache(token, listener); + } + } else { + // could be a password change? + cache.invalidate(token.principal(), future); + authenticateWithCache(token, listener); + } } - listener.onResponse(result); - }, listener::onFailure); + } else { + cache.invalidate(token.principal(), future); + if (createdAndStartedFuture) { + listener.onResponse(authResult); + } else { + authenticateWithCache(token, listener); + } + } + } - doAuthenticate(token, wrapped); + private void handleFailure(ListenableFuture> future, boolean createdAndStarted, + UsernamePasswordToken token, Exception e, ActionListener listener) { + cache.invalidate(token.principal(), future); + if (createdAndStarted) { + listener.onFailure(e); + } else { + authenticateWithCache(token, listener); + } } @Override @@ -160,29 +188,34 @@ public Map usageStats() { @Override public final void lookupUser(String username, ActionListener listener) { if (cache != null) { - UserWithHash withHash = cache.get(username); - if (withHash == null) { - try { - doLookupUser(username, ActionListener.wrap((user) -> { - Runnable action = () -> listener.onResponse(null); + try { + ListenableFuture> future = cache.computeIfAbsent(username, key -> { + ListenableFuture> created = new ListenableFuture<>(); + doLookupUser(username, ActionListener.wrap(user -> { if (user != null) { UserWithHash userWithHash = new UserWithHash(user, null, null); - try { - // computeIfAbsent is used here to avoid overwriting a value from a concurrent authenticate call as it - // contains the password hash, which provides a performance boost and we shouldn't just erase that - cache.computeIfAbsent(username, (n) -> userWithHash); - action = () -> listener.onResponse(userWithHash.user); - } catch (ExecutionException e) { - action = () -> listener.onFailure(e); - } + created.onResponse(new Tuple<>(null, userWithHash)); + } else { + created.onResponse(new Tuple<>(null, null)); } - action.run(); - }, listener::onFailure)); - } catch (Exception e) { - listener.onFailure(e); - } - } else { - listener.onResponse(withHash.user); + }, created::onFailure)); + return created; + }); + + future.addListener(ActionListener.wrap(tuple -> { + if (tuple != null) { + if (tuple.v2() == null) { + cache.invalidate(username, future); + listener.onResponse(null); + } else { + listener.onResponse(tuple.v2().user); + } + } else { + listener.onResponse(null); + } + }, listener::onFailure), threadPool.executor(ThreadPool.Names.GENERIC)); + } catch (ExecutionException e) { + listener.onFailure(e); } } else { doLookupUser(username, listener); @@ -192,12 +225,12 @@ public final void lookupUser(String username, ActionListener listener) { protected abstract void doLookupUser(String username, ActionListener listener); private static class UserWithHash { - User user; - char[] hash; - Hasher hasher; + final User user; + final char[] hash; + final Hasher hasher; UserWithHash(User user, SecureString password, Hasher hasher) { - this.user = user; + this.user = Objects.requireNonNull(user); this.hash = password == null ? null : hasher.hash(password); this.hasher = hasher; } @@ -205,9 +238,5 @@ private static class UserWithHash { boolean verify(SecureString password) { return hash != null && hasher.verify(password, hash); } - - boolean hasHash() { - return hash != null; - } } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportGetUsersActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportGetUsersActionTests.java index 6750560b0b0d2..2ad467236820f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportGetUsersActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportGetUsersActionTests.java @@ -13,9 +13,9 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.security.action.user.GetUsersRequest; @@ -28,6 +28,7 @@ import org.elasticsearch.xpack.security.authc.esnative.ReservedRealm; import org.elasticsearch.xpack.security.authc.esnative.ReservedRealmTests; import org.elasticsearch.xpack.security.support.SecurityIndexManager; +import org.junit.After; import org.junit.Before; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -62,6 +63,7 @@ public class TransportGetUsersActionTests extends ESTestCase { private boolean anonymousEnabled; private Settings settings; + private ThreadPool threadPool; @Before public void maybeEnableAnonymous() { @@ -71,6 +73,14 @@ public void maybeEnableAnonymous() { } else { settings = Settings.EMPTY; } + threadPool = new TestThreadPool("TransportGetUsersActionTests"); + } + + @After + public void terminateThreadPool() throws InterruptedException { + if (threadPool != null) { + terminate(threadPool); + } } public void testAnonymousUser() { @@ -79,10 +89,10 @@ public void testAnonymousUser() { when(securityIndex.isAvailable()).thenReturn(true); AnonymousUser anonymousUser = new AnonymousUser(settings); ReservedRealm reservedRealm = - new ReservedRealm(mock(Environment.class), settings, usersStore, anonymousUser, securityIndex, new ThreadContext(Settings.EMPTY)); + new ReservedRealm(mock(Environment.class), settings, usersStore, anonymousUser, securityIndex, threadPool); TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); - TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ThreadPool.class), mock(ActionFilters.class), + TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, threadPool, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class), usersStore, transportService, reservedRealm); GetUsersRequest request = new GetUsersRequest(); @@ -117,7 +127,7 @@ public void testInternalUser() { NativeUsersStore usersStore = mock(NativeUsersStore.class); TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); - TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ThreadPool.class), mock(ActionFilters.class), + TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, threadPool, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class), usersStore, transportService, mock(ReservedRealm.class)); GetUsersRequest request = new GetUsersRequest(); @@ -151,7 +161,7 @@ public void testReservedUsersOnly() { ReservedRealmTests.mockGetAllReservedUserInfo(usersStore, Collections.emptyMap()); ReservedRealm reservedRealm = - new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(settings), securityIndex, new ThreadContext(Settings.EMPTY)); + new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(settings), securityIndex, threadPool); PlainActionFuture> userFuture = new PlainActionFuture<>(); reservedRealm.users(userFuture); final Collection allReservedUsers = userFuture.actionGet(); @@ -160,7 +170,7 @@ public void testReservedUsersOnly() { final List names = reservedUsers.stream().map(User::principal).collect(Collectors.toList()); TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); - TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ThreadPool.class), mock(ActionFilters.class), + TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, threadPool, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class), usersStore, transportService, reservedRealm); logger.error("names {}", names); @@ -197,10 +207,10 @@ public void testGetAllUsers() { when(securityIndex.isAvailable()).thenReturn(true); ReservedRealmTests.mockGetAllReservedUserInfo(usersStore, Collections.emptyMap()); ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(settings), - securityIndex, new ThreadContext(Settings.EMPTY)); + securityIndex, threadPool); TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); - TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ThreadPool.class), mock(ActionFilters.class), + TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, threadPool, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class), usersStore, transportService, reservedRealm); GetUsersRequest request = new GetUsersRequest(); @@ -247,7 +257,7 @@ public void testGetStoreOnlyUsers() { NativeUsersStore usersStore = mock(NativeUsersStore.class); TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); - TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ThreadPool.class), mock(ActionFilters.class), + TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, threadPool, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class), usersStore, transportService, mock(ReservedRealm.class)); GetUsersRequest request = new GetUsersRequest(); @@ -295,7 +305,7 @@ public void testException() { NativeUsersStore usersStore = mock(NativeUsersStore.class); TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); - TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, mock(ThreadPool.class), mock(ActionFilters.class), + TransportGetUsersAction action = new TransportGetUsersAction(Settings.EMPTY, threadPool, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class), usersStore, transportService, mock(ReservedRealm.class)); GetUsersRequest request = new GetUsersRequest(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportPutUserActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportPutUserActionTests.java index 65cf74971a55c..d059911a6807a 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportPutUserActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/user/TransportPutUserActionTests.java @@ -121,14 +121,16 @@ public void testReservedUser() { when(securityIndex.isAvailable()).thenReturn(true); ReservedRealmTests.mockGetAllReservedUserInfo(usersStore, Collections.emptyMap()); Settings settings = Settings.builder().put("path.home", createTempDir()).build(); + final ThreadPool threadPool = mock(ThreadPool.class); + when(threadPool.getThreadContext()).thenReturn(new ThreadContext(settings)); ReservedRealm reservedRealm = new ReservedRealm(TestEnvironment.newEnvironment(settings), settings, usersStore, - new AnonymousUser(settings), securityIndex, new ThreadContext(settings)); + new AnonymousUser(settings), securityIndex, threadPool); PlainActionFuture> userFuture = new PlainActionFuture<>(); reservedRealm.users(userFuture); final User reserved = randomFrom(userFuture.actionGet().toArray(new User[0])); TransportService transportService = new TransportService(Settings.EMPTY, null, null, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); - TransportPutUserAction action = new TransportPutUserAction(Settings.EMPTY, mock(ThreadPool.class), mock(ActionFilters.class), + TransportPutUserAction action = new TransportPutUserAction(Settings.EMPTY, threadPool, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class), usersStore, transportService); PutUserRequest request = new PutUserRequest(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmTests.java index 7e2d5242101c1..633360318c217 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/NativeRealmTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.security.authc.RealmConfig; import org.elasticsearch.xpack.security.support.SecurityIndexManager; @@ -18,6 +19,7 @@ import static org.elasticsearch.xpack.security.test.SecurityTestUtils.getClusterIndexHealth; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class NativeRealmTests extends ESTestCase { @@ -26,12 +28,15 @@ private SecurityIndexManager.State dummyState(ClusterHealthStatus indexStatus) { } public void testCacheClearOnIndexHealthChange() { + final ThreadPool threadPool = mock(ThreadPool.class); + final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + when(threadPool.getThreadContext()).thenReturn(threadContext); final AtomicInteger numInvalidation = new AtomicInteger(0); int expectedInvalidation = 0; Settings settings = Settings.builder().put("path.home", createTempDir()).build(); RealmConfig config = new RealmConfig("native", Settings.EMPTY, settings, TestEnvironment.newEnvironment(settings), new ThreadContext(settings)); - final NativeRealm nativeRealm = new NativeRealm(config, mock(NativeUsersStore.class)) { + final NativeRealm nativeRealm = new NativeRealm(config, mock(NativeUsersStore.class), threadPool) { @Override void clearCache() { numInvalidation.incrementAndGet(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealmTests.java index 9fc52e8af63bc..b483595f8ec20 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ReservedRealmTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.esnative.ClientReservedRealm; @@ -63,6 +64,7 @@ public class ReservedRealmTests extends ESTestCase { private static final SecureString EMPTY_PASSWORD = new SecureString("".toCharArray()); private NativeUsersStore usersStore; private SecurityIndexManager securityIndex; + private ThreadPool threadPool; @Before public void setupMocks() throws Exception { @@ -71,6 +73,8 @@ public void setupMocks() throws Exception { when(securityIndex.isAvailable()).thenReturn(true); when(securityIndex.checkMappingVersion(any())).thenReturn(true); mockGetAllReservedUserInfo(usersStore, Collections.emptyMap()); + threadPool = mock(ThreadPool.class); + when(threadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); } public void testReservedUserEmptyPasswordAuthenticationFails() throws Throwable { @@ -78,7 +82,7 @@ public void testReservedUserEmptyPasswordAuthenticationFails() throws Throwable UsernamesField.BEATS_NAME); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), Settings.EMPTY, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); PlainActionFuture listener = new PlainActionFuture<>(); @@ -94,7 +98,7 @@ public void testAuthenticationDisabled() throws Throwable { } final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, - new AnonymousUser(settings), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(settings), securityIndex, threadPool); final User expected = randomReservedUser(true); final String principal = expected.principal(); @@ -116,7 +120,7 @@ public void testAuthenticationDisabledUserWithStoredPassword() throws Throwable private void verifySuccessfulAuthentication(boolean enabled) throws Exception { final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), Settings.EMPTY, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); final User expectedUser = randomReservedUser(enabled); final String principal = expectedUser.principal(); final SecureString newPassword = new SecureString("foobar".toCharArray()); @@ -157,7 +161,7 @@ private void verifySuccessfulAuthentication(boolean enabled) throws Exception { public void testLookup() throws Exception { final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), Settings.EMPTY, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); final User expectedUser = randomReservedUser(true); final String principal = expectedUser.principal(); @@ -182,7 +186,7 @@ public void testLookupDisabled() throws Exception { Settings settings = Settings.builder().put(XPackSettings.RESERVED_REALM_ENABLED_SETTING.getKey(), false).build(); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, new AnonymousUser(settings), - securityIndex, new ThreadContext(Settings.EMPTY)); + securityIndex, threadPool); final User expectedUser = randomReservedUser(true); final String principal = expectedUser.principal(); @@ -196,7 +200,7 @@ public void testLookupDisabled() throws Exception { public void testLookupThrows() throws Exception { final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), Settings.EMPTY, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); final User expectedUser = randomReservedUser(true); final String principal = expectedUser.principal(); when(securityIndex.indexExists()).thenReturn(true); @@ -243,7 +247,7 @@ public void testIsReservedDisabled() { public void testGetUsers() { final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), Settings.EMPTY, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); PlainActionFuture> userFuture = new PlainActionFuture<>(); reservedRealm.users(userFuture); assertThat(userFuture.actionGet(), @@ -258,7 +262,7 @@ public void testGetUsersDisabled() { .build(); final AnonymousUser anonymousUser = new AnonymousUser(settings); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, anonymousUser, - securityIndex, new ThreadContext(Settings.EMPTY)); + securityIndex, threadPool); PlainActionFuture> userFuture = new PlainActionFuture<>(); reservedRealm.users(userFuture); if (anonymousEnabled) { @@ -275,7 +279,7 @@ public void testFailedAuthentication() throws Exception { ReservedUserInfo userInfo = new ReservedUserInfo(hash, true, false); mockGetAllReservedUserInfo(usersStore, Collections.singletonMap("elastic", userInfo)); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), Settings.EMPTY, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); if (randomBoolean()) { PlainActionFuture future = new PlainActionFuture<>(); @@ -305,7 +309,7 @@ public void testBootstrapElasticPasswordWorksOnceSecurityIndexExists() throws Ex when(securityIndex.indexExists()).thenReturn(true); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); PlainActionFuture listener = new PlainActionFuture<>(); doAnswer((i) -> { @@ -327,7 +331,7 @@ public void testBootstrapElasticPasswordFailsOnceElasticUserExists() throws Exce when(securityIndex.indexExists()).thenReturn(true); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); PlainActionFuture listener = new PlainActionFuture<>(); SecureString password = new SecureString("password".toCharArray()); doAnswer((i) -> { @@ -354,7 +358,7 @@ public void testBootstrapElasticPasswordWorksBeforeSecurityIndexExists() throws when(securityIndex.indexExists()).thenReturn(false); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); PlainActionFuture listener = new PlainActionFuture<>(); reservedRealm.doAuthenticate(new UsernamePasswordToken(new ElasticUser(true).principal(), @@ -372,7 +376,7 @@ public void testNonElasticUsersCannotUseBootstrapPasswordWhenSecurityIndexExists when(securityIndex.indexExists()).thenReturn(true); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); PlainActionFuture listener = new PlainActionFuture<>(); final String principal = randomFrom(KibanaUser.NAME, LogstashSystemUser.NAME, BeatsSystemUser.NAME); @@ -394,7 +398,7 @@ public void testNonElasticUsersCannotUseBootstrapPasswordWhenSecurityIndexDoesNo when(securityIndex.indexExists()).thenReturn(false); final ReservedRealm reservedRealm = new ReservedRealm(mock(Environment.class), settings, usersStore, - new AnonymousUser(Settings.EMPTY), securityIndex, new ThreadContext(Settings.EMPTY)); + new AnonymousUser(Settings.EMPTY), securityIndex, threadPool); PlainActionFuture listener = new PlainActionFuture<>(); final String principal = randomFrom(KibanaUser.NAME, LogstashSystemUser.NAME, BeatsSystemUser.NAME); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/file/FileRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/file/FileRealmTests.java index b1500cc75208c..b0f53229377f0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/file/FileRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/file/FileRealmTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.RealmConfig; @@ -50,20 +51,26 @@ public class FileRealmTests extends ESTestCase { private FileUserPasswdStore userPasswdStore; private FileUserRolesStore userRolesStore; private Settings globalSettings; + private ThreadPool threadPool; + private ThreadContext threadContext; @Before public void init() throws Exception { userPasswdStore = mock(FileUserPasswdStore.class); userRolesStore = mock(FileUserRolesStore.class); globalSettings = Settings.builder().put("path.home", createTempDir()).build(); + threadPool = mock(ThreadPool.class); + threadContext = new ThreadContext(globalSettings); + when(threadPool.getThreadContext()).thenReturn(threadContext); } public void testAuthenticate() throws Exception { when(userPasswdStore.verifyPassword(eq("user1"), eq(new SecureString("test123")), any(Supplier.class))) .thenAnswer(VERIFY_PASSWORD_ANSWER); when(userRolesStore.roles("user1")).thenReturn(new String[] { "role1", "role2" }); - RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); - FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore); + RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), + threadContext); + FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("user1", new SecureString("test123")), future); final AuthenticationResult result = future.actionGet(); @@ -80,11 +87,12 @@ public void testAuthenticateCaching() throws Exception { Settings settings = Settings.builder() .put("cache.hash_algo", Hasher.values()[randomIntBetween(0, Hasher.values().length - 1)].name().toLowerCase(Locale.ROOT)) .build(); - RealmConfig config = new RealmConfig("file-test", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); + RealmConfig config = new RealmConfig("file-test", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), + threadContext); when(userPasswdStore.verifyPassword(eq("user1"), eq(new SecureString("test123")), any(Supplier.class))) .thenAnswer(VERIFY_PASSWORD_ANSWER); when(userRolesStore.roles("user1")).thenReturn(new String[]{"role1", "role2"}); - FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore); + FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("user1", new SecureString("test123")), future); User user1 = future.actionGet().getUser(); @@ -95,13 +103,14 @@ public void testAuthenticateCaching() throws Exception { } public void testAuthenticateCachingRefresh() throws Exception { - RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); + RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), + threadContext); userPasswdStore = spy(new UserPasswdStore(config)); userRolesStore = spy(new UserRolesStore(config)); when(userPasswdStore.verifyPassword(eq("user1"), eq(new SecureString("test123")), any(Supplier.class))) .thenAnswer(VERIFY_PASSWORD_ANSWER); doReturn(new String[] { "role1", "role2" }).when(userRolesStore).roles("user1"); - FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore); + FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("user1", new SecureString("test123")), future); User user1 = future.actionGet().getUser(); @@ -134,11 +143,12 @@ public void testAuthenticateCachingRefresh() throws Exception { } public void testToken() throws Exception { - RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); + RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), + threadContext); when(userPasswdStore.verifyPassword(eq("user1"), eq(new SecureString("test123")), any(Supplier.class))) .thenAnswer(VERIFY_PASSWORD_ANSWER); when(userRolesStore.roles("user1")).thenReturn(new String[]{"role1", "role2"}); - FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore); + FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore, threadPool); ThreadContext threadContext = new ThreadContext(Settings.EMPTY); UsernamePasswordToken.putTokenHeader(threadContext, new UsernamePasswordToken("user1", new SecureString("test123"))); @@ -153,8 +163,9 @@ public void testToken() throws Exception { public void testLookup() throws Exception { when(userPasswdStore.userExists("user1")).thenReturn(true); when(userRolesStore.roles("user1")).thenReturn(new String[] { "role1", "role2" }); - RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); - FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore); + RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), + threadContext); + FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.lookupUser("user1", future); @@ -170,8 +181,9 @@ public void testLookup() throws Exception { public void testLookupCaching() throws Exception { when(userPasswdStore.userExists("user1")).thenReturn(true); when(userRolesStore.roles("user1")).thenReturn(new String[] { "role1", "role2" }); - RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); - FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore); + RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), + threadContext); + FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.lookupUser("user1", future); @@ -185,12 +197,13 @@ public void testLookupCaching() throws Exception { } public void testLookupCachingWithRefresh() throws Exception { - RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); + RealmConfig config = new RealmConfig("file-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), + threadContext); userPasswdStore = spy(new UserPasswdStore(config)); userRolesStore = spy(new UserRolesStore(config)); doReturn(true).when(userPasswdStore).userExists("user1"); doReturn(new String[] { "role1", "role2" }).when(userRolesStore).roles("user1"); - FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore); + FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.lookupUser("user1", future); User user1 = future.actionGet(); @@ -231,8 +244,9 @@ public void testUsageStats() throws Exception { int order = randomIntBetween(0, 10); settings.put("order", order); - RealmConfig config = new RealmConfig("file-realm", settings.build(), globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(globalSettings)); - FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore); + RealmConfig config = new RealmConfig("file-realm", settings.build(), globalSettings, TestEnvironment.newEnvironment(globalSettings), + threadContext); + FileRealm realm = new FileRealm(config, userPasswdStore, userRolesStore, threadPool); Map usage = realm.usageStats(); assertThat(usage, is(notNullValue())); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java index 87f62cd97a198..38a6344f98e54 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java @@ -14,6 +14,8 @@ import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.SecuritySettingsSourceField; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.security.authc.AuthenticationResult; import org.elasticsearch.xpack.core.security.authc.Realm; import org.elasticsearch.xpack.core.security.authc.RealmConfig; @@ -22,6 +24,7 @@ import org.elasticsearch.xpack.core.security.authc.support.Hasher; import org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken; import org.elasticsearch.xpack.core.security.user.User; +import org.junit.After; import org.junit.Before; import java.util.ArrayList; @@ -42,10 +45,19 @@ public class CachingUsernamePasswordRealmTests extends ESTestCase { private Settings globalSettings; + private ThreadPool threadPool; @Before public void setup() { globalSettings = Settings.builder().put("path.home", createTempDir()).build(); + threadPool = new TestThreadPool("caching username password realm tests"); + } + + @After + public void stop() throws InterruptedException { + if (threadPool != null) { + terminate(threadPool); + } } public void testSettings() throws Exception { @@ -61,7 +73,7 @@ public void testSettings() throws Exception { RealmConfig config = new RealmConfig("test_realm", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(Settings.EMPTY)); - CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm("test", config) { + CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm("test", config, threadPool) { @Override protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { listener.onResponse(AuthenticationResult.success(new User("username", new String[]{"r1", "r2", "r3"}))); @@ -77,7 +89,7 @@ protected void doLookupUser(String username, ActionListener listener) { } public void testAuthCache() { - AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings); + AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings, threadPool); SecureString pass = new SecureString("pass"); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("a", pass), future); @@ -106,7 +118,7 @@ public void testAuthCache() { } public void testLookupCache() { - AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings); + AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.lookupUser("a", future); future.actionGet(); @@ -133,7 +145,7 @@ public void testLookupCache() { } public void testLookupAndAuthCache() { - AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings); + AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings, threadPool); // lookup first PlainActionFuture lookupFuture = new PlainActionFuture<>(); realm.lookupUser("a", lookupFuture); @@ -172,7 +184,7 @@ public void testLookupAndAuthCache() { } public void testCacheChangePassword() { - AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings); + AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings, threadPool); String user = "testUser"; SecureString pass1 = new SecureString("pass"); @@ -198,7 +210,7 @@ public void testCacheChangePassword() { } public void testCacheDisabledUser() { - AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings); + AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(globalSettings, threadPool); realm.setUsersEnabled(false); String user = "testUser"; @@ -233,7 +245,7 @@ public void testCacheWithVeryLowTtlExpiresBetweenAuthenticateCalls() throws Inte .build(); RealmConfig config = new RealmConfig("test_cache_ttl", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(Settings.EMPTY)); - AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(config); + AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(config, threadPool); final UsernamePasswordToken authToken = new UsernamePasswordToken("the-user", new SecureString("the-password")); @@ -262,7 +274,7 @@ public void testReadsDoNotPreventCacheExpiry() throws InterruptedException { .build(); RealmConfig config = new RealmConfig("test_cache_ttl", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(Settings.EMPTY)); - AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(config); + AlwaysAuthenticateCachingRealm realm = new AlwaysAuthenticateCachingRealm(config, threadPool); final UsernamePasswordToken authToken = new UsernamePasswordToken("the-user", new SecureString("the-password")); PlainActionFuture future = new PlainActionFuture<>(); @@ -304,13 +316,13 @@ private void sleepUntil(long until) throws InterruptedException { } public void testAuthenticateContract() throws Exception { - Realm realm = new FailingAuthenticationRealm(Settings.EMPTY, globalSettings); + Realm realm = new FailingAuthenticationRealm(Settings.EMPTY, globalSettings, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("user", new SecureString("pass")), future); User user = future.actionGet().getUser(); assertThat(user, nullValue()); - realm = new ThrowingAuthenticationRealm(Settings.EMPTY, globalSettings); + realm = new ThrowingAuthenticationRealm(Settings.EMPTY, globalSettings, threadPool); future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("user", new SecureString("pass")), future); RuntimeException e = expectThrows(RuntimeException.class, future::actionGet); @@ -318,19 +330,85 @@ public void testAuthenticateContract() throws Exception { } public void testLookupContract() throws Exception { - Realm realm = new FailingAuthenticationRealm(Settings.EMPTY, globalSettings); + Realm realm = new FailingAuthenticationRealm(Settings.EMPTY, globalSettings, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.lookupUser("user", future); User user = future.actionGet(); assertThat(user, nullValue()); - realm = new ThrowingAuthenticationRealm(Settings.EMPTY, globalSettings); + realm = new ThrowingAuthenticationRealm(Settings.EMPTY, globalSettings, threadPool); future = new PlainActionFuture<>(); realm.lookupUser("user", future); RuntimeException e = expectThrows(RuntimeException.class, future::actionGet); assertThat(e.getMessage(), containsString("lookup exception")); } + public void testSingleAuthPerUserLimit() throws Exception { + final String username = "username"; + final SecureString password = SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING; + final AtomicInteger authCounter = new AtomicInteger(0); + + final String passwordHash = new String(Hasher.BCRYPT.hash(password)); + RealmConfig config = new RealmConfig("test_realm", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), + new ThreadContext(Settings.EMPTY)); + final CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm("test", config, threadPool) { + @Override + protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { + authCounter.incrementAndGet(); + // do something slow + if (BCrypt.checkpw(token.credentials(), passwordHash)) { + listener.onResponse(AuthenticationResult.success(new User(username, new String[]{"r1", "r2", "r3"}))); + } else { + listener.onFailure(new IllegalStateException("password auth should never fail")); + } + } + + @Override + protected void doLookupUser(String username, ActionListener listener) { + listener.onFailure(new UnsupportedOperationException("this method should not be called")); + } + }; + + final int numberOfProcessors = Runtime.getRuntime().availableProcessors(); + final int numberOfThreads = scaledRandomIntBetween((numberOfProcessors + 1) / 2, numberOfProcessors * 3); + final int numberOfIterations = scaledRandomIntBetween(20, 100); + final CountDownLatch latch = new CountDownLatch(1 + numberOfThreads); + List threads = new ArrayList<>(numberOfThreads); + for (int i = 0; i < numberOfThreads; i++) { + threads.add(new Thread(() -> { + try { + latch.countDown(); + latch.await(); + for (int i1 = 0; i1 < numberOfIterations; i1++) { + UsernamePasswordToken token = new UsernamePasswordToken(username, password); + + realm.authenticate(token, ActionListener.wrap((result) -> { + if (result.isAuthenticated() == false) { + throw new IllegalStateException("proper password led to an unauthenticated result: " + result); + } + }, (e) -> { + logger.error("caught exception", e); + fail("unexpected exception - " + e); + })); + } + + } catch (InterruptedException e) { + logger.error("thread was interrupted", e); + Thread.currentThread().interrupt(); + } + })); + } + + for (Thread thread : threads) { + thread.start(); + } + latch.countDown(); + for (Thread thread : threads) { + thread.join(); + } + assertEquals(1, authCounter.get()); + } + public void testCacheConcurrency() throws Exception { final String username = "username"; final SecureString password = SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING; @@ -339,7 +417,7 @@ public void testCacheConcurrency() throws Exception { final String passwordHash = new String(Hasher.BCRYPT.hash(password)); RealmConfig config = new RealmConfig("test_realm", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(Settings.EMPTY)); - final CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm("test", config) { + final CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm("test", config, threadPool) { @Override protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { // do something slow @@ -356,37 +434,37 @@ protected void doLookupUser(String username, ActionListener listener) { } }; - final CountDownLatch latch = new CountDownLatch(1); final int numberOfProcessors = Runtime.getRuntime().availableProcessors(); final int numberOfThreads = scaledRandomIntBetween((numberOfProcessors + 1) / 2, numberOfProcessors * 3); final int numberOfIterations = scaledRandomIntBetween(20, 100); - List threads = new ArrayList<>(); + final CountDownLatch latch = new CountDownLatch(1 + numberOfThreads); + List threads = new ArrayList<>(numberOfThreads); for (int i = 0; i < numberOfThreads; i++) { final boolean invalidPassword = randomBoolean(); - threads.add(new Thread() { - @Override - public void run() { - try { - latch.await(); - for (int i = 0; i < numberOfIterations; i++) { - UsernamePasswordToken token = new UsernamePasswordToken(username, invalidPassword ? randomPassword : password); - - realm.authenticate(token, ActionListener.wrap((result) -> { - if (invalidPassword && result.isAuthenticated()) { - throw new RuntimeException("invalid password led to an authenticated user: " + result); - } else if (invalidPassword == false && result.isAuthenticated() == false) { - throw new RuntimeException("proper password led to an unauthenticated result: " + result); - } - }, (e) -> { - logger.error("caught exception", e); - fail("unexpected exception - " + e); - })); - } - - } catch (InterruptedException e) { + threads.add(new Thread(() -> { + try { + latch.countDown(); + latch.await(); + for (int i1 = 0; i1 < numberOfIterations; i1++) { + UsernamePasswordToken token = new UsernamePasswordToken(username, invalidPassword ? randomPassword : password); + + realm.authenticate(token, ActionListener.wrap((result) -> { + if (invalidPassword && result.isAuthenticated()) { + throw new RuntimeException("invalid password led to an authenticated user: " + result); + } else if (invalidPassword == false && result.isAuthenticated() == false) { + throw new RuntimeException("proper password led to an unauthenticated result: " + result); + } + }, (e) -> { + logger.error("caught exception", e); + fail("unexpected exception - " + e); + })); } + + } catch (InterruptedException e) { + logger.error("thread was interrupted", e); + Thread.currentThread().interrupt(); } - }); + })); } for (Thread thread : threads) { @@ -400,10 +478,11 @@ public void run() { public void testUserLookupConcurrency() throws Exception { final String username = "username"; + final AtomicInteger lookupCounter = new AtomicInteger(0); RealmConfig config = new RealmConfig("test_realm", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), new ThreadContext(Settings.EMPTY)); - final CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm("test", config) { + final CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm("test", config, threadPool) { @Override protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { listener.onFailure(new UnsupportedOperationException("authenticate should not be called!")); @@ -411,36 +490,37 @@ protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { + lookupCounter.incrementAndGet(); listener.onResponse(new User(username, new String[]{"r1", "r2", "r3"})); } }; - final CountDownLatch latch = new CountDownLatch(1); final int numberOfProcessors = Runtime.getRuntime().availableProcessors(); final int numberOfThreads = scaledRandomIntBetween(numberOfProcessors, numberOfProcessors * 3); final int numberOfIterations = scaledRandomIntBetween(10000, 100000); - List threads = new ArrayList<>(); + final CountDownLatch latch = new CountDownLatch(1 + numberOfThreads); + List threads = new ArrayList<>(numberOfThreads); for (int i = 0; i < numberOfThreads; i++) { - threads.add(new Thread() { - @Override - public void run() { - try { - latch.await(); - for (int i = 0; i < numberOfIterations; i++) { - realm.lookupUser(username, ActionListener.wrap((user) -> { - if (user == null) { - throw new RuntimeException("failed to lookup user"); - } - }, (e) -> { - logger.error("caught exception", e); - fail("unexpected exception"); - })); - } - - } catch (InterruptedException e) { + threads.add(new Thread(() -> { + try { + latch.countDown(); + latch.await(); + for (int i1 = 0; i1 < numberOfIterations; i1++) { + realm.lookupUser(username, ActionListener.wrap((user) -> { + if (user == null) { + throw new RuntimeException("failed to lookup user"); + } + }, (e) -> { + logger.error("caught exception", e); + fail("unexpected exception"); + })); } + + } catch (InterruptedException e) { + logger.error("thread was interrupted", e); + Thread.currentThread().interrupt(); } - }); + })); } for (Thread thread : threads) { @@ -450,13 +530,14 @@ public void run() { for (Thread thread : threads) { thread.join(); } + assertEquals(1, lookupCounter.get()); } static class FailingAuthenticationRealm extends CachingUsernamePasswordRealm { - FailingAuthenticationRealm(Settings settings, Settings global) { + FailingAuthenticationRealm(Settings settings, Settings global, ThreadPool threadPool) { super("failing", new RealmConfig("failing-test", settings, global, TestEnvironment.newEnvironment(global), - new ThreadContext(Settings.EMPTY))); + threadPool.getThreadContext()), threadPool); } @Override @@ -472,9 +553,9 @@ protected void doLookupUser(String username, ActionListener listener) { static class ThrowingAuthenticationRealm extends CachingUsernamePasswordRealm { - ThrowingAuthenticationRealm(Settings settings, Settings globalSettings) { + ThrowingAuthenticationRealm(Settings settings, Settings globalSettings, ThreadPool threadPool) { super("throwing", new RealmConfig("throwing-test", settings, globalSettings, TestEnvironment.newEnvironment(globalSettings), - new ThreadContext(Settings.EMPTY))); + threadPool.getThreadContext()), threadPool); } @Override @@ -495,13 +576,13 @@ static class AlwaysAuthenticateCachingRealm extends CachingUsernamePasswordRealm private boolean usersEnabled = true; - AlwaysAuthenticateCachingRealm(Settings globalSettings) { + AlwaysAuthenticateCachingRealm(Settings globalSettings, ThreadPool threadPool) { this(new RealmConfig("always-test", Settings.EMPTY, globalSettings, TestEnvironment.newEnvironment(globalSettings), - new ThreadContext(Settings.EMPTY))); + threadPool.getThreadContext()), threadPool); } - AlwaysAuthenticateCachingRealm(RealmConfig config) { - super("always", config); + AlwaysAuthenticateCachingRealm(RealmConfig config, ThreadPool threadPool) { + super("always", config, threadPool); } void setUsersEnabled(boolean usersEnabled) { @@ -527,9 +608,9 @@ static class LookupNotSupportedRealm extends CachingUsernamePasswordRealm { public final AtomicInteger authInvocationCounter = new AtomicInteger(0); public final AtomicInteger lookupInvocationCounter = new AtomicInteger(0); - LookupNotSupportedRealm(Settings globalSettings) { + LookupNotSupportedRealm(Settings globalSettings, ThreadPool threadPool) { super("lookup", new RealmConfig("lookup-notsupported-test", Settings.EMPTY, globalSettings, - TestEnvironment.newEnvironment(globalSettings), new ThreadContext(Settings.EMPTY))); + TestEnvironment.newEnvironment(globalSettings), threadPool.getThreadContext()), threadPool); } @Override diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java index 2a1c2dabe30b7..052ba38551021 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java @@ -198,7 +198,7 @@ private NativeRoleMappingStore buildRoleMappingStoreForInvalidationTesting(Atomi final Environment env = TestEnvironment.newEnvironment(settings); final RealmConfig realmConfig = new RealmConfig(getTestName(), Settings.EMPTY, settings, env, threadContext); - final CachingUsernamePasswordRealm mockRealm = new CachingUsernamePasswordRealm("test", realmConfig) { + final CachingUsernamePasswordRealm mockRealm = new CachingUsernamePasswordRealm("test", realmConfig, threadPool) { @Override protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { listener.onResponse(AuthenticationResult.notHandled()); From cf0e0606af28795c1d55341c57e1a979c71f6cf4 Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Thu, 24 May 2018 14:46:15 -0400 Subject: [PATCH 13/20] Use geohash cell instead of just a corner in geo_bounding_box (#30698) Treats geohashes as grid cells instead of just points when the geohashes are used to specify the edges in the geo_bounding_box query. For example, if a geohash is used to specify the top_left corner, the top left corner of the geohash cell will be used as the corner of the bounding box. Closes #25154 --- .../migration/migrate_7_0/search.asciidoc | 3 + .../query-dsl/geo-bounding-box-query.asciidoc | 32 ++++++++++ .../elasticsearch/common/geo/GeoPoint.java | 29 +++++---- .../elasticsearch/common/geo/GeoUtils.java | 49 ++++++++++++++- .../query/GeoBoundingBoxQueryBuilder.java | 11 ++-- .../GeoBoundingBoxQueryBuilderTests.java | 59 +++++++++++++++++++ .../index/search/geo/GeoUtilsTests.java | 14 +++++ 7 files changed, 178 insertions(+), 19 deletions(-) diff --git a/docs/reference/migration/migrate_7_0/search.asciidoc b/docs/reference/migration/migrate_7_0/search.asciidoc index 529bd1fa5995b..1fe0bc62418bb 100644 --- a/docs/reference/migration/migrate_7_0/search.asciidoc +++ b/docs/reference/migration/migrate_7_0/search.asciidoc @@ -12,6 +12,9 @@ * Purely negative queries (only MUST_NOT clauses) now return a score of `0` rather than `1`. +* The boundary specified using geohashes in the `geo_bounding_box` query + now include entire geohash cell, instead of just geohash center. + ==== Adaptive replica selection enabled by default Adaptive replica selection has been enabled by default. If you wish to return to diff --git a/docs/reference/query-dsl/geo-bounding-box-query.asciidoc b/docs/reference/query-dsl/geo-bounding-box-query.asciidoc index 21a689703e01e..fdf5ca5de16e5 100644 --- a/docs/reference/query-dsl/geo-bounding-box-query.asciidoc +++ b/docs/reference/query-dsl/geo-bounding-box-query.asciidoc @@ -231,6 +231,38 @@ GET /_search -------------------------------------------------- // CONSOLE + +When geohashes are used to specify the bounding the edges of the +bounding box, the geohashes are treated as rectangles. The bounding +box is defined in such a way that its top left corresponds to the top +left corner of the geohash specified in the `top_left` parameter and +its bottom right is defined as the bottom right of the geohash +specified in the `bottom_right` parameter. + +In order to specify a bounding box that would match entire area of a +geohash the geohash can be specified in both `top_left` and +`bottom_right` parameters: + +[source,js] +-------------------------------------------------- +GET /_search +{ + "query": { + "geo_bounding_box" : { + "pin.location" : { + "top_left" : "dr", + "bottom_right" : "dr" + } + } + } +} +-------------------------------------------------- +// CONSOLE + +In this example, the geohash `dr` will produce the bounding box +query with the top left corner at `45.0,-78.75` and the bottom right +corner at `39.375,-67.5`. + [float] ==== Vertices diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java b/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java index 8a0c3efa5afd9..bb22cb9e01f69 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoPoint.java @@ -22,6 +22,7 @@ import org.apache.lucene.document.LatLonDocValuesField; import org.apache.lucene.document.LatLonPoint; import org.apache.lucene.geo.GeoEncodingUtils; +import org.apache.lucene.geo.Rectangle; import org.apache.lucene.index.IndexableField; import org.apache.lucene.util.BitUtil; import org.apache.lucene.util.BytesRef; @@ -85,21 +86,27 @@ public GeoPoint resetFromString(String value) { public GeoPoint resetFromString(String value, final boolean ignoreZValue) { if (value.contains(",")) { - String[] vals = value.split(","); - if (vals.length > 3) { - throw new ElasticsearchParseException("failed to parse [{}], expected 2 or 3 coordinates " - + "but found: [{}]", vals.length); - } - double lat = Double.parseDouble(vals[0].trim()); - double lon = Double.parseDouble(vals[1].trim()); - if (vals.length > 2) { - GeoPoint.assertZValue(ignoreZValue, Double.parseDouble(vals[2].trim())); - } - return reset(lat, lon); + return resetFromCoordinates(value, ignoreZValue); } return resetFromGeoHash(value); } + + public GeoPoint resetFromCoordinates(String value, final boolean ignoreZValue) { + String[] vals = value.split(","); + if (vals.length > 3) { + throw new ElasticsearchParseException("failed to parse [{}], expected 2 or 3 coordinates " + + "but found: [{}]", vals.length); + } + double lat = Double.parseDouble(vals[0].trim()); + double lon = Double.parseDouble(vals[1].trim()); + if (vals.length > 2) { + GeoPoint.assertZValue(ignoreZValue, Double.parseDouble(vals[2].trim())); + } + return reset(lat, lon); + } + + public GeoPoint resetFromIndexHash(long hash) { lon = GeoHashUtils.decodeLongitude(hash); lat = GeoHashUtils.decodeLatitude(hash); diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java index 57e87e06389c4..2f3443639cdb7 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java @@ -387,6 +387,25 @@ public static GeoPoint parseGeoPoint(Object value, final boolean ignoreZValue) t } } + /** + * Represents the point of the geohash cell that should be used as the value of geohash + */ + public enum EffectivePoint { + TOP_LEFT, + TOP_RIGHT, + BOTTOM_LEFT, + BOTTOM_RIGHT + } + + /** + * Parse a geopoint represented as an object, string or an array. If the geopoint is represented as a geohash, + * the left bottom corner of the geohash cell is used as the geopoint coordinates.GeoBoundingBoxQueryBuilder.java + */ + public static GeoPoint parseGeoPoint(XContentParser parser, GeoPoint point, final boolean ignoreZValue) + throws IOException, ElasticsearchParseException { + return parseGeoPoint(parser, point, ignoreZValue, EffectivePoint.BOTTOM_LEFT); + } + /** * Parse a {@link GeoPoint} with a {@link XContentParser}. A geopoint has one of the following forms: * @@ -401,7 +420,7 @@ public static GeoPoint parseGeoPoint(Object value, final boolean ignoreZValue) t * @param point A {@link GeoPoint} that will be reset by the values parsed * @return new {@link GeoPoint} parsed from the parse */ - public static GeoPoint parseGeoPoint(XContentParser parser, GeoPoint point, final boolean ignoreZValue) + public static GeoPoint parseGeoPoint(XContentParser parser, GeoPoint point, final boolean ignoreZValue, EffectivePoint effectivePoint) throws IOException, ElasticsearchParseException { double lat = Double.NaN; double lon = Double.NaN; @@ -458,7 +477,7 @@ public static GeoPoint parseGeoPoint(XContentParser parser, GeoPoint point, fina if(!Double.isNaN(lat) || !Double.isNaN(lon)) { throw new ElasticsearchParseException("field must be either lat/lon or geohash"); } else { - return point.resetFromGeoHash(geohash); + return parseGeoHash(point, geohash, effectivePoint); } } else if (numberFormatException != null) { throw new ElasticsearchParseException("[{}] and [{}] must be valid double values", numberFormatException, LATITUDE, @@ -489,12 +508,36 @@ public static GeoPoint parseGeoPoint(XContentParser parser, GeoPoint point, fina } return point.reset(lat, lon); } else if(parser.currentToken() == Token.VALUE_STRING) { - return point.resetFromString(parser.text(), ignoreZValue); + String val = parser.text(); + if (val.contains(",")) { + return point.resetFromString(val, ignoreZValue); + } else { + return parseGeoHash(point, val, effectivePoint); + } + } else { throw new ElasticsearchParseException("geo_point expected"); } } + private static GeoPoint parseGeoHash(GeoPoint point, String geohash, EffectivePoint effectivePoint) { + if (effectivePoint == EffectivePoint.BOTTOM_LEFT) { + return point.resetFromGeoHash(geohash); + } else { + Rectangle rectangle = GeoHashUtils.bbox(geohash); + switch (effectivePoint) { + case TOP_LEFT: + return point.reset(rectangle.maxLat, rectangle.minLon); + case TOP_RIGHT: + return point.reset(rectangle.maxLat, rectangle.maxLon); + case BOTTOM_RIGHT: + return point.reset(rectangle.minLat, rectangle.maxLon); + default: + throw new IllegalArgumentException("Unsupported effective point " + effectivePoint); + } + } + } + /** * Parse a precision that can be expressed as an integer or a distance measure like "1km", "10m". * diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java index 3fea896342270..3fd325afe0914 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilder.java @@ -491,19 +491,19 @@ public static Rectangle parseBoundingBox(XContentParser parser) throws IOExcepti right = parser.doubleValue(); } else { if (TOP_LEFT_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - GeoUtils.parseGeoPoint(parser, sparse); + GeoUtils.parseGeoPoint(parser, sparse, false, GeoUtils.EffectivePoint.TOP_LEFT); top = sparse.getLat(); left = sparse.getLon(); } else if (BOTTOM_RIGHT_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - GeoUtils.parseGeoPoint(parser, sparse); + GeoUtils.parseGeoPoint(parser, sparse, false, GeoUtils.EffectivePoint.BOTTOM_RIGHT); bottom = sparse.getLat(); right = sparse.getLon(); } else if (TOP_RIGHT_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - GeoUtils.parseGeoPoint(parser, sparse); + GeoUtils.parseGeoPoint(parser, sparse, false, GeoUtils.EffectivePoint.TOP_RIGHT); top = sparse.getLat(); right = sparse.getLon(); } else if (BOTTOM_LEFT_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { - GeoUtils.parseGeoPoint(parser, sparse); + GeoUtils.parseGeoPoint(parser, sparse, false, GeoUtils.EffectivePoint.BOTTOM_LEFT); bottom = sparse.getLat(); left = sparse.getLon(); } else { @@ -515,7 +515,8 @@ public static Rectangle parseBoundingBox(XContentParser parser) throws IOExcepti } } if (envelope != null) { - if ((Double.isNaN(top) || Double.isNaN(bottom) || Double.isNaN(left) || Double.isNaN(right)) == false) { + if (Double.isNaN(top) == false || Double.isNaN(bottom) == false || Double.isNaN(left) == false || + Double.isNaN(right) == false) { throw new ElasticsearchParseException("failed to parse bounding box. Conflicting definition found " + "using well-known text and explicit corners."); } diff --git a/server/src/test/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilderTests.java index aeaca328ceb7b..0f17609ceeee8 100644 --- a/server/src/test/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/GeoBoundingBoxQueryBuilderTests.java @@ -24,6 +24,7 @@ import org.apache.lucene.search.IndexOrDocValuesQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; +import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.geo.GeoUtils; import org.elasticsearch.index.mapper.MappedFieldType; @@ -450,6 +451,64 @@ public void testFromWKT() throws IOException { assertEquals(expectedJson, GeoExecType.MEMORY, parsed.type()); } + public void testFromGeohash() throws IOException { + String json = + "{\n" + + " \"geo_bounding_box\" : {\n" + + " \"pin.location\" : {\n" + + " \"top_left\" : \"dr\",\n" + + " \"bottom_right\" : \"dq\"\n" + + " },\n" + + " \"validation_method\" : \"STRICT\",\n" + + " \"type\" : \"MEMORY\",\n" + + " \"ignore_unmapped\" : false,\n" + + " \"boost\" : 1.0\n" + + " }\n" + + "}"; + + String expectedJson = + "{\n" + + " \"geo_bounding_box\" : {\n" + + " \"pin.location\" : {\n" + + " \"top_left\" : [ -78.75, 45.0 ],\n" + + " \"bottom_right\" : [ -67.5, 33.75 ]\n" + + " },\n" + + " \"validation_method\" : \"STRICT\",\n" + + " \"type\" : \"MEMORY\",\n" + + " \"ignore_unmapped\" : false,\n" + + " \"boost\" : 1.0\n" + + " }\n" + + "}"; + GeoBoundingBoxQueryBuilder parsed = (GeoBoundingBoxQueryBuilder) parseQuery(json); + checkGeneratedJson(expectedJson, parsed); + assertEquals(json, "pin.location", parsed.fieldName()); + assertEquals(json, -78.75, parsed.topLeft().getLon(), 0.0001); + assertEquals(json, 45.0, parsed.topLeft().getLat(), 0.0001); + assertEquals(json, -67.5, parsed.bottomRight().getLon(), 0.0001); + assertEquals(json, 33.75, parsed.bottomRight().getLat(), 0.0001); + assertEquals(json, 1.0, parsed.boost(), 0.0001); + assertEquals(json, GeoExecType.MEMORY, parsed.type()); + } + + public void testMalformedGeohashes() { + String jsonGeohashAndWkt = + "{\n" + + " \"geo_bounding_box\" : {\n" + + " \"pin.location\" : {\n" + + " \"top_left\" : [ -78.75, 45.0 ],\n" + + " \"wkt\" : \"BBOX (-74.1, -71.12, 40.73, 40.01)\"\n" + + " },\n" + + " \"validation_method\" : \"STRICT\",\n" + + " \"type\" : \"MEMORY\",\n" + + " \"ignore_unmapped\" : false,\n" + + " \"boost\" : 1.0\n" + + " }\n" + + "}"; + + ElasticsearchParseException e1 = expectThrows(ElasticsearchParseException.class, () -> parseQuery(jsonGeohashAndWkt)); + assertThat(e1.getMessage(), containsString("Conflicting definition found using well-known text and explicit corners.")); + } + @Override public void testMustRewrite() throws IOException { assumeTrue("test runs only when at least a type is registered", getCurrentTypes().length > 0); diff --git a/server/src/test/java/org/elasticsearch/index/search/geo/GeoUtilsTests.java b/server/src/test/java/org/elasticsearch/index/search/geo/GeoUtilsTests.java index 4ddb80c4b0633..d390490dd225c 100644 --- a/server/src/test/java/org/elasticsearch/index/search/geo/GeoUtilsTests.java +++ b/server/src/test/java/org/elasticsearch/index/search/geo/GeoUtilsTests.java @@ -610,6 +610,20 @@ public void testPrefixTreeCellSizes() { } } + public void testParseGeoPointGeohashPositions() throws IOException { + assertNormalizedPoint(parseGeohash("drt5", GeoUtils.EffectivePoint.TOP_LEFT), new GeoPoint(42.890625, -71.71875)); + assertNormalizedPoint(parseGeohash("drt5", GeoUtils.EffectivePoint.TOP_RIGHT), new GeoPoint(42.890625, -71.3671875)); + assertNormalizedPoint(parseGeohash("drt5", GeoUtils.EffectivePoint.BOTTOM_LEFT), new GeoPoint(42.71484375, -71.71875)); + assertNormalizedPoint(parseGeohash("drt5", GeoUtils.EffectivePoint.BOTTOM_RIGHT), new GeoPoint(42.71484375, -71.3671875)); + assertNormalizedPoint(parseGeohash("drtk", GeoUtils.EffectivePoint.BOTTOM_LEFT), new GeoPoint(42.890625, -71.3671875)); + } + + private GeoPoint parseGeohash(String geohash, GeoUtils.EffectivePoint effectivePoint) throws IOException { + XContentParser parser = createParser(jsonBuilder().startObject().field("geohash", geohash).endObject()); + parser.nextToken(); + return GeoUtils.parseGeoPoint(parser, new GeoPoint(), randomBoolean(), effectivePoint); + } + private static void assertNormalizedPoint(GeoPoint input, GeoPoint expected) { GeoUtils.normalizePoint(input); if (Double.isNaN(expected.lat())) { From 362248688970cf20cdd405eb92b6520ae8da16fb Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Thu, 24 May 2018 14:39:04 -0400 Subject: [PATCH 14/20] Mute IndexMasterFailoverIT.testMasterFailoverDuringIndexingWithMappingChanges Tracked by #30844 --- .../action/support/master/IndexingMasterFailoverIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java b/server/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java index 7a18ca4cff199..c6fb7c49802f1 100644 --- a/server/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java +++ b/server/src/test/java/org/elasticsearch/action/support/master/IndexingMasterFailoverIT.java @@ -66,6 +66,7 @@ protected Settings nodeSettings(int nodeOrdinal) { * This retry logic is implemented in TransportMasterNodeAction and tested by the following master failover scenario. */ @TestLogging("_root:DEBUG") + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/30844") public void testMasterFailoverDuringIndexingWithMappingChanges() throws Throwable { logger.info("--> start 4 nodes, 3 master, 1 data"); From f55b09bae4d6860af425d1dd2cb12e8753b7ea6c Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Thu, 24 May 2018 08:55:14 -0700 Subject: [PATCH 15/20] Update the version checks around ip_range bucket keys, now that the change was backported. --- .../rest-api-spec/test/search.aggregation/40_range.yml | 4 ++-- .../search/aggregations/bucket/range/InternalBinaryRange.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml index bc845928a0460..c75f4175e6b7b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml @@ -213,8 +213,8 @@ setup: --- "IP Range Key Generation": - skip: - version: " - 6.99.99" - reason: "Before 7.0.0, ip_range did not always generate bucket keys (see #21045)." + version: " - 6.3.99" + reason: "Before 6.4.0, ip_range did not always generate bucket keys (see #21045)." - do: search: diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRange.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRange.java index c647a38f7e06e..60431e2f82932 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRange.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/range/InternalBinaryRange.java @@ -74,7 +74,7 @@ private static String generateKey(BytesRef from, BytesRef to, DocValueFormat for } private static Bucket createFromStream(StreamInput in, DocValueFormat format, boolean keyed) throws IOException { - String key = in.getVersion().onOrAfter(Version.V_7_0_0_alpha1) + String key = in.getVersion().onOrAfter(Version.V_6_4_0) ? in.readString() : in.readOptionalString(); @@ -88,7 +88,7 @@ private static Bucket createFromStream(StreamInput in, DocValueFormat format, bo @Override public void writeTo(StreamOutput out) throws IOException { - if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { + if (out.getVersion().onOrAfter(Version.V_6_4_0)) { out.writeString(key); } else { out.writeOptionalString(key); From 5a97423b7ae69afede4060045a62d375aa8f0c7c Mon Sep 17 00:00:00 2001 From: Sohaib Iftikhar Date: Fri, 25 May 2018 01:02:26 +0200 Subject: [PATCH 16/20] REST high-level client: add put ingest pipeline API (#30793) REST high-level client: add put ingest pipeline API Adds the put ingest pipeline API to the high level rest client. --- .../elasticsearch/client/ClusterClient.java | 24 +++++ .../client/RequestConverters.java | 16 +++ .../elasticsearch/client/ClusterClientIT.java | 42 ++++++++ .../client/RequestConvertersTests.java | 23 +++++ .../ClusterClientDocumentationIT.java | 97 +++++++++++++++++-- .../high-level/cluster/put_pipeline.asciidoc | 83 ++++++++++++++++ .../high-level/supported-apis.asciidoc | 2 + .../action/ingest/PutPipelineRequest.java | 14 ++- .../action/ingest/PutPipelineResponse.java | 62 ++++++++++++ .../org/elasticsearch/ingest/Pipeline.java | 8 +- .../ingest/PutPipelineRequestTests.java | 25 +++++ .../ingest/PutPipelineResponseTests.java | 53 ++++++++++ 12 files changed, 435 insertions(+), 14 deletions(-) create mode 100644 docs/java-rest/high-level/cluster/put_pipeline.asciidoc create mode 100644 server/src/main/java/org/elasticsearch/action/ingest/PutPipelineResponse.java create mode 100644 server/src/test/java/org/elasticsearch/action/ingest/PutPipelineResponseTests.java diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java index e78e4686d6991..0f9e9e582263c 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ClusterClient.java @@ -25,6 +25,8 @@ import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.action.ingest.PutPipelineResponse; import java.io.IOException; @@ -87,4 +89,26 @@ public void listTasksAsync(ListTasksRequest request, ActionListener + * See + * Put Pipeline API on elastic.co + */ + public PutPipelineResponse putPipeline(PutPipelineRequest request, Header... headers) throws IOException { + return restHighLevelClient.performRequestAndParseEntity( request, RequestConverters::putPipeline, + PutPipelineResponse::fromXContent, emptySet(), headers); + } + + /** + * Asynchronously add a pipeline or update an existing pipeline in the cluster + *

+ * See + * Put Pipeline API on elastic.co + */ + public void putPipelineAsync(PutPipelineRequest request, ActionListener listener, Header... headers) { + restHighLevelClient.performRequestAsyncAndParseEntity( request, RequestConverters::putPipeline, + PutPipelineResponse::fromXContent, listener, emptySet(), headers); + } } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java index c9526346e5bc1..6126d59b16a71 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java @@ -58,6 +58,7 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.ingest.PutPipelineRequest; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.SearchRequest; @@ -609,6 +610,21 @@ static Request clusterPutSettings(ClusterUpdateSettingsRequest clusterUpdateSett return request; } + static Request putPipeline(PutPipelineRequest putPipelineRequest) throws IOException { + String endpoint = new EndpointBuilder() + .addPathPartAsIs("_ingest/pipeline") + .addPathPart(putPipelineRequest.getId()) + .build(); + Request request = new Request(HttpPut.METHOD_NAME, endpoint); + + Params parameters = new Params(request); + parameters.withTimeout(putPipelineRequest.timeout()); + parameters.withMasterTimeout(putPipelineRequest.masterNodeTimeout()); + + request.setEntity(createEntity(putPipelineRequest, REQUEST_BODY_CONTENT_TYPE)); + return request; + } + static Request listTasks(ListTasksRequest listTaskRequest) { if (listTaskRequest.getTaskId() != null && listTaskRequest.getTaskId().isSet()) { throw new IllegalArgumentException("TaskId cannot be used for list tasks request"); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ClusterClientIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ClusterClientIT.java index fa3086442f528..d41117ceb6dd6 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ClusterClientIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ClusterClientIT.java @@ -25,12 +25,17 @@ import org.elasticsearch.action.admin.cluster.node.tasks.list.TaskGroup; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.action.ingest.PutPipelineResponse; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.indices.recovery.RecoverySettings; +import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.TaskInfo; @@ -136,4 +141,41 @@ public void testListTasks() throws IOException { } assertTrue("List tasks were not found", listTasksFound); } + + public void testPutPipeline() throws IOException { + String id = "some_pipeline_id"; + XContentType xContentType = randomFrom(XContentType.values()); + XContentBuilder pipelineBuilder = XContentBuilder.builder(xContentType.xContent()); + pipelineBuilder.startObject(); + { + pipelineBuilder.field(Pipeline.DESCRIPTION_KEY, "some random set of processors"); + pipelineBuilder.startArray(Pipeline.PROCESSORS_KEY); + { + pipelineBuilder.startObject().startObject("set"); + { + pipelineBuilder + .field("field", "foo") + .field("value", "bar"); + } + pipelineBuilder.endObject().endObject(); + pipelineBuilder.startObject().startObject("convert"); + { + pipelineBuilder + .field("field", "rank") + .field("type", "integer"); + } + pipelineBuilder.endObject().endObject(); + } + pipelineBuilder.endArray(); + } + pipelineBuilder.endObject(); + PutPipelineRequest request = new PutPipelineRequest( + id, + BytesReference.bytes(pipelineBuilder), + pipelineBuilder.contentType()); + + PutPipelineResponse putPipelineResponse = + execute(request, highLevelClient().cluster()::putPipeline, highLevelClient().cluster()::putPipelineAsync); + assertTrue(putPipelineResponse.isAcknowledged()); + } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java index c5ee387d315cb..1573071da3372 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java @@ -61,6 +61,7 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.ingest.PutPipelineRequest; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.SearchRequest; @@ -91,6 +92,7 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.RandomCreateIndexGenerator; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.query.TermQueryBuilder; @@ -119,6 +121,7 @@ import java.io.IOException; import java.io.InputStream; +import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; @@ -1402,6 +1405,26 @@ public void testClusterPutSettings() throws IOException { assertEquals(expectedParams, expectedRequest.getParameters()); } + public void testPutPipeline() throws IOException { + String pipelineId = "some_pipeline_id"; + PutPipelineRequest request = new PutPipelineRequest( + "some_pipeline_id", + new BytesArray("{}".getBytes(StandardCharsets.UTF_8)), + XContentType.JSON + ); + Map expectedParams = new HashMap<>(); + setRandomMasterTimeout(request, expectedParams); + setRandomTimeout(request::timeout, AcknowledgedRequest.DEFAULT_ACK_TIMEOUT, expectedParams); + + Request expectedRequest = RequestConverters.putPipeline(request); + StringJoiner endpoint = new StringJoiner("/", "/", ""); + endpoint.add("_ingest/pipeline"); + endpoint.add(pipelineId); + assertEquals(endpoint.toString(), expectedRequest.getEndpoint()); + assertEquals(HttpPut.METHOD_NAME, expectedRequest.getMethod()); + assertEquals(expectedParams, expectedRequest.getParameters()); + } + public void testRollover() throws IOException { RolloverRequest rolloverRequest = new RolloverRequest(randomAlphaOfLengthBetween(3, 10), randomBoolean() ? null : randomAlphaOfLengthBetween(3, 10)); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java index d41b11c68fe44..b9329f99a3cde 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ClusterClientDocumentationIT.java @@ -21,7 +21,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksRequest; @@ -29,9 +28,12 @@ import org.elasticsearch.action.admin.cluster.node.tasks.list.TaskGroup; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.elasticsearch.action.ingest.PutPipelineRequest; +import org.elasticsearch.action.ingest.PutPipelineResponse; import org.elasticsearch.client.ESRestHighLevelClientTestCase; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.TimeValue; @@ -41,6 +43,7 @@ import org.elasticsearch.tasks.TaskInfo; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -80,19 +83,19 @@ public void testClusterPutSettings() throws IOException { // end::put-settings-request // tag::put-settings-create-settings - String transientSettingKey = + String transientSettingKey = RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(); int transientSettingValue = 10; - Settings transientSettings = + Settings transientSettings = Settings.builder() .put(transientSettingKey, transientSettingValue, ByteSizeUnit.BYTES) .build(); // <1> - String persistentSettingKey = + String persistentSettingKey = EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(); - String persistentSettingValue = + String persistentSettingValue = EnableAllocationDecider.Allocation.NONE.name(); - Settings persistentSettings = + Settings persistentSettings = Settings.builder() .put(persistentSettingKey, persistentSettingValue) .build(); // <2> @@ -105,9 +108,9 @@ public void testClusterPutSettings() throws IOException { { // tag::put-settings-settings-builder - Settings.Builder transientSettingsBuilder = + Settings.Builder transientSettingsBuilder = Settings.builder() - .put(transientSettingKey, transientSettingValue, ByteSizeUnit.BYTES); + .put(transientSettingKey, transientSettingValue, ByteSizeUnit.BYTES); request.transientSettings(transientSettingsBuilder); // <1> // end::put-settings-settings-builder } @@ -164,7 +167,7 @@ public void testClusterUpdateSettingsAsync() throws Exception { ClusterUpdateSettingsRequest request = new ClusterUpdateSettingsRequest(); // tag::put-settings-execute-listener - ActionListener listener = + ActionListener listener = new ActionListener() { @Override public void onResponse(ClusterUpdateSettingsResponse response) { @@ -272,4 +275,80 @@ public void onFailure(Exception e) { assertTrue(latch.await(30L, TimeUnit.SECONDS)); } } + + public void testPutPipeline() throws IOException { + RestHighLevelClient client = highLevelClient(); + + { + // tag::put-pipeline-request + String source = + "{\"description\":\"my set of processors\"," + + "\"processors\":[{\"set\":{\"field\":\"foo\",\"value\":\"bar\"}}]}"; + PutPipelineRequest request = new PutPipelineRequest( + "my-pipeline-id", // <1> + new BytesArray(source.getBytes(StandardCharsets.UTF_8)), // <2> + XContentType.JSON // <3> + ); + // end::put-pipeline-request + + // tag::put-pipeline-request-timeout + request.timeout(TimeValue.timeValueMinutes(2)); // <1> + request.timeout("2m"); // <2> + // end::put-pipeline-request-timeout + + // tag::put-pipeline-request-masterTimeout + request.masterNodeTimeout(TimeValue.timeValueMinutes(1)); // <1> + request.masterNodeTimeout("1m"); // <2> + // end::put-pipeline-request-masterTimeout + + // tag::put-pipeline-execute + PutPipelineResponse response = client.cluster().putPipeline(request); // <1> + // end::put-pipeline-execute + + // tag::put-pipeline-response + boolean acknowledged = response.isAcknowledged(); // <1> + // end::put-pipeline-response + assertTrue(acknowledged); + } + } + + public void testPutPipelineAsync() throws Exception { + RestHighLevelClient client = highLevelClient(); + + { + String source = + "{\"description\":\"my set of processors\"," + + "\"processors\":[{\"set\":{\"field\":\"foo\",\"value\":\"bar\"}}]}"; + PutPipelineRequest request = new PutPipelineRequest( + "my-pipeline-id", + new BytesArray(source.getBytes(StandardCharsets.UTF_8)), + XContentType.JSON + ); + + // tag::put-pipeline-execute-listener + ActionListener listener = + new ActionListener() { + @Override + public void onResponse(PutPipelineResponse response) { + // <1> + } + + @Override + public void onFailure(Exception e) { + // <2> + } + }; + // end::put-pipeline-execute-listener + + // Replace the empty listener by a blocking listener in test + final CountDownLatch latch = new CountDownLatch(1); + listener = new LatchedActionListener<>(listener, latch); + + // tag::put-pipeline-execute-async + client.cluster().putPipelineAsync(request, listener); // <1> + // end::put-pipeline-execute-async + + assertTrue(latch.await(30L, TimeUnit.SECONDS)); + } + } } diff --git a/docs/java-rest/high-level/cluster/put_pipeline.asciidoc b/docs/java-rest/high-level/cluster/put_pipeline.asciidoc new file mode 100644 index 0000000000000..d50a6741cc0a9 --- /dev/null +++ b/docs/java-rest/high-level/cluster/put_pipeline.asciidoc @@ -0,0 +1,83 @@ +[[java-rest-high-cluster-put-pipeline]] +=== Put Pipeline API + +[[java-rest-high-cluster-put-pipeline-request]] +==== Put Pipeline Request + +A `PutPipelineRequest` requires an `id` argument, a source and a `XContentType`. The source consists +of a description and a list of `Processor` objects. + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-request] +-------------------------------------------------- +<1> The pipeline id +<2> The source for the pipeline as a `ByteArray`. +<3> The XContentType for the pipeline source supplied above. + +==== Optional arguments +The following arguments can optionally be provided: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-request-timeout] +-------------------------------------------------- +<1> Timeout to wait for the all the nodes to acknowledge the index creation as a `TimeValue` +<2> Timeout to wait for the all the nodes to acknowledge the index creation as a `String` + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-request-masterTimeout] +-------------------------------------------------- +<1> Timeout to connect to the master node as a `TimeValue` +<2> Timeout to connect to the master node as a `String` + +[[java-rest-high-cluster-put-pipeline-sync]] +==== Synchronous Execution + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-execute] +-------------------------------------------------- +<1> Execute the request and get back the response in a PutPipelineResponse object. + +[[java-rest-high-cluster-put-pipeline-async]] +==== Asynchronous Execution + +The asynchronous execution of a put pipeline request requires both the `PutPipelineRequest` +instance and an `ActionListener` instance to be passed to the asynchronous +method: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-execute-async] +-------------------------------------------------- +<1> The `PutPipelineRequest` to execute and the `ActionListener` to use when +the execution completes + +The asynchronous method does not block and returns immediately. Once it is +completed the `ActionListener` is called back using the `onResponse` method +if the execution successfully completed or using the `onFailure` method if +it failed. + +A typical listener for `PutPipelineResponse` looks like: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-execute-listener] +-------------------------------------------------- +<1> Called when the execution is successfully completed. The response is +provided as an argument +<2> Called in case of failure. The raised exception is provided as an argument + +[[java-rest-high-cluster-put-pipeline-response]] +==== Put Pipeline Response + +The returned `PutPipelineResponse` allows to retrieve information about the executed + operation as follows: + +["source","java",subs="attributes,callouts,macros"] +-------------------------------------------------- +include-tagged::{doc-tests}/ClusterClientDocumentationIT.java[put-pipeline-response] +-------------------------------------------------- +<1> Indicates whether all of the nodes have acknowledged the request diff --git a/docs/java-rest/high-level/supported-apis.asciidoc b/docs/java-rest/high-level/supported-apis.asciidoc index d8ec67dade10b..b04cbb8df79b7 100644 --- a/docs/java-rest/high-level/supported-apis.asciidoc +++ b/docs/java-rest/high-level/supported-apis.asciidoc @@ -105,9 +105,11 @@ The Java High Level REST Client supports the following Cluster APIs: * <> * <> +* <> include::cluster/put_settings.asciidoc[] include::cluster/list_tasks.asciidoc[] +include::cluster/put_pipeline.asciidoc[] == Snapshot APIs diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java index 722473d64e40c..6447b0557db0c 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java @@ -25,13 +25,15 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import java.io.IOException; import java.util.Objects; -public class PutPipelineRequest extends AcknowledgedRequest { +public class PutPipelineRequest extends AcknowledgedRequest implements ToXContentObject { private String id; private BytesReference source; @@ -96,4 +98,14 @@ public void writeTo(StreamOutput out) throws IOException { out.writeEnum(xContentType); } } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (source != null) { + builder.rawValue(source.streamInput(), xContentType); + } else { + builder.startObject().endObject(); + } + return builder; + } } diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineResponse.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineResponse.java new file mode 100644 index 0000000000000..13960ca99ef7e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineResponse.java @@ -0,0 +1,62 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentParser; + +import java.io.IOException; + +public class PutPipelineResponse extends AcknowledgedResponse implements ToXContentObject { + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "cluster_put_pipeline", true, args -> new PutPipelineResponse((boolean) args[0])); + + static { + declareAcknowledgedField(PARSER); + } + + public PutPipelineResponse() { + } + + public PutPipelineResponse(boolean acknowledged) { + super(acknowledged); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + super.readFrom(in); + readAcknowledged(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + writeAcknowledged(out); + } + + public static PutPipelineResponse fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } +} diff --git a/server/src/main/java/org/elasticsearch/ingest/Pipeline.java b/server/src/main/java/org/elasticsearch/ingest/Pipeline.java index 473b555c05d22..1b0553a54902b 100644 --- a/server/src/main/java/org/elasticsearch/ingest/Pipeline.java +++ b/server/src/main/java/org/elasticsearch/ingest/Pipeline.java @@ -32,10 +32,10 @@ */ public final class Pipeline { - static final String DESCRIPTION_KEY = "description"; - static final String PROCESSORS_KEY = "processors"; - static final String VERSION_KEY = "version"; - static final String ON_FAILURE_KEY = "on_failure"; + public static final String DESCRIPTION_KEY = "description"; + public static final String PROCESSORS_KEY = "processors"; + public static final String VERSION_KEY = "version"; + public static final String ON_FAILURE_KEY = "on_failure"; private final String id; @Nullable diff --git a/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineRequestTests.java b/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineRequestTests.java index 304904f2612c4..7f64b3fe585f9 100644 --- a/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineRequestTests.java @@ -20,9 +20,13 @@ package org.elasticsearch.action.ingest; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.ingest.Pipeline; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -43,4 +47,25 @@ public void testSerializationWithXContent() throws IOException { assertEquals(XContentType.JSON, serialized.getXContentType()); assertEquals("{}", serialized.getSource().utf8ToString()); } + + public void testToXContent() throws IOException { + XContentType xContentType = randomFrom(XContentType.values()); + XContentBuilder pipelineBuilder = XContentBuilder.builder(xContentType.xContent()); + pipelineBuilder.startObject().field(Pipeline.DESCRIPTION_KEY, "some random set of processors"); + pipelineBuilder.startArray(Pipeline.PROCESSORS_KEY); + //Start first processor + pipelineBuilder.startObject(); + pipelineBuilder.startObject("set"); + pipelineBuilder.field("field", "foo"); + pipelineBuilder.field("value", "bar"); + pipelineBuilder.endObject(); + pipelineBuilder.endObject(); + //End first processor + pipelineBuilder.endArray(); + pipelineBuilder.endObject(); + PutPipelineRequest request = new PutPipelineRequest("1", BytesReference.bytes(pipelineBuilder), xContentType); + XContentBuilder requestBuilder = XContentBuilder.builder(xContentType.xContent()); + BytesReference actualRequestBody = BytesReference.bytes(request.toXContent(requestBuilder, ToXContent.EMPTY_PARAMS)); + assertEquals(BytesReference.bytes(pipelineBuilder), actualRequestBody); + } } diff --git a/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineResponseTests.java b/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineResponseTests.java new file mode 100644 index 0000000000000..438d3e550442c --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/ingest/PutPipelineResponseTests.java @@ -0,0 +1,53 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.ingest; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.test.AbstractStreamableXContentTestCase; + +public class PutPipelineResponseTests extends AbstractStreamableXContentTestCase { + + public void testToXContent() { + PutPipelineResponse response = new PutPipelineResponse(true); + String output = Strings.toString(response); + assertEquals("{\"acknowledged\":true}", output); + } + + @Override + protected PutPipelineResponse doParseInstance(XContentParser parser) { + return PutPipelineResponse.fromXContent(parser); + } + + @Override + protected PutPipelineResponse createTestInstance() { + return new PutPipelineResponse(randomBoolean()); + } + + @Override + protected PutPipelineResponse createBlankInstance() { + return new PutPipelineResponse(); + } + + @Override + protected PutPipelineResponse mutateInstance(PutPipelineResponse response) { + return new PutPipelineResponse(response.isAcknowledged() == false); + } +} From e8b70273c15f81c04a15d75de94dc251e99ac1e2 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 24 May 2018 17:33:29 -0600 Subject: [PATCH 17/20] Remove Throwable usage from transport modules (#30845) Currently nio and netty modules use the CompletableFuture class for managing listeners. This is unfortunate as that class accepts Throwable. This commit adds a class CompletableContext that wraps the CompletableFuture but does not accept Throwable. This allows the modification of netty and nio logic to no longer handle Throwable. --- .../common/concurrent/CompletableContext.java | 63 +++++++++++++++++++ libs/elasticsearch-nio/build.gradle | 2 + .../elasticsearch/nio/BytesWriteHandler.java | 2 +- .../org/elasticsearch/nio/ChannelContext.java | 9 +-- .../org/elasticsearch/nio/FlushOperation.java | 6 +- .../elasticsearch/nio/FlushReadyWrite.java | 2 +- .../org/elasticsearch/nio/NioChannel.java | 2 +- .../elasticsearch/nio/NioSocketChannel.java | 2 +- .../elasticsearch/nio/ReadWriteHandler.java | 2 +- .../nio/SocketChannelContext.java | 10 +-- .../org/elasticsearch/nio/SocketSelector.java | 4 +- .../org/elasticsearch/nio/WriteOperation.java | 2 +- .../nio/BytesChannelContextTests.java | 4 +- .../nio/ChannelContextTests.java | 2 +- .../nio/FlushOperationTests.java | 2 +- .../nio/SocketChannelContextTests.java | 8 +-- .../nio/SocketSelectorTests.java | 2 +- .../transport/netty4/NettyTcpChannel.java | 15 ++--- .../http/nio/HttpReadWriteHandler.java | 2 +- .../http/nio/HttpWriteOperation.java | 6 +- .../elasticsearch/http/nio/NettyListener.java | 21 ++++--- .../http/nio/NioHttpChannel.java | 2 +- .../elasticsearch/action/ActionListener.java | 10 +-- .../transport/MockTcpTransport.java | 6 +- .../transport/nio/SSLChannelContext.java | 4 +- .../transport/nio/SSLChannelContextTests.java | 4 +- 26 files changed, 127 insertions(+), 67 deletions(-) create mode 100644 libs/elasticsearch-core/src/main/java/org/elasticsearch/common/concurrent/CompletableContext.java diff --git a/libs/elasticsearch-core/src/main/java/org/elasticsearch/common/concurrent/CompletableContext.java b/libs/elasticsearch-core/src/main/java/org/elasticsearch/common/concurrent/CompletableContext.java new file mode 100644 index 0000000000000..e838842931837 --- /dev/null +++ b/libs/elasticsearch-core/src/main/java/org/elasticsearch/common/concurrent/CompletableContext.java @@ -0,0 +1,63 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common.concurrent; + +import java.util.concurrent.CompletableFuture; +import java.util.function.BiConsumer; + +/** + * A thread-safe completable context that allows listeners to be attached. This class relies on the + * {@link CompletableFuture} for the concurrency logic. However, it does not accept {@link Throwable} as + * an exceptional result. This allows attaching listeners that only handle {@link Exception}. + * + * @param the result type + */ +public class CompletableContext { + + private final CompletableFuture completableFuture = new CompletableFuture<>(); + + public void addListener(BiConsumer listener) { + BiConsumer castThrowable = (v, t) -> { + if (t == null) { + listener.accept(v, null); + } else { + assert !(t instanceof Error) : "Cannot be error"; + listener.accept(v, (Exception) t); + } + }; + completableFuture.whenComplete(castThrowable); + } + + public boolean isDone() { + return completableFuture.isDone(); + } + + public boolean isCompletedExceptionally() { + return completableFuture.isCompletedExceptionally(); + } + + public boolean completeExceptionally(Exception ex) { + return completableFuture.completeExceptionally(ex); + } + + public boolean complete(T value) { + return completableFuture.complete(value); + } +} diff --git a/libs/elasticsearch-nio/build.gradle b/libs/elasticsearch-nio/build.gradle index 018874adf7082..f8b0b8fba136d 100644 --- a/libs/elasticsearch-nio/build.gradle +++ b/libs/elasticsearch-nio/build.gradle @@ -33,6 +33,8 @@ publishing { } dependencies { + compile "org.elasticsearch:elasticsearch-core:${version}" + testCompile "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" testCompile "junit:junit:${versions.junit}" testCompile "org.hamcrest:hamcrest-all:${versions.hamcrest}" diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java index ba379e2873210..87c0ff2817eb7 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/BytesWriteHandler.java @@ -28,7 +28,7 @@ public abstract class BytesWriteHandler implements ReadWriteHandler { private static final List EMPTY_LIST = Collections.emptyList(); - public WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer listener) { + public WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer listener) { assert message instanceof ByteBuffer[] : "This channel only supports messages that are of type: " + ByteBuffer[].class + ". Found type: " + message.getClass() + "."; return new FlushReadyWrite(context, (ByteBuffer[]) message, listener); diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/ChannelContext.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/ChannelContext.java index 01f35347aa411..93930bbabf058 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/ChannelContext.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/ChannelContext.java @@ -19,11 +19,12 @@ package org.elasticsearch.nio; +import org.elasticsearch.common.concurrent.CompletableContext; + import java.io.IOException; import java.nio.channels.NetworkChannel; import java.nio.channels.SelectableChannel; import java.nio.channels.SelectionKey; -import java.util.concurrent.CompletableFuture; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -37,7 +38,7 @@ public abstract class ChannelContext exceptionHandler; - private final CompletableFuture closeContext = new CompletableFuture<>(); + private final CompletableContext closeContext = new CompletableContext<>(); private volatile SelectionKey selectionKey; ChannelContext(S rawChannel, Consumer exceptionHandler) { @@ -81,8 +82,8 @@ public void closeFromSelector() throws IOException { * * @param listener to be called */ - public void addCloseListener(BiConsumer listener) { - closeContext.whenComplete(listener); + public void addCloseListener(BiConsumer listener) { + closeContext.addListener(listener); } public boolean isOpen() { diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/FlushOperation.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/FlushOperation.java index 3102c972a6795..7a1696483db06 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/FlushOperation.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/FlushOperation.java @@ -25,13 +25,13 @@ public class FlushOperation { - private final BiConsumer listener; + private final BiConsumer listener; private final ByteBuffer[] buffers; private final int[] offsets; private final int length; private int internalIndex; - public FlushOperation(ByteBuffer[] buffers, BiConsumer listener) { + public FlushOperation(ByteBuffer[] buffers, BiConsumer listener) { this.listener = listener; this.buffers = buffers; this.offsets = new int[buffers.length]; @@ -44,7 +44,7 @@ public FlushOperation(ByteBuffer[] buffers, BiConsumer listener length = offset; } - public BiConsumer getListener() { + public BiConsumer getListener() { return listener; } diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/FlushReadyWrite.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/FlushReadyWrite.java index 65bc8f17aaf4b..61c997603ff97 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/FlushReadyWrite.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/FlushReadyWrite.java @@ -27,7 +27,7 @@ public class FlushReadyWrite extends FlushOperation implements WriteOperation { private final SocketChannelContext channelContext; private final ByteBuffer[] buffers; - FlushReadyWrite(SocketChannelContext channelContext, ByteBuffer[] buffers, BiConsumer listener) { + FlushReadyWrite(SocketChannelContext channelContext, ByteBuffer[] buffers, BiConsumer listener) { super(buffers, listener); this.channelContext = channelContext; this.buffers = buffers; diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/NioChannel.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/NioChannel.java index 2f9705f5f8f61..ea633bd3276a3 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/NioChannel.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/NioChannel.java @@ -53,7 +53,7 @@ public InetSocketAddress getLocalAddress() { * * @param listener to be called at close */ - public void addCloseListener(BiConsumer listener) { + public void addCloseListener(BiConsumer listener) { getContext().addCloseListener(listener); } diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/NioSocketChannel.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/NioSocketChannel.java index 1b8f11e73d431..32e934766913e 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/NioSocketChannel.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/NioSocketChannel.java @@ -60,7 +60,7 @@ public InetSocketAddress getRemoteAddress() { return remoteAddress; } - public void addConnectListener(BiConsumer listener) { + public void addConnectListener(BiConsumer listener) { context.addConnectListener(listener); } diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java index f0637ea265280..6b8688eccfd8c 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/ReadWriteHandler.java @@ -38,7 +38,7 @@ public interface ReadWriteHandler { * @param listener the listener to be called when the message is sent * @return the write operation to be queued */ - WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer listener); + WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer listener); /** * This method is called on the event loop thread. It should serialize a write operation object to bytes diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java index f2d299a9d328a..6a769b4d17381 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java @@ -19,6 +19,7 @@ package org.elasticsearch.nio; +import org.elasticsearch.common.concurrent.CompletableContext; import org.elasticsearch.nio.utils.ExceptionsHelper; import java.io.IOException; @@ -27,7 +28,6 @@ import java.nio.channels.SocketChannel; import java.util.ArrayList; import java.util.LinkedList; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; import java.util.function.Consumer; @@ -48,7 +48,7 @@ public abstract class SocketChannelContext extends ChannelContext protected final AtomicBoolean isClosing = new AtomicBoolean(false); private final ReadWriteHandler readWriteHandler; private final SocketSelector selector; - private final CompletableFuture connectContext = new CompletableFuture<>(); + private final CompletableContext connectContext = new CompletableContext<>(); private final LinkedList pendingFlushes = new LinkedList<>(); private boolean ioException; private boolean peerClosed; @@ -73,8 +73,8 @@ public NioSocketChannel getChannel() { return channel; } - public void addConnectListener(BiConsumer listener) { - connectContext.whenComplete(listener); + public void addConnectListener(BiConsumer listener) { + connectContext.addListener(listener); } public boolean isConnectComplete() { @@ -121,7 +121,7 @@ public boolean connect() throws IOException { return isConnected; } - public void sendMessage(Object message, BiConsumer listener) { + public void sendMessage(Object message, BiConsumer listener) { if (isClosing.get()) { listener.accept(null, new ClosedChannelException()); return; diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/SocketSelector.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/SocketSelector.java index 88b3cef41cd01..30ef7b317a3f8 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/SocketSelector.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/SocketSelector.java @@ -138,7 +138,7 @@ public void queueWriteInChannelBuffer(WriteOperation writeOperation) { * @param listener to be executed * @param value to provide to listener */ - public void executeListener(BiConsumer listener, V value) { + public void executeListener(BiConsumer listener, V value) { assertOnSelectorThread(); try { listener.accept(value, null); @@ -154,7 +154,7 @@ public void executeListener(BiConsumer listener, V value) { * @param listener to be executed * @param exception to provide to listener */ - public void executeFailedListener(BiConsumer listener, Exception exception) { + public void executeFailedListener(BiConsumer listener, Exception exception) { assertOnSelectorThread(); try { listener.accept(null, exception); diff --git a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/WriteOperation.java b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/WriteOperation.java index 25de6ab7326f3..3d17519be7e1f 100644 --- a/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/WriteOperation.java +++ b/libs/elasticsearch-nio/src/main/java/org/elasticsearch/nio/WriteOperation.java @@ -27,7 +27,7 @@ */ public interface WriteOperation { - BiConsumer getListener(); + BiConsumer getListener(); SocketChannelContext getChannel(); diff --git a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/BytesChannelContextTests.java b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/BytesChannelContextTests.java index addfcdedbf99f..e5c236e48a89c 100644 --- a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/BytesChannelContextTests.java +++ b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/BytesChannelContextTests.java @@ -45,7 +45,7 @@ public class BytesChannelContextTests extends ESTestCase { private BytesChannelContext context; private InboundChannelBuffer channelBuffer; private SocketSelector selector; - private BiConsumer listener; + private BiConsumer listener; private int messageLength; @Before @@ -191,7 +191,7 @@ public void testPartialFlush() throws IOException { public void testMultipleWritesPartialFlushes() throws IOException { assertFalse(context.readyForFlush()); - BiConsumer listener2 = mock(BiConsumer.class); + BiConsumer listener2 = mock(BiConsumer.class); FlushReadyWrite flushOperation1 = mock(FlushReadyWrite.class); FlushReadyWrite flushOperation2 = mock(FlushReadyWrite.class); when(flushOperation1.getBuffersToWrite()).thenReturn(new ByteBuffer[0]); diff --git a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/ChannelContextTests.java b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/ChannelContextTests.java index f262dd063306d..586dae83d082d 100644 --- a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/ChannelContextTests.java +++ b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/ChannelContextTests.java @@ -83,7 +83,7 @@ public void testCloseException() throws IOException { if (t == null) { throw new AssertionError("Close should not fail"); } else { - exception.set((Exception) t); + exception.set(t); } }); diff --git a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/FlushOperationTests.java b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/FlushOperationTests.java index a244de51f3591..0f3078715fdac 100644 --- a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/FlushOperationTests.java +++ b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/FlushOperationTests.java @@ -31,7 +31,7 @@ public class FlushOperationTests extends ESTestCase { - private BiConsumer listener; + private BiConsumer listener; @Before @SuppressWarnings("unchecked") diff --git a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java index d6787f7cc1534..f27052ac5d5f4 100644 --- a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java +++ b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/SocketChannelContextTests.java @@ -50,7 +50,7 @@ public class SocketChannelContextTests extends ESTestCase { private TestSocketChannelContext context; private Consumer exceptionHandler; private NioSocketChannel channel; - private BiConsumer listener; + private BiConsumer listener; private SocketSelector selector; private ReadWriteHandler readWriteHandler; @@ -125,7 +125,7 @@ public void testConnectFails() throws IOException { if (t == null) { throw new AssertionError("Connection should not succeed"); } else { - exception.set((Exception) t); + exception.set(t); } }); @@ -206,7 +206,7 @@ public void testFlushOpsClearedOnClose() throws Exception { ByteBuffer[] buffer = {ByteBuffer.allocate(10)}; WriteOperation writeOperation = mock(WriteOperation.class); - BiConsumer listener2 = mock(BiConsumer.class); + BiConsumer listener2 = mock(BiConsumer.class); when(readWriteHandler.writeToBytes(writeOperation)).thenReturn(Arrays.asList(new FlushOperation(buffer, listener), new FlushOperation(buffer, listener2))); context.queueWriteOperation(writeOperation); @@ -232,7 +232,7 @@ public void testWillPollForFlushOpsToClose() throws Exception { ByteBuffer[] buffer = {ByteBuffer.allocate(10)}; - BiConsumer listener2 = mock(BiConsumer.class); + BiConsumer listener2 = mock(BiConsumer.class); assertFalse(context.readyForFlush()); when(channel.isOpen()).thenReturn(true); diff --git a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/SocketSelectorTests.java b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/SocketSelectorTests.java index 78911f2028953..f8775d03b4277 100644 --- a/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/SocketSelectorTests.java +++ b/libs/elasticsearch-nio/src/test/java/org/elasticsearch/nio/SocketSelectorTests.java @@ -50,7 +50,7 @@ public class SocketSelectorTests extends ESTestCase { private NioSocketChannel channel; private TestSelectionKey selectionKey; private SocketChannelContext channelContext; - private BiConsumer listener; + private BiConsumer listener; private ByteBuffer[] buffers = {ByteBuffer.allocate(1)}; private Selector rawSelector; diff --git a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java index 602835b5ca29e..f650e757e7a62 100644 --- a/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java +++ b/modules/transport-netty4/src/main/java/org/elasticsearch/transport/netty4/NettyTcpChannel.java @@ -20,26 +20,21 @@ package org.elasticsearch.transport.netty4; import io.netty.channel.Channel; -import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelOption; import io.netty.channel.ChannelPromise; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.logging.log4j.util.Supplier; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.concurrent.CompletableContext; import org.elasticsearch.transport.TcpChannel; import org.elasticsearch.transport.TransportException; import java.net.InetSocketAddress; -import java.nio.channels.ClosedSelectorException; -import java.util.concurrent.CompletableFuture; public class NettyTcpChannel implements TcpChannel { private final Channel channel; private final String profile; - private final CompletableFuture closeContext = new CompletableFuture<>(); + private final CompletableContext closeContext = new CompletableContext<>(); NettyTcpChannel(Channel channel, String profile) { this.channel = channel; @@ -51,9 +46,9 @@ public class NettyTcpChannel implements TcpChannel { Throwable cause = f.cause(); if (cause instanceof Error) { Netty4Utils.maybeDie(cause); - closeContext.completeExceptionally(cause); + closeContext.completeExceptionally(new Exception(cause)); } else { - closeContext.completeExceptionally(cause); + closeContext.completeExceptionally((Exception) cause); } } }); @@ -71,7 +66,7 @@ public String getProfile() { @Override public void addCloseListener(ActionListener listener) { - closeContext.whenComplete(ActionListener.toBiConsumer(listener)); + closeContext.addListener(ActionListener.toBiConsumer(listener)); } @Override diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadWriteHandler.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadWriteHandler.java index e3481e3c254d2..681736a311db5 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadWriteHandler.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpReadWriteHandler.java @@ -96,7 +96,7 @@ public int consumeReads(InboundChannelBuffer channelBuffer) throws IOException { } @Override - public WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer listener) { + public WriteOperation createWriteOperation(SocketChannelContext context, Object message, BiConsumer listener) { assert message instanceof NioHttpResponse : "This channel only supports messages that are of type: " + NioHttpResponse.class + ". Found type: " + message.getClass() + "."; return new HttpWriteOperation(context, (NioHttpResponse) message, listener); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java index 8ddce7a5b73b5..207843bfe396a 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/HttpWriteOperation.java @@ -28,16 +28,16 @@ public class HttpWriteOperation implements WriteOperation { private final SocketChannelContext channelContext; private final NioHttpResponse response; - private final BiConsumer listener; + private final BiConsumer listener; - HttpWriteOperation(SocketChannelContext channelContext, NioHttpResponse response, BiConsumer listener) { + HttpWriteOperation(SocketChannelContext channelContext, NioHttpResponse response, BiConsumer listener) { this.channelContext = channelContext; this.response = response; this.listener = listener; } @Override - public BiConsumer getListener() { + public BiConsumer getListener() { return listener; } diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NettyListener.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NettyListener.java index b907c0f2bc6f6..2cdaa4708d15a 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NettyListener.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NettyListener.java @@ -36,7 +36,7 @@ * complete that promise when accept is called. It delegates the normal promise methods to the underlying * promise. */ -public class NettyListener implements BiConsumer, ChannelPromise { +public class NettyListener implements BiConsumer, ChannelPromise { private final ChannelPromise promise; @@ -45,11 +45,11 @@ private NettyListener(ChannelPromise promise) { } @Override - public void accept(Void v, Throwable throwable) { - if (throwable == null) { + public void accept(Void v, Exception exception) { + if (exception == null) { promise.setSuccess(); } else { - promise.setFailure(throwable); + promise.setFailure(exception); } } @@ -212,17 +212,22 @@ public ChannelPromise unvoid() { return promise.unvoid(); } - public static NettyListener fromBiConsumer(BiConsumer biConsumer, Channel channel) { + public static NettyListener fromBiConsumer(BiConsumer biConsumer, Channel channel) { if (biConsumer instanceof NettyListener) { return (NettyListener) biConsumer; } else { ChannelPromise channelPromise = channel.newPromise(); channelPromise.addListener(f -> { - if (f.cause() == null) { + Throwable cause = f.cause(); + if (cause == null) { biConsumer.accept(null, null); } else { - ExceptionsHelper.dieOnError(f.cause()); - biConsumer.accept(null, f.cause()); + if (cause instanceof Error) { + ExceptionsHelper.dieOnError(cause); + biConsumer.accept(null, new Exception(cause)); + } else { + biConsumer.accept(null, (Exception) cause); + } } }); diff --git a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java index 97eba20a16f16..61cafed86a51c 100644 --- a/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java +++ b/plugins/transport-nio/src/main/java/org/elasticsearch/http/nio/NioHttpChannel.java @@ -120,7 +120,7 @@ public void sendResponse(RestResponse response) { toClose.add(nioChannel::close); } - BiConsumer listener = (aVoid, throwable) -> Releasables.close(toClose); + BiConsumer listener = (aVoid, ex) -> Releasables.close(toClose); nioChannel.getContext().sendMessage(new NioHttpResponse(sequence, resp), listener); success = true; } finally { diff --git a/server/src/main/java/org/elasticsearch/action/ActionListener.java b/server/src/main/java/org/elasticsearch/action/ActionListener.java index 8579fb55613ce..f639f139b55fc 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/ActionListener.java @@ -90,18 +90,12 @@ static ActionListener wrap(Runnable runnable) { * @param the type of the response * @return a bi consumer that will complete the wrapped listener */ - static BiConsumer toBiConsumer(ActionListener listener) { + static BiConsumer toBiConsumer(ActionListener listener) { return (response, throwable) -> { if (throwable == null) { listener.onResponse(response); } else { - if (throwable instanceof Exception) { - listener.onFailure((Exception) throwable); - } else if (throwable instanceof Error) { - throw (Error) throwable; - } else { - throw new AssertionError("Should have been either Error or Exception", throwable); - } + listener.onFailure(throwable); } }; } diff --git a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java index 8b814df8af920..37bf95d0b153a 100644 --- a/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/transport/MockTcpTransport.java @@ -36,6 +36,7 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.mocksocket.MockServerSocket; import org.elasticsearch.mocksocket.MockSocket; +import org.elasticsearch.common.concurrent.CompletableContext; import org.elasticsearch.threadpool.ThreadPool; import java.io.BufferedInputStream; @@ -50,7 +51,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; @@ -218,7 +218,7 @@ public final class MockChannel implements Closeable, TcpChannel { private final Socket activeChannel; private final String profile; private final CancellableThreads cancellableThreads = new CancellableThreads(); - private final CompletableFuture closeFuture = new CompletableFuture<>(); + private final CompletableContext closeFuture = new CompletableContext<>(); /** * Constructs a new MockChannel instance intended for handling the actual incoming / outgoing traffic. @@ -364,7 +364,7 @@ public String getProfile() { @Override public void addCloseListener(ActionListener listener) { - closeFuture.whenComplete(ActionListener.toBiConsumer(listener)); + closeFuture.addListener(ActionListener.toBiConsumer(listener)); } @Override diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java index 075e68183933f..171507de7413e 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java @@ -159,7 +159,7 @@ public void closeFromSelector() throws IOException { private static class CloseNotifyOperation implements WriteOperation { - private static final BiConsumer LISTENER = (v, t) -> {}; + private static final BiConsumer LISTENER = (v, t) -> {}; private static final Object WRITE_OBJECT = new Object(); private final SocketChannelContext channelContext; @@ -168,7 +168,7 @@ private CloseNotifyOperation(SocketChannelContext channelContext) { } @Override - public BiConsumer getListener() { + public BiConsumer getListener() { return LISTENER; } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java index fc501c68922e5..168dcd64e6c5f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java @@ -41,7 +41,7 @@ public class SSLChannelContextTests extends ESTestCase { private SSLChannelContext context; private InboundChannelBuffer channelBuffer; private SocketSelector selector; - private BiConsumer listener; + private BiConsumer listener; private Consumer exceptionHandler; private SSLDriver sslDriver; private ByteBuffer readBuffer = ByteBuffer.allocate(1 << 14); @@ -266,7 +266,7 @@ public void testPartialFlush() throws IOException { @SuppressWarnings("unchecked") public void testMultipleWritesPartialFlushes() throws IOException { - BiConsumer listener2 = mock(BiConsumer.class); + BiConsumer listener2 = mock(BiConsumer.class); ByteBuffer[] buffers1 = {ByteBuffer.allocate(10)}; ByteBuffer[] buffers2 = {ByteBuffer.allocate(5)}; FlushReadyWrite flushOperation1 = mock(FlushReadyWrite.class); From 4bd2607597f7ca73ed6380e4b9478df19cbc280b Mon Sep 17 00:00:00 2001 From: Andy Bristol Date: Thu, 24 May 2018 17:05:21 -0700 Subject: [PATCH 18/20] [docs] explainer for java packaging tests (#30825) --- qa/vagrant/README.md | 119 ++++++++++++++++++ .../packaging/test/ArchiveTestCase.java | 5 +- 2 files changed, 121 insertions(+), 3 deletions(-) create mode 100644 qa/vagrant/README.md diff --git a/qa/vagrant/README.md b/qa/vagrant/README.md new file mode 100644 index 0000000000000..ce253a2e3e002 --- /dev/null +++ b/qa/vagrant/README.md @@ -0,0 +1,119 @@ +# packaging tests + +This project contains tests that verify the distributions we build work +correctly on the operating systems we support. They're intended to cover the +steps a user would take when installing and configuring an Elasticsearch +distribution. They're not intended to have significant coverage of the behavior +of Elasticsearch's features. + +There are two types of tests in this project. The old tests live in +`src/test/` and are written in [Bats](https://github.com/sstephenson/bats), +which is a flavor of bash scripts that run as unit tests. These tests are +deprecated because Bats is unmaintained and cannot run on Windows. + +The new tests live in `src/main/` and are written in Java. Like the old tests, +this project's tests are run inside the VM, not on your host. All new packaging +tests should be added to this set of tests if possible. + +## Running these tests + +See the section in [TESTING.asciidoc](../../TESTING.asciidoc#testing-packaging) + +## Adding a new test class + +When gradle runs the packaging tests on a VM, it runs the full suite by +default. To add a test class to the suite, add its `class` to the +`@SuiteClasses` annotation in [PackagingTests.java](src/main/java/org/elasticsearch/packaging/PackagingTests.java). +If a test class is added to the project but not to this annotation, it will not +run in CI jobs. The test classes are run in the order they are listed in the +annotation. + +## Choosing which distributions to test + +Distributions are represented by [enum values](src/main/java/org/elasticsearch/packaging/util/Distribution.java) +which know if they are compatible with the platform the tests are currently +running on. To skip a test if the distribution it's using isn't compatible with +the current platform, put this [assumption](https://github.com/junit-team/junit4/wiki/assumptions-with-assume) +in your test method or in a `@Before` method + +```java +assumeTrue(distribution.packaging.compatible); +``` + +Similarly if you write a test that is intended only for particular platforms, +you can make an assumption using the constants and methods in [Platforms.java](src/main/java/org/elasticsearch/packaging/util/Platforms.java) + +```java +assumeTrue("only run on windows", Platforms.WINDOWS); + +assumeTrue("only run if using systemd", Platforms.isSystemd()); +``` + +## Writing a test that covers multiple distributions + +It seems like the way to do this that makes it the most straightforward to run +and reproduce specific test cases is to create a test case class with an +abstract method that provides the distribution + +```java +public abstract class MyTestCase { + @Test + public void myTest() { /* do something with the value of #distribution() */ } + abstract Distribution distribution(); +} +``` + +and then for each distribution you want to test, create a subclass + +```java +public class MyTestDefaultTar extends MyTestCase { + @Override + Distribution distribution() { return Distribution.DEFAULT_TAR; } +} +``` + +That way when a test fails the user gets told explicitly that `MyTestDefaultTar` +failed, and to reproduce it they should run that class. See [ArchiveTestCase](src/main/java/org/elasticsearch/packaging/test/ArchiveTestCase.java) +and its children for an example of this. + +## Running external commands + +In general it's probably best to avoid running external commands when a good +Java alternative exists. For example most filesystem operations can be done with +the java.nio.file APIs. For those that aren't, use an instance of [Shell](src/main/java/org/elasticsearch/packaging/util/Shell.java) + +Despite the name, commands run with this class are not run in a shell, and any +familiar features of shells like variables or expansion won't work. + +If you do need the shell, you must explicitly invoke the shell's command. For +example to run a command with Bash, use the `bash -c command` syntax. Note that +the entire script must be in a single string argument + +```java +Shell sh = new Shell(); +sh.run("bash", "-c", "echo $foo; echo $bar"); +``` + +Similary for powershell - again, the entire powershell script must go in a +single string argument + +```java +sh.run("powershell.exe", "-Command", "Write-Host $foo; Write-Host $bar"); +``` + +On Linux, most commands you'll want to use will be executable files and will +work fine without a shell + +```java +sh.run("tar", "-xzpf", "elasticsearch-6.1.0.tar.gz"); +``` + +On Windows you'll mostly want to use powershell as it can do a lot more and +gives much better feedback than Windows' legacy command line. Unfortunately that +means that you'll need to use the `powershell.exe -Command` syntax as +powershell's [Cmdlets](https://msdn.microsoft.com/en-us/library/ms714395.aspx) +don't correspond to executable files and are not runnable by `Runtime` directly. + +When writing powershell commands this way, make sure to test them as some types +of formatting can cause it to return a successful exit code but not run +anything. diff --git a/qa/vagrant/src/main/java/org/elasticsearch/packaging/test/ArchiveTestCase.java b/qa/vagrant/src/main/java/org/elasticsearch/packaging/test/ArchiveTestCase.java index f683cb9c145db..ab4a11922cc21 100644 --- a/qa/vagrant/src/main/java/org/elasticsearch/packaging/test/ArchiveTestCase.java +++ b/qa/vagrant/src/main/java/org/elasticsearch/packaging/test/ArchiveTestCase.java @@ -31,8 +31,7 @@ import static org.elasticsearch.packaging.util.Cleanup.cleanEverything; import static org.elasticsearch.packaging.util.Archives.installArchive; import static org.elasticsearch.packaging.util.Archives.verifyArchiveInstallation; -import static org.hamcrest.CoreMatchers.is; -import static org.junit.Assume.assumeThat; +import static org.junit.Assume.assumeTrue; /** * Tests that apply to the archive distributions (tar, zip). To add a case for a distribution, subclass and @@ -54,7 +53,7 @@ public static void cleanup() { @Before public void onlyCompatibleDistributions() { - assumeThat(distribution().packaging.compatible, is(true)); + assumeTrue("only compatible distributions", distribution().packaging.compatible); } @Test From e1ffbeb824bd66a715f9b7f2c94d9d5a0df5bd96 Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Thu, 24 May 2018 19:21:57 -0500 Subject: [PATCH 19/20] Fix bad version check writing Repository nodes (#30846) The writeTo method of VerifyRepositoryResponse incorrectly used its local version to determine what it was receiving, rather than the sender's version. This fixes a bug that ocassionally happened when a 6.4 master node sent data to a 7.0 client, causing the number of bytes to be improperly read. This also unmutes the test. Closes #30807 --- .../rest-api-spec/test/snapshot.get_repository/10_basic.yml | 3 --- .../cluster/repositories/verify/VerifyRepositoryResponse.java | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get_repository/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get_repository/10_basic.yml index 47f5ac0934ce9..b944fe43791e4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get_repository/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/snapshot.get_repository/10_basic.yml @@ -51,9 +51,6 @@ setup: --- "Verify created repository": - - skip: - version: " - 6.99.99" - reason: AwaitsFix for https://github.com/elastic/elasticsearch/issues/30807 - do: snapshot.verify_repository: repository: test_repo_get_2 diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java index c3fb2d58bebf3..b399923b2025b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java @@ -151,7 +151,7 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - if (Version.CURRENT.onOrAfter(Version.V_7_0_0_alpha1)) { + if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) { out.writeList(getNodes()); } else { clusterName.writeTo(out); From ae2f021f1c86cb0e6fb3a074958a78ece9912cd0 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 25 May 2018 07:17:50 +0200 Subject: [PATCH 20/20] Move score script context from SearchScript to its own class (#30816) --- .../expression/ExpressionScriptEngine.java | 43 ++++++++ .../expertscript/ExpertScriptPlugin.java | 16 +-- .../search/function/ScriptScoreFunction.java | 10 +- .../ScriptScoreFunctionBuilder.java | 5 +- .../org/elasticsearch/script/ScoreScript.java | 102 ++++++++++++++++++ .../elasticsearch/script/ScriptModule.java | 2 +- .../elasticsearch/script/SearchScript.java | 2 - .../functionscore/ExplainableScriptIT.java | 38 +++---- .../script/MockScriptEngine.java | 45 +++++++- 9 files changed, 221 insertions(+), 42 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/script/ScoreScript.java diff --git a/modules/lang-expression/src/main/java/org/elasticsearch/script/expression/ExpressionScriptEngine.java b/modules/lang-expression/src/main/java/org/elasticsearch/script/expression/ExpressionScriptEngine.java index b50eb788c6f57..1cde9c258b4f1 100644 --- a/modules/lang-expression/src/main/java/org/elasticsearch/script/expression/ExpressionScriptEngine.java +++ b/modules/lang-expression/src/main/java/org/elasticsearch/script/expression/ExpressionScriptEngine.java @@ -23,8 +23,10 @@ import org.apache.lucene.expressions.SimpleBindings; import org.apache.lucene.expressions.js.JavascriptCompiler; import org.apache.lucene.expressions.js.VariableContext; +import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.queries.function.ValueSource; import org.apache.lucene.queries.function.valuesource.DoubleConstValueSource; +import org.apache.lucene.search.Scorer; import org.apache.lucene.search.SortField; import org.elasticsearch.SpecialPermission; import org.elasticsearch.common.Nullable; @@ -39,12 +41,14 @@ import org.elasticsearch.script.ClassPermission; import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.FilterScript; +import org.elasticsearch.script.ScoreScript; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptEngine; import org.elasticsearch.script.ScriptException; import org.elasticsearch.script.SearchScript; import org.elasticsearch.search.lookup.SearchLookup; +import java.io.IOException; import java.security.AccessControlContext; import java.security.AccessController; import java.security.PrivilegedAction; @@ -111,6 +115,9 @@ protected Class loadClass(String name, boolean resolve) throws ClassNotFoundE } else if (context.instanceClazz.equals(FilterScript.class)) { FilterScript.Factory factory = (p, lookup) -> newFilterScript(expr, lookup, p); return context.factoryClazz.cast(factory); + } else if (context.instanceClazz.equals(ScoreScript.class)) { + ScoreScript.Factory factory = (p, lookup) -> newScoreScript(expr, lookup, p); + return context.factoryClazz.cast(factory); } throw new IllegalArgumentException("expression engine does not know how to handle script context [" + context.name + "]"); } @@ -260,6 +267,42 @@ public void setDocument(int docid) { }; }; } + + private ScoreScript.LeafFactory newScoreScript(Expression expr, SearchLookup lookup, @Nullable Map vars) { + SearchScript.LeafFactory searchLeafFactory = newSearchScript(expr, lookup, vars); + return new ScoreScript.LeafFactory() { + @Override + public boolean needs_score() { + return searchLeafFactory.needs_score(); + } + + @Override + public ScoreScript newInstance(LeafReaderContext ctx) throws IOException { + SearchScript script = searchLeafFactory.newInstance(ctx); + return new ScoreScript(vars, lookup, ctx) { + @Override + public double execute() { + return script.runAsDouble(); + } + + @Override + public void setDocument(int docid) { + script.setDocument(docid); + } + + @Override + public void setScorer(Scorer scorer) { + script.setScorer(scorer); + } + + @Override + public double get_score() { + return script.getScore(); + } + }; + } + }; + } /** * converts a ParseException at compile-time or link-time to a ScriptException diff --git a/plugins/examples/script-expert-scoring/src/main/java/org/elasticsearch/example/expertscript/ExpertScriptPlugin.java b/plugins/examples/script-expert-scoring/src/main/java/org/elasticsearch/example/expertscript/ExpertScriptPlugin.java index b910526ef3d98..cead97696a028 100644 --- a/plugins/examples/script-expert-scoring/src/main/java/org/elasticsearch/example/expertscript/ExpertScriptPlugin.java +++ b/plugins/examples/script-expert-scoring/src/main/java/org/elasticsearch/example/expertscript/ExpertScriptPlugin.java @@ -30,9 +30,9 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.ScriptPlugin; +import org.elasticsearch.script.ScoreScript; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptEngine; -import org.elasticsearch.script.SearchScript; /** * An example script plugin that adds a {@link ScriptEngine} implementing expert scoring. @@ -54,12 +54,12 @@ public String getType() { @Override public T compile(String scriptName, String scriptSource, ScriptContext context, Map params) { - if (context.equals(SearchScript.SCRIPT_SCORE_CONTEXT) == false) { + if (context.equals(ScoreScript.CONTEXT) == false) { throw new IllegalArgumentException(getType() + " scripts cannot be used for context [" + context.name + "]"); } // we use the script "source" as the script identifier if ("pure_df".equals(scriptSource)) { - SearchScript.Factory factory = (p, lookup) -> new SearchScript.LeafFactory() { + ScoreScript.Factory factory = (p, lookup) -> new ScoreScript.LeafFactory() { final String field; final String term; { @@ -74,18 +74,18 @@ public T compile(String scriptName, String scriptSource, ScriptContext co } @Override - public SearchScript newInstance(LeafReaderContext context) throws IOException { + public ScoreScript newInstance(LeafReaderContext context) throws IOException { PostingsEnum postings = context.reader().postings(new Term(field, term)); if (postings == null) { // the field and/or term don't exist in this segment, so always return 0 - return new SearchScript(p, lookup, context) { + return new ScoreScript(p, lookup, context) { @Override - public double runAsDouble() { + public double execute() { return 0.0d; } }; } - return new SearchScript(p, lookup, context) { + return new ScoreScript(p, lookup, context) { int currentDocid = -1; @Override public void setDocument(int docid) { @@ -100,7 +100,7 @@ public void setDocument(int docid) { currentDocid = docid; } @Override - public double runAsDouble() { + public double execute() { if (postings.docID() != currentDocid) { // advance moved past the current doc, so this doc has no occurrences of the term return 0.0d; diff --git a/server/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java b/server/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java index bcca4c4a03580..7f8b10349bc7d 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/search/function/ScriptScoreFunction.java @@ -24,8 +24,8 @@ import org.apache.lucene.search.Explanation; import org.apache.lucene.search.Scorer; import org.elasticsearch.script.ExplainableSearchScript; +import org.elasticsearch.script.ScoreScript; import org.elasticsearch.script.Script; -import org.elasticsearch.script.SearchScript; import java.io.IOException; import java.util.Objects; @@ -58,10 +58,10 @@ public DocIdSetIterator iterator() { private final Script sScript; - private final SearchScript.LeafFactory script; + private final ScoreScript.LeafFactory script; - public ScriptScoreFunction(Script sScript, SearchScript.LeafFactory script) { + public ScriptScoreFunction(Script sScript, ScoreScript.LeafFactory script) { super(CombineFunction.REPLACE); this.sScript = sScript; this.script = script; @@ -69,7 +69,7 @@ public ScriptScoreFunction(Script sScript, SearchScript.LeafFactory script) { @Override public LeafScoreFunction getLeafScoreFunction(LeafReaderContext ctx) throws IOException { - final SearchScript leafScript = script.newInstance(ctx); + final ScoreScript leafScript = script.newInstance(ctx); final CannedScorer scorer = new CannedScorer(); leafScript.setScorer(scorer); return new LeafScoreFunction() { @@ -78,7 +78,7 @@ public double score(int docId, float subQueryScore) throws IOException { leafScript.setDocument(docId); scorer.docid = docId; scorer.score = subQueryScore; - double result = leafScript.runAsDouble(); + double result = leafScript.execute(); return result; } diff --git a/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreFunctionBuilder.java b/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreFunctionBuilder.java index ed4c5f5a26952..9592ffe0b1fe5 100644 --- a/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreFunctionBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/functionscore/ScriptScoreFunctionBuilder.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.query.QueryShardException; +import org.elasticsearch.script.ScoreScript; import org.elasticsearch.script.Script; import org.elasticsearch.script.SearchScript; @@ -92,8 +93,8 @@ protected int doHashCode() { @Override protected ScoreFunction doToFunction(QueryShardContext context) { try { - SearchScript.Factory factory = context.getScriptService().compile(script, SearchScript.SCRIPT_SCORE_CONTEXT); - SearchScript.LeafFactory searchScript = factory.newFactory(script.getParams(), context.lookup()); + ScoreScript.Factory factory = context.getScriptService().compile(script, ScoreScript.CONTEXT); + ScoreScript.LeafFactory searchScript = factory.newFactory(script.getParams(), context.lookup()); return new ScriptScoreFunction(script, searchScript); } catch (Exception e) { throw new QueryShardException(context, "script_score: the script could not be loaded", e); diff --git a/server/src/main/java/org/elasticsearch/script/ScoreScript.java b/server/src/main/java/org/elasticsearch/script/ScoreScript.java new file mode 100644 index 0000000000000..d9e56d5573cae --- /dev/null +++ b/server/src/main/java/org/elasticsearch/script/ScoreScript.java @@ -0,0 +1,102 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.script; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.Scorer; +import org.elasticsearch.index.fielddata.ScriptDocValues; +import org.elasticsearch.search.lookup.LeafSearchLookup; +import org.elasticsearch.search.lookup.SearchLookup; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Map; +import java.util.function.DoubleSupplier; + +/** + * A script used for adjusting the score on a per document basis. + */ +public abstract class ScoreScript { + + public static final String[] PARAMETERS = new String[]{}; + + /** The generic runtime parameters for the script. */ + private final Map params; + + /** A leaf lookup for the bound segment this script will operate on. */ + private final LeafSearchLookup leafLookup; + + private DoubleSupplier scoreSupplier = () -> 0.0; + + public ScoreScript(Map params, SearchLookup lookup, LeafReaderContext leafContext) { + this.params = params; + this.leafLookup = lookup.getLeafSearchLookup(leafContext); + } + + public abstract double execute(); + + /** Return the parameters for this script. */ + public Map getParams() { + return params; + } + + /** The doc lookup for the Lucene segment this script was created for. */ + public final Map> getDoc() { + return leafLookup.doc(); + } + + /** Set the current document to run the script on next. */ + public void setDocument(int docid) { + leafLookup.setDocument(docid); + } + + public void setScorer(Scorer scorer) { + this.scoreSupplier = () -> { + try { + return scorer.score(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }; + } + + public double get_score() { + return scoreSupplier.getAsDouble(); + } + + /** A factory to construct {@link ScoreScript} instances. */ + public interface LeafFactory { + + /** + * Return {@code true} if the script needs {@code _score} calculated, or {@code false} otherwise. + */ + boolean needs_score(); + + ScoreScript newInstance(LeafReaderContext ctx) throws IOException; + } + + /** A factory to construct stateful {@link ScoreScript} factories for a specific index. */ + public interface Factory { + + ScoreScript.LeafFactory newFactory(Map params, SearchLookup lookup); + + } + + public static final ScriptContext CONTEXT = new ScriptContext<>("score", ScoreScript.Factory.class); +} diff --git a/server/src/main/java/org/elasticsearch/script/ScriptModule.java b/server/src/main/java/org/elasticsearch/script/ScriptModule.java index 583421be8e581..7074d3ad9fe44 100644 --- a/server/src/main/java/org/elasticsearch/script/ScriptModule.java +++ b/server/src/main/java/org/elasticsearch/script/ScriptModule.java @@ -42,7 +42,7 @@ public class ScriptModule { CORE_CONTEXTS = Stream.of( SearchScript.CONTEXT, SearchScript.AGGS_CONTEXT, - SearchScript.SCRIPT_SCORE_CONTEXT, + ScoreScript.CONTEXT, SearchScript.SCRIPT_SORT_CONTEXT, SearchScript.TERMS_SET_QUERY_CONTEXT, ExecutableScript.CONTEXT, diff --git a/server/src/main/java/org/elasticsearch/script/SearchScript.java b/server/src/main/java/org/elasticsearch/script/SearchScript.java index e5762adb1bbe9..43ea020aa6e24 100644 --- a/server/src/main/java/org/elasticsearch/script/SearchScript.java +++ b/server/src/main/java/org/elasticsearch/script/SearchScript.java @@ -162,8 +162,6 @@ public interface Factory { public static final ScriptContext AGGS_CONTEXT = new ScriptContext<>("aggs", Factory.class); // Can return a double. (For ScriptSortType#NUMBER only, for ScriptSortType#STRING normal CONTEXT should be used) public static final ScriptContext SCRIPT_SORT_CONTEXT = new ScriptContext<>("sort", Factory.class); - // Can return a float - public static final ScriptContext SCRIPT_SCORE_CONTEXT = new ScriptContext<>("score", Factory.class); // Can return a long public static final ScriptContext TERMS_SET_QUERY_CONTEXT = new ScriptContext<>("terms_set", Factory.class); } diff --git a/server/src/test/java/org/elasticsearch/search/functionscore/ExplainableScriptIT.java b/server/src/test/java/org/elasticsearch/search/functionscore/ExplainableScriptIT.java index 842748107d1d1..6657ad9823ffe 100644 --- a/server/src/test/java/org/elasticsearch/search/functionscore/ExplainableScriptIT.java +++ b/server/src/test/java/org/elasticsearch/search/functionscore/ExplainableScriptIT.java @@ -30,14 +30,14 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.ScriptPlugin; import org.elasticsearch.script.ExplainableSearchScript; +import org.elasticsearch.script.ScoreScript; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptContext; import org.elasticsearch.script.ScriptEngine; import org.elasticsearch.script.ScriptType; -import org.elasticsearch.script.SearchScript; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; -import org.elasticsearch.search.lookup.LeafDocLookup; +import org.elasticsearch.search.lookup.SearchLookup; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; @@ -76,16 +76,17 @@ public String getType() { @Override public T compile(String scriptName, String scriptSource, ScriptContext context, Map params) { assert scriptSource.equals("explainable_script"); - assert context == SearchScript.SCRIPT_SCORE_CONTEXT; - SearchScript.Factory factory = (p, lookup) -> new SearchScript.LeafFactory() { - @Override - public SearchScript newInstance(LeafReaderContext context) throws IOException { - return new MyScript(lookup.doc().getLeafDocLookup(context)); - } + assert context == ScoreScript.CONTEXT; + ScoreScript.Factory factory = (params1, lookup) -> new ScoreScript.LeafFactory() { @Override public boolean needs_score() { return false; } + + @Override + public ScoreScript newInstance(LeafReaderContext ctx) throws IOException { + return new MyScript(params1, lookup, ctx); + } }; return context.factoryClazz.cast(factory); } @@ -93,28 +94,21 @@ public boolean needs_score() { } } - static class MyScript extends SearchScript implements ExplainableSearchScript { - LeafDocLookup docLookup; + static class MyScript extends ScoreScript implements ExplainableSearchScript { - MyScript(LeafDocLookup docLookup) { - super(null, null, null); - this.docLookup = docLookup; + MyScript(Map params, SearchLookup lookup, LeafReaderContext leafContext) { + super(params, lookup, leafContext); } - - @Override - public void setDocument(int doc) { - docLookup.setDocument(doc); - } - + @Override public Explanation explain(Explanation subQueryScore) throws IOException { Explanation scoreExp = Explanation.match(subQueryScore.getValue(), "_score: ", subQueryScore); - return Explanation.match((float) (runAsDouble()), "This script returned " + runAsDouble(), scoreExp); + return Explanation.match((float) (execute()), "This script returned " + execute(), scoreExp); } @Override - public double runAsDouble() { - return ((Number) ((ScriptDocValues) docLookup.get("number_field")).getValues().get(0)).doubleValue(); + public double execute() { + return ((Number) ((ScriptDocValues) getDoc().get("number_field")).getValues().get(0)).doubleValue(); } } diff --git a/test/framework/src/main/java/org/elasticsearch/script/MockScriptEngine.java b/test/framework/src/main/java/org/elasticsearch/script/MockScriptEngine.java index 00303b344b92a..b86cb9ff29352 100644 --- a/test/framework/src/main/java/org/elasticsearch/script/MockScriptEngine.java +++ b/test/framework/src/main/java/org/elasticsearch/script/MockScriptEngine.java @@ -25,7 +25,6 @@ import org.elasticsearch.index.similarity.ScriptedSimilarity.Field; import org.elasticsearch.index.similarity.ScriptedSimilarity.Query; import org.elasticsearch.index.similarity.ScriptedSimilarity.Term; -import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.search.aggregations.pipeline.movfn.MovingFunctionScript; import org.elasticsearch.search.aggregations.pipeline.movfn.MovingFunctions; import org.elasticsearch.search.lookup.LeafSearchLookup; @@ -36,7 +35,6 @@ import java.util.HashMap; import java.util.Map; import java.util.function.Function; -import java.util.function.Predicate; import static java.util.Collections.emptyMap; @@ -114,6 +112,9 @@ public String execute() { } else if (context.instanceClazz.equals(MovingFunctionScript.class)) { MovingFunctionScript.Factory factory = mockCompiled::createMovingFunctionScript; return context.factoryClazz.cast(factory); + } else if (context.instanceClazz.equals(ScoreScript.class)) { + ScoreScript.Factory factory = new MockScoreScript(script); + return context.factoryClazz.cast(factory); } throw new IllegalArgumentException("mock script engine does not know how to handle context [" + context.name + "]"); } @@ -342,5 +343,45 @@ public double execute(Map params, double[] values) { return MovingFunctions.unweightedAvg(values); } } + + public class MockScoreScript implements ScoreScript.Factory { + + private final Function, Object> scripts; + + MockScoreScript(Function, Object> scripts) { + this.scripts = scripts; + } + + @Override + public ScoreScript.LeafFactory newFactory(Map params, SearchLookup lookup) { + return new ScoreScript.LeafFactory() { + @Override + public boolean needs_score() { + return true; + } + + @Override + public ScoreScript newInstance(LeafReaderContext ctx) throws IOException { + Scorer[] scorerHolder = new Scorer[1]; + return new ScoreScript(params, lookup, ctx) { + @Override + public double execute() { + Map vars = new HashMap<>(getParams()); + vars.put("doc", getDoc()); + if (scorerHolder[0] != null) { + vars.put("_score", new ScoreAccessor(scorerHolder[0])); + } + return ((Number) scripts.apply(vars)).doubleValue(); + } + + @Override + public void setScorer(Scorer scorer) { + scorerHolder[0] = scorer; + } + }; + } + }; + } + } }